diff --git a/docs/generated/sql/bnf/alter_table_set_storage_param.bnf b/docs/generated/sql/bnf/alter_table_set_storage_param.bnf
index 9046b8639fcb..96228772c943 100644
--- a/docs/generated/sql/bnf/alter_table_set_storage_param.bnf
+++ b/docs/generated/sql/bnf/alter_table_set_storage_param.bnf
@@ -1,3 +1,3 @@
 alter_onetable_stmt ::=
-	'ALTER' 'TABLE' table_name 'SET' '(' storage_parameter_key '=' var_value ) ) )* ')'
-	| 'ALTER' 'TABLE' 'IF' 'EXISTS' table_name 'SET' '(' storage_parameter_key '=' var_value ) ) )* ')'
+	'ALTER' 'TABLE' table_name 'SET' '(' storage_parameter_key '=' var_value ')'
+	| 'ALTER' 'TABLE' 'IF' 'EXISTS' table_name 'SET' '(' storage_parameter_key '=' var_value ')'
diff --git a/pkg/ccl/backupccl/backupinfo/backup_metadata.go b/pkg/ccl/backupccl/backupinfo/backup_metadata.go
index ac074f8e023c..61d97da7e10f 100644
--- a/pkg/ccl/backupccl/backupinfo/backup_metadata.go
+++ b/pkg/ccl/backupccl/backupinfo/backup_metadata.go
@@ -618,7 +618,7 @@ func debugDumpFileSST(
 		}
 		encOpts = &roachpb.FileEncryptionOptions{Key: key}
 	}
-	iter, err := storageccl.ExternalSSTReader(ctx, store, fileInfoPath, encOpts)
+	iter, err := storageccl.DeprecatingExternalSSTReader(ctx, store, fileInfoPath, encOpts)
 	if err != nil {
 		return err
 	}
@@ -665,7 +665,7 @@ func DebugDumpMetadataSST(
 		encOpts = &roachpb.FileEncryptionOptions{Key: key}
 	}
 
-	iter, err := storageccl.ExternalSSTReader(ctx, store, path, encOpts)
+	iter, err := storageccl.DeprecatingExternalSSTReader(ctx, store, path, encOpts)
 	if err != nil {
 		return err
 	}
@@ -805,7 +805,7 @@ func NewBackupMetadata(
 		encOpts = &roachpb.FileEncryptionOptions{Key: key}
 	}
 
-	iter, err := storageccl.ExternalSSTReader(ctx, exportStore, sstFileName, encOpts)
+	iter, err := storageccl.DeprecatingExternalSSTReader(ctx, exportStore, sstFileName, encOpts)
 	if err != nil {
 		return nil, err
 	}
@@ -922,7 +922,7 @@ func (b *BackupMetadata) FileIter(ctx context.Context) FileIterator {
 			break
 		}
 
-		iter, err := storageccl.ExternalSSTReader(ctx, b.store, path, encOpts)
+		iter, err := storageccl.DeprecatingExternalSSTReader(ctx, b.store, path, encOpts)
 		if err != nil {
 			return FileIterator{err: err}
 		}
@@ -1232,7 +1232,7 @@ func makeBytesIter(
 		encOpts = &roachpb.FileEncryptionOptions{Key: key}
 	}
 
-	iter, err := storageccl.ExternalSSTReader(ctx, store, path, encOpts)
+	iter, err := storageccl.DeprecatingExternalSSTReader(ctx, store, path, encOpts)
 	if err != nil {
 		return bytesIter{iterError: err}
 	}
diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go
index 0c9ca8de4e06..00e62c7c4054 100644
--- a/pkg/ccl/backupccl/restore_data_processor.go
+++ b/pkg/ccl/backupccl/restore_data_processor.go
@@ -278,18 +278,13 @@ func (rd *restoreDataProcessor) openSSTs(
 ) error {
 	ctxDone := ctx.Done()
 
-	// The sstables only contain MVCC data and no intents, so using an MVCC
-	// iterator is sufficient.
-	var iters []storage.SimpleMVCCIterator
+	// TODO(msbutler): use a a map of external storage factories to avoid reopening the same dir
+	// in a given restore span entry
 	var dirs []cloud.ExternalStorage
 
 	// If we bail early and haven't handed off responsibility of the dirs/iters to
 	// the channel, close anything that we had open.
 	defer func() {
-		for _, iter := range iters {
-			iter.Close()
-		}
-
 		for _, dir := range dirs {
 			if err := dir.Close(); err != nil {
 				log.Warningf(ctx, "close export storage failed %v", err)
@@ -297,18 +292,13 @@ func (rd *restoreDataProcessor) openSSTs(
 		}
 	}()
 
-	// sendIters sends all of the currently accumulated iterators over the
+	// sendIter sends a multiplexed iterator covering the currently accumulated files over the
 	// channel.
-	sendIters := func(itersToSend []storage.SimpleMVCCIterator, dirsToSend []cloud.ExternalStorage) error {
-		multiIter := storage.MakeMultiIterator(itersToSend)
-		readAsOfIter := storage.NewReadAsOfIterator(multiIter, rd.spec.RestoreTime)
+	sendIter := func(iter storage.SimpleMVCCIterator, dirsToSend []cloud.ExternalStorage) error {
+		readAsOfIter := storage.NewReadAsOfIterator(iter, rd.spec.RestoreTime)
 
 		cleanup := func() {
 			readAsOfIter.Close()
-			multiIter.Close()
-			for _, iter := range itersToSend {
-				iter.Close()
-			}
 
 			for _, dir := range dirsToSend {
 				if err := dir.Close(); err != nil {
@@ -329,13 +319,13 @@ func (rd *restoreDataProcessor) openSSTs(
 			return ctx.Err()
 		}
 
-		iters = make([]storage.SimpleMVCCIterator, 0)
 		dirs = make([]cloud.ExternalStorage, 0)
 		return nil
 	}
 
 	log.VEventf(ctx, 1 /* level */, "ingesting span [%s-%s)", entry.Span.Key, entry.Span.EndKey)
 
+	filePaths := make([]string, 0, len(EntryFiles{}))
 	for _, file := range entry.Files {
 		log.VEventf(ctx, 2, "import file %s which starts at %s", file.Path, entry.Span.Key)
 
@@ -344,17 +334,23 @@ func (rd *restoreDataProcessor) openSSTs(
 			return err
 		}
 		dirs = append(dirs, dir)
+		filePaths = append(filePaths, file.Path)
 
 		// TODO(pbardea): When memory monitoring is added, send the currently
 		// accumulated iterators on the channel if we run into memory pressure.
-		iter, err := storageccl.ExternalSSTReader(ctx, dir, file.Path, rd.spec.Encryption)
-		if err != nil {
-			return err
-		}
-		iters = append(iters, iter)
 	}
-
-	return sendIters(iters, dirs)
+	iterOpts := storage.IterOptions{
+		RangeKeyMaskingBelow: rd.spec.RestoreTime,
+		KeyTypes:             storage.IterKeyTypePointsAndRanges,
+		LowerBound:           keys.LocalMax,
+		UpperBound:           keys.MaxKey,
+	}
+	iter, err := storageccl.ExternalSSTReader(ctx, dirs, filePaths, rd.spec.Encryption,
+		iterOpts)
+	if err != nil {
+		return err
+	}
+	return sendIter(iter, dirs)
 }
 
 func (rd *restoreDataProcessor) runRestoreWorkers(ctx context.Context, ssts chan mergedSST) error {
diff --git a/pkg/ccl/cliccl/debug_backup.go b/pkg/ccl/cliccl/debug_backup.go
index 42459e3a5c32..74a9d7d63c9a 100644
--- a/pkg/ccl/cliccl/debug_backup.go
+++ b/pkg/ccl/cliccl/debug_backup.go
@@ -591,7 +591,7 @@ func makeIters(
 			return nil, nil, errors.Wrapf(err, "making external storage")
 		}
 
-		iters[i], err = storageccl.ExternalSSTReader(ctx, dirStorage[i], file.Path, nil)
+		iters[i], err = storageccl.DeprecatingExternalSSTReader(ctx, dirStorage[i], file.Path, nil)
 		if err != nil {
 			return nil, nil, errors.Wrapf(err, "fetching sst reader")
 		}
diff --git a/pkg/ccl/storageccl/external_sst_reader.go b/pkg/ccl/storageccl/external_sst_reader.go
index 5d8e4f7dcc56..c5a8e73f7e1f 100644
--- a/pkg/ccl/storageccl/external_sst_reader.go
+++ b/pkg/ccl/storageccl/external_sst_reader.go
@@ -25,6 +25,8 @@ import (
 	"github.com/cockroachdb/pebble/sstable"
 )
 
+// RemoteSSTs lets external SSTables get iterated directly in some cases,
+// rather than being downloaded entirely first.
 var remoteSSTs = settings.RegisterBoolSetting(
 	settings.TenantWritable,
 	"kv.bulk_ingest.stream_external_ssts.enabled",
@@ -39,12 +41,126 @@ var remoteSSTSuffixCacheSize = settings.RegisterByteSizeSetting(
 	64<<10,
 )
 
-// ExternalSSTReader returns opens an SST in external storage, optionally
-// decrypting with the supplied parameters, and returns iterator over it.
+func getFileWithRetry(
+	ctx context.Context, basename string, e cloud.ExternalStorage,
+) (ioctx.ReadCloserCtx, int64, error) {
+	// Do an initial read of the file, from the beginning, to get the file size as
+	// this is used e.g. to read the trailer.
+	var f ioctx.ReadCloserCtx
+	var sz int64
+	const maxAttempts = 3
+	if err := retry.WithMaxAttempts(ctx, base.DefaultRetryOptions(), maxAttempts, func() error {
+		var err error
+		f, sz, err = e.ReadFileAt(ctx, basename, 0)
+		return err
+	}); err != nil {
+		return nil, 0, err
+	}
+	return f, sz, nil
+}
+
+// newMemPebbleSSTReader returns a PebbleSSTIterator for in-memory SSTs from
+// external storage, optionally decrypting with the supplied parameters.
+//
+// ctx is captured and used throughout the life of the returned iterator, until
+// the iterator's Close() method is called.
+func newMemPebbleSSTReader(
+	ctx context.Context,
+	e []cloud.ExternalStorage,
+	basenames []string,
+	encryption *roachpb.FileEncryptionOptions,
+	iterOps storage.IterOptions,
+) (storage.SimpleMVCCIterator, error) {
+
+	inMemorySSTs := make([][]byte, 0, len(basenames))
+
+	for i, basename := range basenames {
+		f, _, err := getFileWithRetry(ctx, basename, e[i])
+		if err != nil {
+			return nil, err
+		}
+		content, err := ioctx.ReadAll(ctx, f)
+		f.Close(ctx)
+		if err != nil {
+			return nil, err
+		}
+		if encryption != nil {
+			content, err = DecryptFile(ctx, content, encryption.Key, nil /* mm */)
+			if err != nil {
+				return nil, err
+			}
+		}
+		inMemorySSTs = append(inMemorySSTs, content)
+	}
+	return storage.NewPebbleMultiMemSSTIterator(inMemorySSTs, false, iterOps)
+}
+
+// ExternalSSTReader returns a PebbleSSTIterator for the SSTs in external storage,
+// optionally decrypting with the supplied parameters.
 //
 // ctx is captured and used throughout the life of the returned iterator, until
 // the iterator's Close() method is called.
 func ExternalSSTReader(
+	ctx context.Context,
+	e []cloud.ExternalStorage,
+	basenames []string,
+	encryption *roachpb.FileEncryptionOptions,
+	iterOps storage.IterOptions,
+) (storage.SimpleMVCCIterator, error) {
+	if !remoteSSTs.Get(&e[0].Settings().SV) {
+		return newMemPebbleSSTReader(ctx, e, basenames, encryption, iterOps)
+	}
+	remoteCacheSize := remoteSSTSuffixCacheSize.Get(&e[0].Settings().SV)
+	readers := make([]sstable.ReadableFile, 0, len(basenames))
+
+	for i, basename := range basenames {
+		f, sz, err := getFileWithRetry(ctx, basename, e[i])
+		if err != nil {
+			return nil, err
+		}
+
+		raw := &sstReader{
+			ctx:  ctx,
+			sz:   sizeStat(sz),
+			body: f,
+			openAt: func(offset int64) (ioctx.ReadCloserCtx, error) {
+				reader, _, err := e[i].ReadFileAt(ctx, basename, offset)
+				return reader, err
+			},
+		}
+
+		var reader sstable.ReadableFile
+
+		if encryption != nil {
+			r, err := decryptingReader(raw, encryption.Key)
+			if err != nil {
+				f.Close(ctx)
+				return nil, err
+			}
+			reader = r
+		} else {
+			// We only explicitly buffer the suffix of the file when not decrypting as
+			// the decrypting reader has its own internal block buffer.
+			if err := raw.readAndCacheSuffix(remoteCacheSize); err != nil {
+				f.Close(ctx)
+				return nil, err
+			}
+			reader = raw
+		}
+		readers = append(readers, reader)
+	}
+	return storage.NewPebbleSSTIterator(readers, iterOps)
+}
+
+// DeprecatingExternalSSTReader returns opens an SST in external storage, optionally
+// decrypting with the supplied parameters, and returns iterator over it.
+//
+// ctx is captured and used throughout the life of the returned iterator, until
+// the iterator's Close() method is called.
+//
+// TODO (msbutler): replace all current calls with new ExternalSSTReader,
+// as it does not handle range keys
+func DeprecatingExternalSSTReader(
 	ctx context.Context,
 	e cloud.ExternalStorage,
 	basename string,
diff --git a/pkg/cmd/docgen/diagrams.go b/pkg/cmd/docgen/diagrams.go
index f6e0bdc8290b..e509f42e6ed5 100644
--- a/pkg/cmd/docgen/diagrams.go
+++ b/pkg/cmd/docgen/diagrams.go
@@ -423,7 +423,7 @@ var specs = []stmtSpec{
 		stmt:   "alter_onetable_stmt",
 		inline: []string{"alter_table_cmds", "alter_table_cmd", "storage_parameter_list", "storage_parameter"},
 		regreplace: map[string]string{
-			`var_value.*`: `var_value ) ) )* ')'`,
+			`var_value.*`: `var_value ')'`,
 		},
 		match: []*regexp.Regexp{regexp.MustCompile("relation_expr 'SET")},
 		replace: map[string]string{
@@ -1311,7 +1311,7 @@ var specs = []stmtSpec{
 			"'BACKUP' string_or_placeholder 'IN' string_or_placeholder": "'BACKUP' subdirectory 'IN' location",
 			"'BACKUP' 'SCHEMAS' string_or_placeholder":                  "'BACKUP' 'SCHEMAS' location",
 		},
-		unlink: []string{"location"},
+		unlink: []string{"subdirectory", "location"},
 	},
 	{
 		name:    "show_jobs",
diff --git a/pkg/cmd/roachtest/tests/gorm.go b/pkg/cmd/roachtest/tests/gorm.go
index 5ed6a6f7b00a..4a7b25ca804c 100644
--- a/pkg/cmd/roachtest/tests/gorm.go
+++ b/pkg/cmd/roachtest/tests/gorm.go
@@ -24,7 +24,7 @@ import (
 )
 
 var gormReleaseTag = regexp.MustCompile(`^v(?P<major>\d+)\.(?P<minor>\d+)\.(?P<point>\d+)$`)
-var gormSupportedTag = "v1.23.5"
+var gormSupportedTag = "v1.23.8"
 
 func registerGORM(r registry.Registry) {
 	runGORM := func(ctx context.Context, t test.Test, c cluster.Cluster) {
diff --git a/pkg/kv/kvclient/kvstreamer/streamer.go b/pkg/kv/kvclient/kvstreamer/streamer.go
index 2c19df5070ce..37154fda23b0 100644
--- a/pkg/kv/kvclient/kvstreamer/streamer.go
+++ b/pkg/kv/kvclient/kvstreamer/streamer.go
@@ -1567,6 +1567,14 @@ func buildResumeSingleRangeBatch(
 	// requests with the ResumeSpans.
 	resumeReq.reqsReservedBytes = fp.resumeReqsMemUsage
 	resumeReq.overheadAccountedFor = req.overheadAccountedFor
+	// Note that due to limitations of the KV layer (#75452) we cannot reuse
+	// original requests because the KV doesn't allow mutability (and all
+	// requests are modified by txnSeqNumAllocator, even if they are not
+	// evaluated due to TargetBytes limit).
+	gets := make([]struct {
+		req   roachpb.GetRequest
+		union roachpb.RequestUnion_Get
+	}, fp.numIncompleteGets)
 	scans := make([]struct {
 		req   roachpb.ScanRequest
 		union roachpb.RequestUnion_Scan
@@ -1583,18 +1591,14 @@ func buildResumeSingleRangeBatch(
 				emptyResponse = false
 				continue
 			}
-			// This Get wasn't completed - include it into the batch again (we
-			// can just reuse the original request since it hasn't been
-			// modified which is also asserted below).
-			if buildutil.CrdbTestBuild {
-				if origSpan := req.reqs[i].GetInner().Header().Span(); !get.ResumeSpan.Equal(origSpan) {
-					panic(errors.AssertionFailedf(
-						"unexpectedly the ResumeSpan %s on the GetResponse is different from the original span %s",
-						get.ResumeSpan, origSpan,
-					))
-				}
-			}
-			resumeReq.reqs[resumeReqIdx] = req.reqs[i]
+			// This Get wasn't completed - create a new request according to the
+			// ResumeSpan and include it into the batch.
+			newGet := gets[0]
+			gets = gets[1:]
+			newGet.req.SetSpan(*get.ResumeSpan)
+			newGet.req.KeyLocking = s.keyLocking
+			newGet.union.Get = &newGet.req
+			resumeReq.reqs[resumeReqIdx].Value = &newGet.union
 			resumeReq.positions = append(resumeReq.positions, position)
 			if req.subRequestIdx != nil {
 				resumeReq.subRequestIdx = append(resumeReq.subRequestIdx, req.subRequestIdx[i])
diff --git a/pkg/sql/colexec/colexecbase/BUILD.bazel b/pkg/sql/colexec/colexecbase/BUILD.bazel
index af1a757dc542..93242de98557 100644
--- a/pkg/sql/colexec/colexecbase/BUILD.bazel
+++ b/pkg/sql/colexec/colexecbase/BUILD.bazel
@@ -34,6 +34,7 @@ go_library(
         "//pkg/util/duration",  # keep
         "//pkg/util/json",  # keep
         "//pkg/util/log",
+        "//pkg/util/timeutil/pgdate",  # keep
         "//pkg/util/uuid",  # keep
         "@com_github_cockroachdb_apd_v3//:apd",  # keep
         "@com_github_cockroachdb_errors//:errors",
diff --git a/pkg/sql/colexec/colexecbase/cast.eg.go b/pkg/sql/colexec/colexecbase/cast.eg.go
index f753073ac91e..7507dd6d39c3 100644
--- a/pkg/sql/colexec/colexecbase/cast.eg.go
+++ b/pkg/sql/colexec/colexecbase/cast.eg.go
@@ -14,6 +14,7 @@ import (
 	"context"
 	"fmt"
 	"math"
+	"strconv"
 	"strings"
 	"time"
 
@@ -26,12 +27,15 @@ import (
 	"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
 	"github.com/cockroachdb/cockroach/pkg/sql/colmem"
 	"github.com/cockroachdb/cockroach/pkg/sql/lex"
+	"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
+	"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
 	"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
 	"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
 	"github.com/cockroachdb/cockroach/pkg/sql/types"
 	"github.com/cockroachdb/cockroach/pkg/util"
 	"github.com/cockroachdb/cockroach/pkg/util/duration"
 	"github.com/cockroachdb/cockroach/pkg/util/json"
+	"github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
 	"github.com/cockroachdb/cockroach/pkg/util/uuid"
 	"github.com/cockroachdb/errors"
 	"github.com/lib/pq/oid"
@@ -46,6 +50,9 @@ var (
 	_ = uuid.FromBytes
 	_ = oid.T_name
 	_ = util.TruncateString
+	_ = pgcode.Syntax
+	_ = pgdate.ParseTimestamp
+	_ = pgerror.Wrapf
 )
 
 func isIdentityCast(fromType, toType *types.T) bool {
@@ -200,6 +207,48 @@ func GetCastOperator(
 					}
 				}
 			}
+		case types.BytesFamily:
+			switch fromType.Width() {
+			case -1:
+			default:
+				switch toType.Family() {
+				case types.UuidFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return &castBytesUuidOp{castOpBase: base}, nil
+					}
+				}
+			}
+		case types.DateFamily:
+			switch fromType.Width() {
+			case -1:
+			default:
+				switch toType.Family() {
+				case types.DecimalFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return &castDateDecimalOp{castOpBase: base}, nil
+					}
+				case types.FloatFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return &castDateFloatOp{castOpBase: base}, nil
+					}
+				case types.IntFamily:
+					switch toType.Width() {
+					case 16:
+						return &castDateInt2Op{castOpBase: base}, nil
+					case 32:
+						return &castDateInt4Op{castOpBase: base}, nil
+					case -1:
+					default:
+						return &castDateIntOp{castOpBase: base}, nil
+					}
+				}
+			}
 		case types.DecimalFamily:
 			switch fromType.Width() {
 			case -1:
@@ -211,6 +260,18 @@ func GetCastOperator(
 					default:
 						return &castDecimalBoolOp{castOpBase: base}, nil
 					}
+				case types.DecimalFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return &castDecimalDecimalOp{castOpBase: base}, nil
+					}
+				case types.FloatFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return &castDecimalFloatOp{castOpBase: base}, nil
+					}
 				case types.IntFamily:
 					switch toType.Width() {
 					case 16:
@@ -221,32 +282,41 @@ func GetCastOperator(
 					default:
 						return &castDecimalIntOp{castOpBase: base}, nil
 					}
-				case types.FloatFamily:
+				}
+			}
+		case types.FloatFamily:
+			switch fromType.Width() {
+			case -1:
+			default:
+				switch toType.Family() {
+				case types.BoolFamily:
 					switch toType.Width() {
 					case -1:
 					default:
-						return &castDecimalFloatOp{castOpBase: base}, nil
+						return &castFloatBoolOp{castOpBase: base}, nil
 					}
 				case types.DecimalFamily:
 					switch toType.Width() {
 					case -1:
 					default:
-						return &castDecimalDecimalOp{castOpBase: base}, nil
+						return &castFloatDecimalOp{castOpBase: base}, nil
 					}
-				}
-			}
-		case types.IntFamily:
-			switch fromType.Width() {
-			case 16:
-				switch toType.Family() {
 				case types.IntFamily:
 					switch toType.Width() {
+					case 16:
+						return &castFloatInt2Op{castOpBase: base}, nil
 					case 32:
-						return &castInt2Int4Op{castOpBase: base}, nil
+						return &castFloatInt4Op{castOpBase: base}, nil
 					case -1:
 					default:
-						return &castInt2IntOp{castOpBase: base}, nil
+						return &castFloatIntOp{castOpBase: base}, nil
 					}
+				}
+			}
+		case types.IntFamily:
+			switch fromType.Width() {
+			case 16:
+				switch toType.Family() {
 				case types.BoolFamily:
 					switch toType.Width() {
 					case -1:
@@ -265,17 +335,17 @@ func GetCastOperator(
 					default:
 						return &castInt2FloatOp{castOpBase: base}, nil
 					}
-				}
-			case 32:
-				switch toType.Family() {
 				case types.IntFamily:
 					switch toType.Width() {
-					case 16:
-						return &castInt4Int2Op{castOpBase: base}, nil
+					case 32:
+						return &castInt2Int4Op{castOpBase: base}, nil
 					case -1:
 					default:
-						return &castInt4IntOp{castOpBase: base}, nil
+						return &castInt2IntOp{castOpBase: base}, nil
 					}
+				}
+			case 32:
+				switch toType.Family() {
 				case types.BoolFamily:
 					switch toType.Width() {
 					case -1:
@@ -294,17 +364,18 @@ func GetCastOperator(
 					default:
 						return &castInt4FloatOp{castOpBase: base}, nil
 					}
-				}
-			case -1:
-			default:
-				switch toType.Family() {
 				case types.IntFamily:
 					switch toType.Width() {
 					case 16:
-						return &castIntInt2Op{castOpBase: base}, nil
-					case 32:
-						return &castIntInt4Op{castOpBase: base}, nil
+						return &castInt4Int2Op{castOpBase: base}, nil
+					case -1:
+					default:
+						return &castInt4IntOp{castOpBase: base}, nil
 					}
+				}
+			case -1:
+			default:
+				switch toType.Family() {
 				case types.BoolFamily:
 					switch toType.Width() {
 					case -1:
@@ -323,95 +394,84 @@ func GetCastOperator(
 					default:
 						return &castIntFloatOp{castOpBase: base}, nil
 					}
+				case types.IntFamily:
+					switch toType.Width() {
+					case 16:
+						return &castIntInt2Op{castOpBase: base}, nil
+					case 32:
+						return &castIntInt4Op{castOpBase: base}, nil
+					}
 				}
 			}
-		case types.FloatFamily:
+		case types.JsonFamily:
 			switch fromType.Width() {
 			case -1:
 			default:
 				switch toType.Family() {
-				case types.BoolFamily:
+				case types.StringFamily:
 					switch toType.Width() {
 					case -1:
 					default:
-						return &castFloatBoolOp{castOpBase: base}, nil
+						return &castJsonbStringOp{castOpBase: base}, nil
 					}
-				case types.DecimalFamily:
+				}
+			}
+		case types.StringFamily:
+			switch fromType.Width() {
+			case -1:
+			default:
+				switch toType.Family() {
+				case types.BoolFamily:
 					switch toType.Width() {
 					case -1:
 					default:
-						return &castFloatDecimalOp{castOpBase: base}, nil
+						return &castStringBoolOp{castOpBase: base}, nil
 					}
-				case types.IntFamily:
+				case types.BytesFamily:
 					switch toType.Width() {
-					case 16:
-						return &castFloatInt2Op{castOpBase: base}, nil
-					case 32:
-						return &castFloatInt4Op{castOpBase: base}, nil
 					case -1:
 					default:
-						return &castFloatIntOp{castOpBase: base}, nil
+						return &castStringBytesOp{castOpBase: base}, nil
 					}
-				}
-			}
-		case types.DateFamily:
-			switch fromType.Width() {
-			case -1:
-			default:
-				switch toType.Family() {
-				case types.IntFamily:
+				case types.DateFamily:
 					switch toType.Width() {
-					case 16:
-						return &castDateInt2Op{castOpBase: base}, nil
-					case 32:
-						return &castDateInt4Op{castOpBase: base}, nil
 					case -1:
 					default:
-						return &castDateIntOp{castOpBase: base}, nil
+						return &castStringDateOp{castOpBase: base}, nil
 					}
-				case types.FloatFamily:
+				case types.DecimalFamily:
 					switch toType.Width() {
 					case -1:
 					default:
-						return &castDateFloatOp{castOpBase: base}, nil
+						return &castStringDecimalOp{castOpBase: base}, nil
 					}
-				case types.DecimalFamily:
+				case types.FloatFamily:
 					switch toType.Width() {
 					case -1:
 					default:
-						return &castDateDecimalOp{castOpBase: base}, nil
+						return &castStringFloatOp{castOpBase: base}, nil
 					}
-				}
-			}
-		case types.BytesFamily:
-			switch fromType.Width() {
-			case -1:
-			default:
-				switch toType.Family() {
-				case types.UuidFamily:
+				case types.IntFamily:
 					switch toType.Width() {
+					case 16:
+						return &castStringInt2Op{castOpBase: base}, nil
+					case 32:
+						return &castStringInt4Op{castOpBase: base}, nil
 					case -1:
 					default:
-						return &castBytesUuidOp{castOpBase: base}, nil
+						return &castStringIntOp{castOpBase: base}, nil
 					}
-				}
-			}
-		case types.StringFamily:
-			switch fromType.Width() {
-			case -1:
-			default:
-				switch toType.Family() {
-				case types.BoolFamily:
+				case types.IntervalFamily:
 					switch toType.Width() {
 					case -1:
 					default:
-						return &castStringBoolOp{castOpBase: base}, nil
+						return &castStringIntervalOp{castOpBase: base}, nil
 					}
-				case types.BytesFamily:
+				case types.JsonFamily:
 					switch toType.Width() {
 					case -1:
 					default:
-						return &castStringBytesOp{castOpBase: base}, nil
+						return &castStringJsonbOp{castOpBase: base}, nil
 					}
 				case types.StringFamily:
 					switch toType.Width() {
@@ -419,24 +479,23 @@ func GetCastOperator(
 					default:
 						return &castStringStringOp{castOpBase: base}, nil
 					}
-				case types.UuidFamily:
+				case types.TimestampFamily:
 					switch toType.Width() {
 					case -1:
 					default:
-						return &castStringUuidOp{castOpBase: base}, nil
+						return &castStringTimestampOp{castOpBase: base}, nil
 					}
-				}
-			}
-		case types.JsonFamily:
-			switch fromType.Width() {
-			case -1:
-			default:
-				switch toType.Family() {
-				case types.StringFamily:
+				case types.TimestampTZFamily:
 					switch toType.Width() {
 					case -1:
 					default:
-						return &castJsonbStringOp{castOpBase: base}, nil
+						return &castStringTimestamptzOp{castOpBase: base}, nil
+					}
+				case types.UuidFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return &castStringUuidOp{castOpBase: base}, nil
 					}
 				}
 			}
@@ -564,23 +623,26 @@ func IsCastSupported(fromType, toType *types.T) bool {
 					}
 				}
 			}
-		case types.DecimalFamily:
+		case types.BytesFamily:
 			switch fromType.Width() {
 			case -1:
 			default:
 				switch toType.Family() {
-				case types.BoolFamily:
+				case types.UuidFamily:
 					switch toType.Width() {
 					case -1:
 					default:
 						return true
 					}
-				case types.IntFamily:
+				}
+			}
+		case types.DateFamily:
+			switch fromType.Width() {
+			case -1:
+			default:
+				switch toType.Family() {
+				case types.DecimalFamily:
 					switch toType.Width() {
-					case 16:
-						return true
-					case 32:
-						return true
 					case -1:
 					default:
 						return true
@@ -591,26 +653,23 @@ func IsCastSupported(fromType, toType *types.T) bool {
 					default:
 						return true
 					}
-				case types.DecimalFamily:
+				case types.IntFamily:
 					switch toType.Width() {
+					case 16:
+						return true
+					case 32:
+						return true
 					case -1:
 					default:
 						return true
 					}
 				}
 			}
-		case types.IntFamily:
+		case types.DecimalFamily:
 			switch fromType.Width() {
-			case 16:
+			case -1:
+			default:
 				switch toType.Family() {
-				case types.IntFamily:
-					switch toType.Width() {
-					case 32:
-						return true
-					case -1:
-					default:
-						return true
-					}
 				case types.BoolFamily:
 					switch toType.Width() {
 					case -1:
@@ -629,17 +688,23 @@ func IsCastSupported(fromType, toType *types.T) bool {
 					default:
 						return true
 					}
-				}
-			case 32:
-				switch toType.Family() {
 				case types.IntFamily:
 					switch toType.Width() {
 					case 16:
 						return true
+					case 32:
+						return true
 					case -1:
 					default:
 						return true
 					}
+				}
+			}
+		case types.FloatFamily:
+			switch fromType.Width() {
+			case -1:
+			default:
+				switch toType.Family() {
 				case types.BoolFamily:
 					switch toType.Width() {
 					case -1:
@@ -652,23 +717,22 @@ func IsCastSupported(fromType, toType *types.T) bool {
 					default:
 						return true
 					}
-				case types.FloatFamily:
+				case types.IntFamily:
 					switch toType.Width() {
-					case -1:
-					default:
-						return true
-					}
-				}
-			case -1:
-			default:
-				switch toType.Family() {
-				case types.IntFamily:
-					switch toType.Width() {
-					case 16:
+					case 16:
 						return true
 					case 32:
 						return true
+					case -1:
+					default:
+						return true
 					}
+				}
+			}
+		case types.IntFamily:
+			switch fromType.Width() {
+			case 16:
+				switch toType.Family() {
 				case types.BoolFamily:
 					switch toType.Width() {
 					case -1:
@@ -687,12 +751,16 @@ func IsCastSupported(fromType, toType *types.T) bool {
 					default:
 						return true
 					}
+				case types.IntFamily:
+					switch toType.Width() {
+					case 32:
+						return true
+					case -1:
+					default:
+						return true
+					}
 				}
-			}
-		case types.FloatFamily:
-			switch fromType.Width() {
-			case -1:
-			default:
+			case 32:
 				switch toType.Family() {
 				case types.BoolFamily:
 					switch toType.Width() {
@@ -706,53 +774,57 @@ func IsCastSupported(fromType, toType *types.T) bool {
 					default:
 						return true
 					}
+				case types.FloatFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return true
+					}
 				case types.IntFamily:
 					switch toType.Width() {
 					case 16:
 						return true
-					case 32:
-						return true
 					case -1:
 					default:
 						return true
 					}
 				}
-			}
-		case types.DateFamily:
-			switch fromType.Width() {
 			case -1:
 			default:
 				switch toType.Family() {
-				case types.IntFamily:
+				case types.BoolFamily:
 					switch toType.Width() {
-					case 16:
-						return true
-					case 32:
-						return true
 					case -1:
 					default:
 						return true
 					}
-				case types.FloatFamily:
+				case types.DecimalFamily:
 					switch toType.Width() {
 					case -1:
 					default:
 						return true
 					}
-				case types.DecimalFamily:
+				case types.FloatFamily:
 					switch toType.Width() {
 					case -1:
 					default:
 						return true
 					}
+				case types.IntFamily:
+					switch toType.Width() {
+					case 16:
+						return true
+					case 32:
+						return true
+					}
 				}
 			}
-		case types.BytesFamily:
+		case types.JsonFamily:
 			switch fromType.Width() {
 			case -1:
 			default:
 				switch toType.Family() {
-				case types.UuidFamily:
+				case types.StringFamily:
 					switch toType.Width() {
 					case -1:
 					default:
@@ -777,31 +849,70 @@ func IsCastSupported(fromType, toType *types.T) bool {
 					default:
 						return true
 					}
-				case types.StringFamily:
+				case types.DateFamily:
 					switch toType.Width() {
 					case -1:
 					default:
 						return true
 					}
-				case types.UuidFamily:
+				case types.DecimalFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return true
+					}
+				case types.FloatFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return true
+					}
+				case types.IntFamily:
+					switch toType.Width() {
+					case 16:
+						return true
+					case 32:
+						return true
+					case -1:
+					default:
+						return true
+					}
+				case types.IntervalFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return true
+					}
+				case types.JsonFamily:
 					switch toType.Width() {
 					case -1:
 					default:
 						return true
 					}
-				}
-			}
-		case types.JsonFamily:
-			switch fromType.Width() {
-			case -1:
-			default:
-				switch toType.Family() {
 				case types.StringFamily:
 					switch toType.Width() {
 					case -1:
 					default:
 						return true
 					}
+				case types.TimestampFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return true
+					}
+				case types.TimestampTZFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return true
+					}
+				case types.UuidFamily:
+					switch toType.Width() {
+					case -1:
+					default:
+						return true
+					}
 				}
 			}
 		}
@@ -1377,11 +1488,1155 @@ func (c *castBoolInt4Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r int32
+
+							r = 0
+							if v {
+								r = 1
+							}
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r int32
+
+							r = 0
+							if v {
+								r = 1
+							}
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			}
+		},
+	)
+	return batch
+}
+
+type castBoolIntOp struct {
+	castOpBase
+}
+
+var _ colexecop.ResettableOperator = &castBoolIntOp{}
+var _ colexecop.ClosableOperator = &castBoolIntOp{}
+
+func (c *castBoolIntOp) Next() coldata.Batch {
+	batch := c.Input.Next()
+	n := batch.Length()
+	if n == 0 {
+		return coldata.ZeroBatch
+	}
+	sel := batch.Selection()
+	inputVec := batch.ColVec(c.colIdx)
+	outputVec := batch.ColVec(c.outputIdx)
+	toType := outputVec.Type()
+	// Remove unused warnings.
+	_ = toType
+	c.allocator.PerformOperation(
+		[]coldata.Vec{outputVec}, func() {
+			inputCol := inputVec.Bool()
+			inputNulls := inputVec.Nulls()
+			outputCol := outputVec.Int64()
+			outputNulls := outputVec.Nulls()
+			if inputVec.MaybeHasNulls() {
+				outputNulls.Copy(inputNulls)
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r int64
+
+							r = 0
+							if v {
+								r = 1
+							}
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r int64
+
+							r = 0
+							if v {
+								r = 1
+							}
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			} else {
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r int64
+
+							r = 0
+							if v {
+								r = 1
+							}
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r int64
+
+							r = 0
+							if v {
+								r = 1
+							}
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			}
+		},
+	)
+	return batch
+}
+
+type castBytesUuidOp struct {
+	castOpBase
+}
+
+var _ colexecop.ResettableOperator = &castBytesUuidOp{}
+var _ colexecop.ClosableOperator = &castBytesUuidOp{}
+
+func (c *castBytesUuidOp) Next() coldata.Batch {
+	batch := c.Input.Next()
+	n := batch.Length()
+	if n == 0 {
+		return coldata.ZeroBatch
+	}
+	sel := batch.Selection()
+	inputVec := batch.ColVec(c.colIdx)
+	outputVec := batch.ColVec(c.outputIdx)
+	toType := outputVec.Type()
+	// Remove unused warnings.
+	_ = toType
+	c.allocator.PerformOperation(
+		[]coldata.Vec{outputVec}, func() {
+			inputCol := inputVec.Bytes()
+			inputNulls := inputVec.Nulls()
+			outputCol := outputVec.Bytes()
+			outputNulls := outputVec.Nulls()
+			if inputVec.MaybeHasNulls() {
+				outputNulls.Copy(inputNulls)
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r []byte
+
+							_uuid, err := uuid.FromBytes(v)
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							r = _uuid.GetBytes()
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r []byte
+
+							_uuid, err := uuid.FromBytes(v)
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							r = _uuid.GetBytes()
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			} else {
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r []byte
+
+							_uuid, err := uuid.FromBytes(v)
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							r = _uuid.GetBytes()
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r []byte
+
+							_uuid, err := uuid.FromBytes(v)
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							r = _uuid.GetBytes()
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			}
+		},
+	)
+	return batch
+}
+
+type castDateDecimalOp struct {
+	castOpBase
+}
+
+var _ colexecop.ResettableOperator = &castDateDecimalOp{}
+var _ colexecop.ClosableOperator = &castDateDecimalOp{}
+
+func (c *castDateDecimalOp) Next() coldata.Batch {
+	batch := c.Input.Next()
+	n := batch.Length()
+	if n == 0 {
+		return coldata.ZeroBatch
+	}
+	sel := batch.Selection()
+	inputVec := batch.ColVec(c.colIdx)
+	outputVec := batch.ColVec(c.outputIdx)
+	toType := outputVec.Type()
+	// Remove unused warnings.
+	_ = toType
+	c.allocator.PerformOperation(
+		[]coldata.Vec{outputVec}, func() {
+			inputCol := inputVec.Int64()
+			inputNulls := inputVec.Nulls()
+			outputCol := outputVec.Decimal()
+			outputNulls := outputVec.Nulls()
+			if inputVec.MaybeHasNulls() {
+				outputNulls.Copy(inputNulls)
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r apd.Decimal
+
+							r.SetInt64(int64(v))
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
+							}
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r apd.Decimal
+
+							r.SetInt64(int64(v))
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
+							}
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			} else {
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r apd.Decimal
+
+							r.SetInt64(int64(v))
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
+							}
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r apd.Decimal
+
+							r.SetInt64(int64(v))
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
+							}
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			}
+		},
+	)
+	return batch
+}
+
+type castDateFloatOp struct {
+	castOpBase
+}
+
+var _ colexecop.ResettableOperator = &castDateFloatOp{}
+var _ colexecop.ClosableOperator = &castDateFloatOp{}
+
+func (c *castDateFloatOp) Next() coldata.Batch {
+	batch := c.Input.Next()
+	n := batch.Length()
+	if n == 0 {
+		return coldata.ZeroBatch
+	}
+	sel := batch.Selection()
+	inputVec := batch.ColVec(c.colIdx)
+	outputVec := batch.ColVec(c.outputIdx)
+	toType := outputVec.Type()
+	// Remove unused warnings.
+	_ = toType
+	c.allocator.PerformOperation(
+		[]coldata.Vec{outputVec}, func() {
+			inputCol := inputVec.Int64()
+			inputNulls := inputVec.Nulls()
+			outputCol := outputVec.Float64()
+			outputNulls := outputVec.Nulls()
+			if inputVec.MaybeHasNulls() {
+				outputNulls.Copy(inputNulls)
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r float64
+
+							r = float64(v)
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r float64
+
+							r = float64(v)
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			} else {
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r float64
+
+							r = float64(v)
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r float64
+
+							r = float64(v)
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			}
+		},
+	)
+	return batch
+}
+
+type castDateInt2Op struct {
+	castOpBase
+}
+
+var _ colexecop.ResettableOperator = &castDateInt2Op{}
+var _ colexecop.ClosableOperator = &castDateInt2Op{}
+
+func (c *castDateInt2Op) Next() coldata.Batch {
+	batch := c.Input.Next()
+	n := batch.Length()
+	if n == 0 {
+		return coldata.ZeroBatch
+	}
+	sel := batch.Selection()
+	inputVec := batch.ColVec(c.colIdx)
+	outputVec := batch.ColVec(c.outputIdx)
+	toType := outputVec.Type()
+	// Remove unused warnings.
+	_ = toType
+	c.allocator.PerformOperation(
+		[]coldata.Vec{outputVec}, func() {
+			inputCol := inputVec.Int64()
+			inputNulls := inputVec.Nulls()
+			outputCol := outputVec.Int16()
+			outputNulls := outputVec.Nulls()
+			if inputVec.MaybeHasNulls() {
+				outputNulls.Copy(inputNulls)
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r int16
+
+							shifted := v >> uint(15)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							}
+							r = int16(v)
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r int16
+
+							shifted := v >> uint(15)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							}
+							r = int16(v)
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			} else {
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r int16
+
+							shifted := v >> uint(15)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							}
+							r = int16(v)
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r int16
+
+							shifted := v >> uint(15)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							}
+							r = int16(v)
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			}
+		},
+	)
+	return batch
+}
+
+type castDateInt4Op struct {
+	castOpBase
+}
+
+var _ colexecop.ResettableOperator = &castDateInt4Op{}
+var _ colexecop.ClosableOperator = &castDateInt4Op{}
+
+func (c *castDateInt4Op) Next() coldata.Batch {
+	batch := c.Input.Next()
+	n := batch.Length()
+	if n == 0 {
+		return coldata.ZeroBatch
+	}
+	sel := batch.Selection()
+	inputVec := batch.ColVec(c.colIdx)
+	outputVec := batch.ColVec(c.outputIdx)
+	toType := outputVec.Type()
+	// Remove unused warnings.
+	_ = toType
+	c.allocator.PerformOperation(
+		[]coldata.Vec{outputVec}, func() {
+			inputCol := inputVec.Int64()
+			inputNulls := inputVec.Nulls()
+			outputCol := outputVec.Int32()
+			outputNulls := outputVec.Nulls()
+			if inputVec.MaybeHasNulls() {
+				outputNulls.Copy(inputNulls)
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r int32
+
+							shifted := v >> uint(31)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+							}
+							r = int32(v)
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r int32
+
+							shifted := v >> uint(31)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+							}
+							r = int32(v)
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			} else {
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r int32
+
+							shifted := v >> uint(31)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+							}
+							r = int32(v)
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r int32
+
+							shifted := v >> uint(31)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+							}
+							r = int32(v)
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			}
+		},
+	)
+	return batch
+}
+
+type castDateIntOp struct {
+	castOpBase
+}
+
+var _ colexecop.ResettableOperator = &castDateIntOp{}
+var _ colexecop.ClosableOperator = &castDateIntOp{}
+
+func (c *castDateIntOp) Next() coldata.Batch {
+	batch := c.Input.Next()
+	n := batch.Length()
+	if n == 0 {
+		return coldata.ZeroBatch
+	}
+	sel := batch.Selection()
+	inputVec := batch.ColVec(c.colIdx)
+	outputVec := batch.ColVec(c.outputIdx)
+	toType := outputVec.Type()
+	// Remove unused warnings.
+	_ = toType
+	c.allocator.PerformOperation(
+		[]coldata.Vec{outputVec}, func() {
+			inputCol := inputVec.Int64()
+			inputNulls := inputVec.Nulls()
+			outputCol := outputVec.Int64()
+			outputNulls := outputVec.Nulls()
+			if inputVec.MaybeHasNulls() {
+				outputNulls.Copy(inputNulls)
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r int64
+							r = int64(v)
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r int64
+							r = int64(v)
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			} else {
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r int64
+							r = int64(v)
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r int64
+							r = int64(v)
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			}
+		},
+	)
+	return batch
+}
+
+type castDecimalBoolOp struct {
+	castOpBase
+}
+
+var _ colexecop.ResettableOperator = &castDecimalBoolOp{}
+var _ colexecop.ClosableOperator = &castDecimalBoolOp{}
+
+func (c *castDecimalBoolOp) Next() coldata.Batch {
+	batch := c.Input.Next()
+	n := batch.Length()
+	if n == 0 {
+		return coldata.ZeroBatch
+	}
+	sel := batch.Selection()
+	inputVec := batch.ColVec(c.colIdx)
+	outputVec := batch.ColVec(c.outputIdx)
+	toType := outputVec.Type()
+	// Remove unused warnings.
+	_ = toType
+	c.allocator.PerformOperation(
+		[]coldata.Vec{outputVec}, func() {
+			inputCol := inputVec.Decimal()
+			inputNulls := inputVec.Nulls()
+			outputCol := outputVec.Bool()
+			outputNulls := outputVec.Nulls()
+			if inputVec.MaybeHasNulls() {
+				outputNulls.Copy(inputNulls)
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r bool
+							r = v.Sign() != 0
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r bool
+							r = v.Sign() != 0
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			} else {
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r bool
+							r = v.Sign() != 0
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r bool
+							r = v.Sign() != 0
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			}
+		},
+	)
+	return batch
+}
+
+type castDecimalDecimalOp struct {
+	castOpBase
+}
+
+var _ colexecop.ResettableOperator = &castDecimalDecimalOp{}
+var _ colexecop.ClosableOperator = &castDecimalDecimalOp{}
+
+func (c *castDecimalDecimalOp) Next() coldata.Batch {
+	batch := c.Input.Next()
+	n := batch.Length()
+	if n == 0 {
+		return coldata.ZeroBatch
+	}
+	sel := batch.Selection()
+	inputVec := batch.ColVec(c.colIdx)
+	outputVec := batch.ColVec(c.outputIdx)
+	toType := outputVec.Type()
+	// Remove unused warnings.
+	_ = toType
+	c.allocator.PerformOperation(
+		[]coldata.Vec{outputVec}, func() {
+			inputCol := inputVec.Decimal()
+			inputNulls := inputVec.Nulls()
+			outputCol := outputVec.Decimal()
+			outputNulls := outputVec.Nulls()
+			if inputVec.MaybeHasNulls() {
+				outputNulls.Copy(inputNulls)
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r apd.Decimal
+
+							r.Set(&v)
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
+							}
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = inputCol.Get(n - 1)
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							//gcassert:bce
+							v := inputCol.Get(tupleIdx)
+							var r apd.Decimal
+
+							r.Set(&v)
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
+							}
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			} else {
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r apd.Decimal
 
-							r = 0
-							if v {
-								r = 1
+							r.Set(&v)
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -1402,11 +2657,11 @@ func (c *castBoolInt4Op) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r apd.Decimal
 
-							r = 0
-							if v {
-								r = 1
+							r.Set(&v)
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
 							}
 
 							//gcassert:bce
@@ -1420,14 +2675,14 @@ func (c *castBoolInt4Op) Next() coldata.Batch {
 	return batch
 }
 
-type castBoolIntOp struct {
+type castDecimalFloatOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castBoolIntOp{}
-var _ colexecop.ClosableOperator = &castBoolIntOp{}
+var _ colexecop.ResettableOperator = &castDecimalFloatOp{}
+var _ colexecop.ClosableOperator = &castDecimalFloatOp{}
 
-func (c *castBoolIntOp) Next() coldata.Batch {
+func (c *castDecimalFloatOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -1441,9 +2696,9 @@ func (c *castBoolIntOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Bool()
+			inputCol := inputVec.Decimal()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int64()
+			outputCol := outputVec.Float64()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -1459,11 +2714,14 @@ func (c *castBoolIntOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r float64
 
-							r = 0
-							if v {
-								r = 1
+							{
+								f, err := v.Float64()
+								if err != nil {
+									colexecerror.ExpectedError(tree.ErrFloatOutOfRange)
+								}
+								r = f
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -1484,11 +2742,14 @@ func (c *castBoolIntOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r float64
 
-							r = 0
-							if v {
-								r = 1
+							{
+								f, err := v.Float64()
+								if err != nil {
+									colexecerror.ExpectedError(tree.ErrFloatOutOfRange)
+								}
+								r = f
 							}
 
 							//gcassert:bce
@@ -1509,11 +2770,14 @@ func (c *castBoolIntOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r float64
 
-							r = 0
-							if v {
-								r = 1
+							{
+								f, err := v.Float64()
+								if err != nil {
+									colexecerror.ExpectedError(tree.ErrFloatOutOfRange)
+								}
+								r = f
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -1534,11 +2798,14 @@ func (c *castBoolIntOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r float64
 
-							r = 0
-							if v {
-								r = 1
+							{
+								f, err := v.Float64()
+								if err != nil {
+									colexecerror.ExpectedError(tree.ErrFloatOutOfRange)
+								}
+								r = f
 							}
 
 							//gcassert:bce
@@ -1552,14 +2819,14 @@ func (c *castBoolIntOp) Next() coldata.Batch {
 	return batch
 }
 
-type castDecimalBoolOp struct {
+type castDecimalInt2Op struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDecimalBoolOp{}
-var _ colexecop.ClosableOperator = &castDecimalBoolOp{}
+var _ colexecop.ResettableOperator = &castDecimalInt2Op{}
+var _ colexecop.ClosableOperator = &castDecimalInt2Op{}
 
-func (c *castDecimalBoolOp) Next() coldata.Batch {
+func (c *castDecimalInt2Op) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -1575,7 +2842,7 @@ func (c *castDecimalBoolOp) Next() coldata.Batch {
 		[]coldata.Vec{outputVec}, func() {
 			inputCol := inputVec.Decimal()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Bool()
+			outputCol := outputVec.Int16()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -1591,8 +2858,27 @@ func (c *castDecimalBoolOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r bool
-							r = v.Sign() != 0
+							var r int16
+
+							{
+								var tmpDec apd.Decimal //gcassert:noescape
+								_, err := tree.DecimalCtx.RoundToIntegralValue(&tmpDec, &v)
+								if err != nil {
+									colexecerror.ExpectedError(err)
+								}
+								_i, err := tmpDec.Int64()
+								if err != nil {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+
+								shifted := _i >> uint(15)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+								r = int16(_i)
+
+							}
+
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -1611,8 +2897,27 @@ func (c *castDecimalBoolOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r bool
-							r = v.Sign() != 0
+							var r int16
+
+							{
+								var tmpDec apd.Decimal //gcassert:noescape
+								_, err := tree.DecimalCtx.RoundToIntegralValue(&tmpDec, &v)
+								if err != nil {
+									colexecerror.ExpectedError(err)
+								}
+								_i, err := tmpDec.Int64()
+								if err != nil {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+
+								shifted := _i >> uint(15)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+								r = int16(_i)
+
+							}
+
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -1631,8 +2936,27 @@ func (c *castDecimalBoolOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r bool
-							r = v.Sign() != 0
+							var r int16
+
+							{
+								var tmpDec apd.Decimal //gcassert:noescape
+								_, err := tree.DecimalCtx.RoundToIntegralValue(&tmpDec, &v)
+								if err != nil {
+									colexecerror.ExpectedError(err)
+								}
+								_i, err := tmpDec.Int64()
+								if err != nil {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+
+								shifted := _i >> uint(15)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+								r = int16(_i)
+
+							}
+
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -1651,8 +2975,27 @@ func (c *castDecimalBoolOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r bool
-							r = v.Sign() != 0
+							var r int16
+
+							{
+								var tmpDec apd.Decimal //gcassert:noescape
+								_, err := tree.DecimalCtx.RoundToIntegralValue(&tmpDec, &v)
+								if err != nil {
+									colexecerror.ExpectedError(err)
+								}
+								_i, err := tmpDec.Int64()
+								if err != nil {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+
+								shifted := _i >> uint(15)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+								r = int16(_i)
+
+							}
+
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -1664,14 +3007,14 @@ func (c *castDecimalBoolOp) Next() coldata.Batch {
 	return batch
 }
 
-type castDecimalInt2Op struct {
+type castDecimalInt4Op struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDecimalInt2Op{}
-var _ colexecop.ClosableOperator = &castDecimalInt2Op{}
+var _ colexecop.ResettableOperator = &castDecimalInt4Op{}
+var _ colexecop.ClosableOperator = &castDecimalInt4Op{}
 
-func (c *castDecimalInt2Op) Next() coldata.Batch {
+func (c *castDecimalInt4Op) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -1687,7 +3030,7 @@ func (c *castDecimalInt2Op) Next() coldata.Batch {
 		[]coldata.Vec{outputVec}, func() {
 			inputCol := inputVec.Decimal()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int16()
+			outputCol := outputVec.Int32()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -1703,7 +3046,7 @@ func (c *castDecimalInt2Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r int32
 
 							{
 								var tmpDec apd.Decimal //gcassert:noescape
@@ -1713,14 +3056,14 @@ func (c *castDecimalInt2Op) Next() coldata.Batch {
 								}
 								_i, err := tmpDec.Int64()
 								if err != nil {
-									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 								}
 
-								shifted := _i >> uint(15)
+								shifted := _i >> uint(31)
 								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
-									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 								}
-								r = int16(_i)
+								r = int32(_i)
 
 							}
 
@@ -1742,7 +3085,7 @@ func (c *castDecimalInt2Op) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r int32
 
 							{
 								var tmpDec apd.Decimal //gcassert:noescape
@@ -1752,14 +3095,14 @@ func (c *castDecimalInt2Op) Next() coldata.Batch {
 								}
 								_i, err := tmpDec.Int64()
 								if err != nil {
-									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 								}
 
-								shifted := _i >> uint(15)
+								shifted := _i >> uint(31)
 								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
-									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 								}
-								r = int16(_i)
+								r = int32(_i)
 
 							}
 
@@ -1781,7 +3124,7 @@ func (c *castDecimalInt2Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r int32
 
 							{
 								var tmpDec apd.Decimal //gcassert:noescape
@@ -1791,14 +3134,14 @@ func (c *castDecimalInt2Op) Next() coldata.Batch {
 								}
 								_i, err := tmpDec.Int64()
 								if err != nil {
-									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 								}
 
-								shifted := _i >> uint(15)
+								shifted := _i >> uint(31)
 								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
-									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 								}
-								r = int16(_i)
+								r = int32(_i)
 
 							}
 
@@ -1820,7 +3163,7 @@ func (c *castDecimalInt2Op) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r int32
 
 							{
 								var tmpDec apd.Decimal //gcassert:noescape
@@ -1830,14 +3173,14 @@ func (c *castDecimalInt2Op) Next() coldata.Batch {
 								}
 								_i, err := tmpDec.Int64()
 								if err != nil {
-									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 								}
 
-								shifted := _i >> uint(15)
+								shifted := _i >> uint(31)
 								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
-									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 								}
-								r = int16(_i)
+								r = int32(_i)
 
 							}
 
@@ -1852,14 +3195,14 @@ func (c *castDecimalInt2Op) Next() coldata.Batch {
 	return batch
 }
 
-type castDecimalInt4Op struct {
+type castDecimalIntOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDecimalInt4Op{}
-var _ colexecop.ClosableOperator = &castDecimalInt4Op{}
+var _ colexecop.ResettableOperator = &castDecimalIntOp{}
+var _ colexecop.ClosableOperator = &castDecimalIntOp{}
 
-func (c *castDecimalInt4Op) Next() coldata.Batch {
+func (c *castDecimalIntOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -1875,7 +3218,7 @@ func (c *castDecimalInt4Op) Next() coldata.Batch {
 		[]coldata.Vec{outputVec}, func() {
 			inputCol := inputVec.Decimal()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int32()
+			outputCol := outputVec.Int64()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -1891,7 +3234,7 @@ func (c *castDecimalInt4Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r int64
 
 							{
 								var tmpDec apd.Decimal //gcassert:noescape
@@ -1901,15 +3244,9 @@ func (c *castDecimalInt4Op) Next() coldata.Batch {
 								}
 								_i, err := tmpDec.Int64()
 								if err != nil {
-									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
-								}
-
-								shifted := _i >> uint(31)
-								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
-									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+									colexecerror.ExpectedError(tree.ErrIntOutOfRange)
 								}
-								r = int32(_i)
-
+								r = int64(_i)
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -1930,7 +3267,7 @@ func (c *castDecimalInt4Op) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r int64
 
 							{
 								var tmpDec apd.Decimal //gcassert:noescape
@@ -1940,15 +3277,9 @@ func (c *castDecimalInt4Op) Next() coldata.Batch {
 								}
 								_i, err := tmpDec.Int64()
 								if err != nil {
-									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
-								}
-
-								shifted := _i >> uint(31)
-								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
-									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+									colexecerror.ExpectedError(tree.ErrIntOutOfRange)
 								}
-								r = int32(_i)
-
+								r = int64(_i)
 							}
 
 							//gcassert:bce
@@ -1969,7 +3300,7 @@ func (c *castDecimalInt4Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r int64
 
 							{
 								var tmpDec apd.Decimal //gcassert:noescape
@@ -1979,15 +3310,9 @@ func (c *castDecimalInt4Op) Next() coldata.Batch {
 								}
 								_i, err := tmpDec.Int64()
 								if err != nil {
-									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
-								}
-
-								shifted := _i >> uint(31)
-								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
-									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+									colexecerror.ExpectedError(tree.ErrIntOutOfRange)
 								}
-								r = int32(_i)
-
+								r = int64(_i)
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -2008,7 +3333,7 @@ func (c *castDecimalInt4Op) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r int64
 
 							{
 								var tmpDec apd.Decimal //gcassert:noescape
@@ -2018,15 +3343,9 @@ func (c *castDecimalInt4Op) Next() coldata.Batch {
 								}
 								_i, err := tmpDec.Int64()
 								if err != nil {
-									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
-								}
-
-								shifted := _i >> uint(31)
-								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
-									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+									colexecerror.ExpectedError(tree.ErrIntOutOfRange)
 								}
-								r = int32(_i)
-
+								r = int64(_i)
 							}
 
 							//gcassert:bce
@@ -2040,14 +3359,14 @@ func (c *castDecimalInt4Op) Next() coldata.Batch {
 	return batch
 }
 
-type castDecimalIntOp struct {
+type castFloatBoolOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDecimalIntOp{}
-var _ colexecop.ClosableOperator = &castDecimalIntOp{}
+var _ colexecop.ResettableOperator = &castFloatBoolOp{}
+var _ colexecop.ClosableOperator = &castFloatBoolOp{}
 
-func (c *castDecimalIntOp) Next() coldata.Batch {
+func (c *castFloatBoolOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -2061,9 +3380,9 @@ func (c *castDecimalIntOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Decimal()
+			inputCol := inputVec.Float64()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int64()
+			outputCol := outputVec.Bool()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -2079,20 +3398,9 @@ func (c *castDecimalIntOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r bool
 
-							{
-								var tmpDec apd.Decimal //gcassert:noescape
-								_, err := tree.DecimalCtx.RoundToIntegralValue(&tmpDec, &v)
-								if err != nil {
-									colexecerror.ExpectedError(err)
-								}
-								_i, err := tmpDec.Int64()
-								if err != nil {
-									colexecerror.ExpectedError(tree.ErrIntOutOfRange)
-								}
-								r = int64(_i)
-							}
+							r = v != 0
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -2112,20 +3420,9 @@ func (c *castDecimalIntOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r bool
 
-							{
-								var tmpDec apd.Decimal //gcassert:noescape
-								_, err := tree.DecimalCtx.RoundToIntegralValue(&tmpDec, &v)
-								if err != nil {
-									colexecerror.ExpectedError(err)
-								}
-								_i, err := tmpDec.Int64()
-								if err != nil {
-									colexecerror.ExpectedError(tree.ErrIntOutOfRange)
-								}
-								r = int64(_i)
-							}
+							r = v != 0
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -2145,20 +3442,9 @@ func (c *castDecimalIntOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r bool
 
-							{
-								var tmpDec apd.Decimal //gcassert:noescape
-								_, err := tree.DecimalCtx.RoundToIntegralValue(&tmpDec, &v)
-								if err != nil {
-									colexecerror.ExpectedError(err)
-								}
-								_i, err := tmpDec.Int64()
-								if err != nil {
-									colexecerror.ExpectedError(tree.ErrIntOutOfRange)
-								}
-								r = int64(_i)
-							}
+							r = v != 0
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -2178,20 +3464,9 @@ func (c *castDecimalIntOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r bool
 
-							{
-								var tmpDec apd.Decimal //gcassert:noescape
-								_, err := tree.DecimalCtx.RoundToIntegralValue(&tmpDec, &v)
-								if err != nil {
-									colexecerror.ExpectedError(err)
-								}
-								_i, err := tmpDec.Int64()
-								if err != nil {
-									colexecerror.ExpectedError(tree.ErrIntOutOfRange)
-								}
-								r = int64(_i)
-							}
+							r = v != 0
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -2204,14 +3479,14 @@ func (c *castDecimalIntOp) Next() coldata.Batch {
 	return batch
 }
 
-type castDecimalFloatOp struct {
+type castFloatDecimalOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDecimalFloatOp{}
-var _ colexecop.ClosableOperator = &castDecimalFloatOp{}
+var _ colexecop.ResettableOperator = &castFloatDecimalOp{}
+var _ colexecop.ClosableOperator = &castFloatDecimalOp{}
 
-func (c *castDecimalFloatOp) Next() coldata.Batch {
+func (c *castFloatDecimalOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -2225,9 +3500,9 @@ func (c *castDecimalFloatOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Decimal()
+			inputCol := inputVec.Float64()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Float64()
+			outputCol := outputVec.Decimal()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -2243,14 +3518,14 @@ func (c *castDecimalFloatOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r float64
+							var r apd.Decimal
 
-							{
-								f, err := v.Float64()
-								if err != nil {
-									colexecerror.ExpectedError(tree.ErrFloatOutOfRange)
-								}
-								r = f
+							if _, err := r.SetFloat64(float64(v)); err != nil {
+								colexecerror.ExpectedError(err)
+							}
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -2271,14 +3546,14 @@ func (c *castDecimalFloatOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r float64
+							var r apd.Decimal
 
-							{
-								f, err := v.Float64()
-								if err != nil {
-									colexecerror.ExpectedError(tree.ErrFloatOutOfRange)
-								}
-								r = f
+							if _, err := r.SetFloat64(float64(v)); err != nil {
+								colexecerror.ExpectedError(err)
+							}
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
 							}
 
 							//gcassert:bce
@@ -2299,14 +3574,14 @@ func (c *castDecimalFloatOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r float64
+							var r apd.Decimal
 
-							{
-								f, err := v.Float64()
-								if err != nil {
-									colexecerror.ExpectedError(tree.ErrFloatOutOfRange)
-								}
-								r = f
+							if _, err := r.SetFloat64(float64(v)); err != nil {
+								colexecerror.ExpectedError(err)
+							}
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -2327,14 +3602,14 @@ func (c *castDecimalFloatOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r float64
+							var r apd.Decimal
 
-							{
-								f, err := v.Float64()
-								if err != nil {
-									colexecerror.ExpectedError(tree.ErrFloatOutOfRange)
-								}
-								r = f
+							if _, err := r.SetFloat64(float64(v)); err != nil {
+								colexecerror.ExpectedError(err)
+							}
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
 							}
 
 							//gcassert:bce
@@ -2348,14 +3623,14 @@ func (c *castDecimalFloatOp) Next() coldata.Batch {
 	return batch
 }
 
-type castDecimalDecimalOp struct {
+type castFloatInt2Op struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDecimalDecimalOp{}
-var _ colexecop.ClosableOperator = &castDecimalDecimalOp{}
+var _ colexecop.ResettableOperator = &castFloatInt2Op{}
+var _ colexecop.ClosableOperator = &castFloatInt2Op{}
 
-func (c *castDecimalDecimalOp) Next() coldata.Batch {
+func (c *castFloatInt2Op) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -2369,9 +3644,9 @@ func (c *castDecimalDecimalOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Decimal()
+			inputCol := inputVec.Float64()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Decimal()
+			outputCol := outputVec.Int16()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -2387,12 +3662,12 @@ func (c *castDecimalDecimalOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
+							var r int16
 
-							r.Set(&v)
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt16) || v >= float64(math.MaxInt16) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
 							}
+							r = int16(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -2412,12 +3687,12 @@ func (c *castDecimalDecimalOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
+							var r int16
 
-							r.Set(&v)
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt16) || v >= float64(math.MaxInt16) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
 							}
+							r = int16(v)
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -2437,12 +3712,12 @@ func (c *castDecimalDecimalOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
+							var r int16
 
-							r.Set(&v)
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt16) || v >= float64(math.MaxInt16) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
 							}
+							r = int16(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -2462,12 +3737,12 @@ func (c *castDecimalDecimalOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
+							var r int16
 
-							r.Set(&v)
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt16) || v >= float64(math.MaxInt16) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
 							}
+							r = int16(v)
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -2480,14 +3755,14 @@ func (c *castDecimalDecimalOp) Next() coldata.Batch {
 	return batch
 }
 
-type castInt2Int4Op struct {
+type castFloatInt4Op struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castInt2Int4Op{}
-var _ colexecop.ClosableOperator = &castInt2Int4Op{}
+var _ colexecop.ResettableOperator = &castFloatInt4Op{}
+var _ colexecop.ClosableOperator = &castFloatInt4Op{}
 
-func (c *castInt2Int4Op) Next() coldata.Batch {
+func (c *castFloatInt4Op) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -2501,7 +3776,7 @@ func (c *castInt2Int4Op) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int16()
+			inputCol := inputVec.Float64()
 			inputNulls := inputVec.Nulls()
 			outputCol := outputVec.Int32()
 			outputNulls := outputVec.Nulls()
@@ -2520,7 +3795,12 @@ func (c *castInt2Int4Op) Next() coldata.Batch {
 							}
 							v := inputCol.Get(tupleIdx)
 							var r int32
+
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt32) || v >= float64(math.MaxInt32) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							}
 							r = int32(v)
+
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -2540,7 +3820,12 @@ func (c *castInt2Int4Op) Next() coldata.Batch {
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
 							var r int32
+
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt32) || v >= float64(math.MaxInt32) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							}
 							r = int32(v)
+
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -2560,7 +3845,12 @@ func (c *castInt2Int4Op) Next() coldata.Batch {
 							}
 							v := inputCol.Get(tupleIdx)
 							var r int32
+
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt32) || v >= float64(math.MaxInt32) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							}
 							r = int32(v)
+
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -2580,7 +3870,12 @@ func (c *castInt2Int4Op) Next() coldata.Batch {
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
 							var r int32
+
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt32) || v >= float64(math.MaxInt32) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							}
 							r = int32(v)
+
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -2592,14 +3887,14 @@ func (c *castInt2Int4Op) Next() coldata.Batch {
 	return batch
 }
 
-type castInt2IntOp struct {
+type castFloatIntOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castInt2IntOp{}
-var _ colexecop.ClosableOperator = &castInt2IntOp{}
+var _ colexecop.ResettableOperator = &castFloatIntOp{}
+var _ colexecop.ClosableOperator = &castFloatIntOp{}
 
-func (c *castInt2IntOp) Next() coldata.Batch {
+func (c *castFloatIntOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -2613,7 +3908,7 @@ func (c *castInt2IntOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int16()
+			inputCol := inputVec.Float64()
 			inputNulls := inputVec.Nulls()
 			outputCol := outputVec.Int64()
 			outputNulls := outputVec.Nulls()
@@ -2632,7 +3927,12 @@ func (c *castInt2IntOp) Next() coldata.Batch {
 							}
 							v := inputCol.Get(tupleIdx)
 							var r int64
+
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt64) || v >= float64(math.MaxInt64) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							}
 							r = int64(v)
+
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -2652,7 +3952,12 @@ func (c *castInt2IntOp) Next() coldata.Batch {
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
 							var r int64
+
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt64) || v >= float64(math.MaxInt64) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							}
 							r = int64(v)
+
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -2672,7 +3977,12 @@ func (c *castInt2IntOp) Next() coldata.Batch {
 							}
 							v := inputCol.Get(tupleIdx)
 							var r int64
+
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt64) || v >= float64(math.MaxInt64) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							}
 							r = int64(v)
+
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -2692,7 +4002,12 @@ func (c *castInt2IntOp) Next() coldata.Batch {
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
 							var r int64
+
+							if math.IsNaN(float64(v)) || v <= float64(math.MinInt64) || v >= float64(math.MaxInt64) {
+								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							}
 							r = int64(v)
+
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -3080,14 +4395,14 @@ func (c *castInt2FloatOp) Next() coldata.Batch {
 	return batch
 }
 
-type castInt4Int2Op struct {
+type castInt2Int4Op struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castInt4Int2Op{}
-var _ colexecop.ClosableOperator = &castInt4Int2Op{}
+var _ colexecop.ResettableOperator = &castInt2Int4Op{}
+var _ colexecop.ClosableOperator = &castInt2Int4Op{}
 
-func (c *castInt4Int2Op) Next() coldata.Batch {
+func (c *castInt2Int4Op) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -3101,9 +4416,9 @@ func (c *castInt4Int2Op) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int32()
+			inputCol := inputVec.Int16()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int16()
+			outputCol := outputVec.Int32()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -3119,14 +4434,8 @@ func (c *castInt4Int2Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int16
-
-							shifted := v >> uint(15)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
-							}
-							r = int16(v)
-
+							var r int32
+							r = int32(v)
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -3145,14 +4454,8 @@ func (c *castInt4Int2Op) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int16
-
-							shifted := v >> uint(15)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
-							}
-							r = int16(v)
-
+							var r int32
+							r = int32(v)
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -3171,14 +4474,8 @@ func (c *castInt4Int2Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int16
-
-							shifted := v >> uint(15)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
-							}
-							r = int16(v)
-
+							var r int32
+							r = int32(v)
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -3197,14 +4494,8 @@ func (c *castInt4Int2Op) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int16
-
-							shifted := v >> uint(15)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
-							}
-							r = int16(v)
-
+							var r int32
+							r = int32(v)
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -3216,14 +4507,14 @@ func (c *castInt4Int2Op) Next() coldata.Batch {
 	return batch
 }
 
-type castInt4IntOp struct {
+type castInt2IntOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castInt4IntOp{}
-var _ colexecop.ClosableOperator = &castInt4IntOp{}
+var _ colexecop.ResettableOperator = &castInt2IntOp{}
+var _ colexecop.ClosableOperator = &castInt2IntOp{}
 
-func (c *castInt4IntOp) Next() coldata.Batch {
+func (c *castInt2IntOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -3237,7 +4528,7 @@ func (c *castInt4IntOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int32()
+			inputCol := inputVec.Int16()
 			inputNulls := inputVec.Nulls()
 			outputCol := outputVec.Int64()
 			outputNulls := outputVec.Nulls()
@@ -3704,14 +4995,14 @@ func (c *castInt4FloatOp) Next() coldata.Batch {
 	return batch
 }
 
-type castIntInt2Op struct {
+type castInt4Int2Op struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castIntInt2Op{}
-var _ colexecop.ClosableOperator = &castIntInt2Op{}
+var _ colexecop.ResettableOperator = &castInt4Int2Op{}
+var _ colexecop.ClosableOperator = &castInt4Int2Op{}
 
-func (c *castIntInt2Op) Next() coldata.Batch {
+func (c *castInt4Int2Op) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -3725,7 +5016,7 @@ func (c *castIntInt2Op) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int64()
+			inputCol := inputVec.Int32()
 			inputNulls := inputVec.Nulls()
 			outputCol := outputVec.Int16()
 			outputNulls := outputVec.Nulls()
@@ -3840,14 +5131,14 @@ func (c *castIntInt2Op) Next() coldata.Batch {
 	return batch
 }
 
-type castIntInt4Op struct {
+type castInt4IntOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castIntInt4Op{}
-var _ colexecop.ClosableOperator = &castIntInt4Op{}
+var _ colexecop.ResettableOperator = &castInt4IntOp{}
+var _ colexecop.ClosableOperator = &castInt4IntOp{}
 
-func (c *castIntInt4Op) Next() coldata.Batch {
+func (c *castInt4IntOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -3861,9 +5152,9 @@ func (c *castIntInt4Op) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int64()
+			inputCol := inputVec.Int32()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int32()
+			outputCol := outputVec.Int64()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -3879,14 +5170,8 @@ func (c *castIntInt4Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int32
-
-							shifted := v >> uint(31)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
-							}
-							r = int32(v)
-
+							var r int64
+							r = int64(v)
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -3905,14 +5190,8 @@ func (c *castIntInt4Op) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int32
-
-							shifted := v >> uint(31)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
-							}
-							r = int32(v)
-
+							var r int64
+							r = int64(v)
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -3931,14 +5210,8 @@ func (c *castIntInt4Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int32
-
-							shifted := v >> uint(31)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
-							}
-							r = int32(v)
-
+							var r int64
+							r = int64(v)
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -3957,14 +5230,8 @@ func (c *castIntInt4Op) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int32
-
-							shifted := v >> uint(31)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
-							}
-							r = int32(v)
-
+							var r int64
+							r = int64(v)
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4352,14 +5619,14 @@ func (c *castIntFloatOp) Next() coldata.Batch {
 	return batch
 }
 
-type castFloatBoolOp struct {
+type castIntInt2Op struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castFloatBoolOp{}
-var _ colexecop.ClosableOperator = &castFloatBoolOp{}
+var _ colexecop.ResettableOperator = &castIntInt2Op{}
+var _ colexecop.ClosableOperator = &castIntInt2Op{}
 
-func (c *castFloatBoolOp) Next() coldata.Batch {
+func (c *castIntInt2Op) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -4373,9 +5640,9 @@ func (c *castFloatBoolOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Float64()
+			inputCol := inputVec.Int64()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Bool()
+			outputCol := outputVec.Int16()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -4391,9 +5658,13 @@ func (c *castFloatBoolOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r bool
+							var r int16
 
-							r = v != 0
+							shifted := v >> uint(15)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							}
+							r = int16(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4413,9 +5684,13 @@ func (c *castFloatBoolOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r bool
+							var r int16
 
-							r = v != 0
+							shifted := v >> uint(15)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							}
+							r = int16(v)
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -4435,9 +5710,13 @@ func (c *castFloatBoolOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r bool
+							var r int16
 
-							r = v != 0
+							shifted := v >> uint(15)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							}
+							r = int16(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4457,9 +5736,13 @@ func (c *castFloatBoolOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r bool
+							var r int16
 
-							r = v != 0
+							shifted := v >> uint(15)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							}
+							r = int16(v)
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -4472,14 +5755,14 @@ func (c *castFloatBoolOp) Next() coldata.Batch {
 	return batch
 }
 
-type castFloatDecimalOp struct {
+type castIntInt4Op struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castFloatDecimalOp{}
-var _ colexecop.ClosableOperator = &castFloatDecimalOp{}
+var _ colexecop.ResettableOperator = &castIntInt4Op{}
+var _ colexecop.ClosableOperator = &castIntInt4Op{}
 
-func (c *castFloatDecimalOp) Next() coldata.Batch {
+func (c *castIntInt4Op) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -4493,9 +5776,9 @@ func (c *castFloatDecimalOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Float64()
+			inputCol := inputVec.Int64()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Decimal()
+			outputCol := outputVec.Int32()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -4511,15 +5794,13 @@ func (c *castFloatDecimalOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
-
-							if _, err := r.SetFloat64(float64(v)); err != nil {
-								colexecerror.ExpectedError(err)
-							}
+							var r int32
 
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
+							shifted := v >> uint(31)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 							}
+							r = int32(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4539,15 +5820,13 @@ func (c *castFloatDecimalOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
-
-							if _, err := r.SetFloat64(float64(v)); err != nil {
-								colexecerror.ExpectedError(err)
-							}
+							var r int32
 
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
+							shifted := v >> uint(31)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 							}
+							r = int32(v)
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -4567,15 +5846,13 @@ func (c *castFloatDecimalOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
-
-							if _, err := r.SetFloat64(float64(v)); err != nil {
-								colexecerror.ExpectedError(err)
-							}
+							var r int32
 
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
+							shifted := v >> uint(31)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 							}
+							r = int32(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4595,15 +5872,13 @@ func (c *castFloatDecimalOp) Next() coldata.Batch {
 							}
 							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
-
-							if _, err := r.SetFloat64(float64(v)); err != nil {
-								colexecerror.ExpectedError(err)
-							}
+							var r int32
 
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
+							shifted := v >> uint(31)
+							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
+								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
 							}
+							r = int32(v)
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -4616,14 +5891,14 @@ func (c *castFloatDecimalOp) Next() coldata.Batch {
 	return batch
 }
 
-type castFloatInt2Op struct {
+type castJsonbStringOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castFloatInt2Op{}
-var _ colexecop.ClosableOperator = &castFloatInt2Op{}
+var _ colexecop.ResettableOperator = &castJsonbStringOp{}
+var _ colexecop.ClosableOperator = &castJsonbStringOp{}
 
-func (c *castFloatInt2Op) Next() coldata.Batch {
+func (c *castJsonbStringOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -4637,9 +5912,9 @@ func (c *castFloatInt2Op) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Float64()
+			inputCol := inputVec.JSON()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int16()
+			outputCol := outputVec.Bytes()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -4655,12 +5930,18 @@ func (c *castFloatInt2Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r []byte
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt16) || v >= float64(math.MaxInt16) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							_string := v.String()
+							switch toType.Oid() {
+							case oid.T_char:
+								// "char" is supposed to truncate long values.
+								_string = util.TruncateString(_string, 1)
+							case oid.T_bpchar:
+								// bpchar types truncate trailing whitespace.
+								_string = strings.TrimRight(_string, " ")
 							}
-							r = int16(v)
+							r = []byte(_string)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4670,24 +5951,26 @@ func (c *castFloatInt2Op) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
-						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
 							tupleIdx = i
 							if true && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r []byte
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt16) || v >= float64(math.MaxInt16) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							_string := v.String()
+							switch toType.Oid() {
+							case oid.T_char:
+								// "char" is supposed to truncate long values.
+								_string = util.TruncateString(_string, 1)
+							case oid.T_bpchar:
+								// bpchar types truncate trailing whitespace.
+								_string = strings.TrimRight(_string, " ")
 							}
-							r = int16(v)
+							r = []byte(_string)
 
-							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -4705,12 +5988,18 @@ func (c *castFloatInt2Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r []byte
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt16) || v >= float64(math.MaxInt16) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							_string := v.String()
+							switch toType.Oid() {
+							case oid.T_char:
+								// "char" is supposed to truncate long values.
+								_string = util.TruncateString(_string, 1)
+							case oid.T_bpchar:
+								// bpchar types truncate trailing whitespace.
+								_string = strings.TrimRight(_string, " ")
 							}
-							r = int16(v)
+							r = []byte(_string)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4720,24 +6009,26 @@ func (c *castFloatInt2Op) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
-						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
 							tupleIdx = i
 							if false && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r []byte
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt16) || v >= float64(math.MaxInt16) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							_string := v.String()
+							switch toType.Oid() {
+							case oid.T_char:
+								// "char" is supposed to truncate long values.
+								_string = util.TruncateString(_string, 1)
+							case oid.T_bpchar:
+								// bpchar types truncate trailing whitespace.
+								_string = strings.TrimRight(_string, " ")
 							}
-							r = int16(v)
+							r = []byte(_string)
 
-							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -4748,14 +6039,14 @@ func (c *castFloatInt2Op) Next() coldata.Batch {
 	return batch
 }
 
-type castFloatInt4Op struct {
+type castStringBoolOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castFloatInt4Op{}
-var _ colexecop.ClosableOperator = &castFloatInt4Op{}
+var _ colexecop.ResettableOperator = &castStringBoolOp{}
+var _ colexecop.ClosableOperator = &castStringBoolOp{}
 
-func (c *castFloatInt4Op) Next() coldata.Batch {
+func (c *castStringBoolOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -4769,9 +6060,9 @@ func (c *castFloatInt4Op) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Float64()
+			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int32()
+			outputCol := outputVec.Bool()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -4787,12 +6078,13 @@ func (c *castFloatInt4Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r bool
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt32) || v >= float64(math.MaxInt32) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							var err error
+							r, err = tree.ParseBool(string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int32(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4802,7 +6094,6 @@ func (c *castFloatInt4Op) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -4810,14 +6101,14 @@ func (c *castFloatInt4Op) Next() coldata.Batch {
 							if true && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r bool
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt32) || v >= float64(math.MaxInt32) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							var err error
+							r, err = tree.ParseBool(string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int32(v)
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -4837,12 +6128,13 @@ func (c *castFloatInt4Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r bool
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt32) || v >= float64(math.MaxInt32) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							var err error
+							r, err = tree.ParseBool(string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int32(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4852,7 +6144,6 @@ func (c *castFloatInt4Op) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -4860,14 +6151,14 @@ func (c *castFloatInt4Op) Next() coldata.Batch {
 							if false && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r bool
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt32) || v >= float64(math.MaxInt32) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							var err error
+							r, err = tree.ParseBool(string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int32(v)
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -4880,14 +6171,14 @@ func (c *castFloatInt4Op) Next() coldata.Batch {
 	return batch
 }
 
-type castFloatIntOp struct {
+type castStringBytesOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castFloatIntOp{}
-var _ colexecop.ClosableOperator = &castFloatIntOp{}
+var _ colexecop.ResettableOperator = &castStringBytesOp{}
+var _ colexecop.ClosableOperator = &castStringBytesOp{}
 
-func (c *castFloatIntOp) Next() coldata.Batch {
+func (c *castStringBytesOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -4901,9 +6192,9 @@ func (c *castFloatIntOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Float64()
+			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int64()
+			outputCol := outputVec.Bytes()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -4919,12 +6210,13 @@ func (c *castFloatIntOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r []byte
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt64) || v >= float64(math.MaxInt64) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							var err error
+							r, err = lex.DecodeRawBytesToByteArrayAuto(v)
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int64(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4934,24 +6226,21 @@ func (c *castFloatIntOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
-						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
 							tupleIdx = i
 							if true && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r []byte
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt64) || v >= float64(math.MaxInt64) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							var err error
+							r, err = lex.DecodeRawBytesToByteArrayAuto(v)
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int64(v)
 
-							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -4969,12 +6258,13 @@ func (c *castFloatIntOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r []byte
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt64) || v >= float64(math.MaxInt64) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							var err error
+							r, err = lex.DecodeRawBytesToByteArrayAuto(v)
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int64(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -4984,24 +6274,21 @@ func (c *castFloatIntOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
-						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
 							tupleIdx = i
 							if false && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int64
+							var r []byte
 
-							if math.IsNaN(float64(v)) || v <= float64(math.MinInt64) || v >= float64(math.MaxInt64) {
-								colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+							var err error
+							r, err = lex.DecodeRawBytesToByteArrayAuto(v)
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int64(v)
 
-							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -5012,14 +6299,14 @@ func (c *castFloatIntOp) Next() coldata.Batch {
 	return batch
 }
 
-type castDateInt2Op struct {
+type castStringDateOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDateInt2Op{}
-var _ colexecop.ClosableOperator = &castDateInt2Op{}
+var _ colexecop.ResettableOperator = &castStringDateOp{}
+var _ colexecop.ClosableOperator = &castStringDateOp{}
 
-func (c *castDateInt2Op) Next() coldata.Batch {
+func (c *castStringDateOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -5033,9 +6320,9 @@ func (c *castDateInt2Op) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int64()
+			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int16()
+			outputCol := outputVec.Int64()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -5051,13 +6338,15 @@ func (c *castDateInt2Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r int64
 
-							shifted := v >> uint(15)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_d, _, err := pgdate.ParseDate(_now, _dateStyle, string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int16(v)
+							r = _d.UnixEpochDays()
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -5067,7 +6356,6 @@ func (c *castDateInt2Op) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -5075,15 +6363,16 @@ func (c *castDateInt2Op) Next() coldata.Batch {
 							if true && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r int64
 
-							shifted := v >> uint(15)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_d, _, err := pgdate.ParseDate(_now, _dateStyle, string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int16(v)
+							r = _d.UnixEpochDays()
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -5103,13 +6392,15 @@ func (c *castDateInt2Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r int64
 
-							shifted := v >> uint(15)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_d, _, err := pgdate.ParseDate(_now, _dateStyle, string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int16(v)
+							r = _d.UnixEpochDays()
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -5119,7 +6410,6 @@ func (c *castDateInt2Op) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -5127,15 +6417,16 @@ func (c *castDateInt2Op) Next() coldata.Batch {
 							if false && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int16
+							var r int64
 
-							shifted := v >> uint(15)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_d, _, err := pgdate.ParseDate(_now, _dateStyle, string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int16(v)
+							r = _d.UnixEpochDays()
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -5148,14 +6439,14 @@ func (c *castDateInt2Op) Next() coldata.Batch {
 	return batch
 }
 
-type castDateInt4Op struct {
+type castStringDecimalOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDateInt4Op{}
-var _ colexecop.ClosableOperator = &castDateInt4Op{}
+var _ colexecop.ResettableOperator = &castStringDecimalOp{}
+var _ colexecop.ClosableOperator = &castStringDecimalOp{}
 
-func (c *castDateInt4Op) Next() coldata.Batch {
+func (c *castStringDecimalOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -5169,9 +6460,9 @@ func (c *castDateInt4Op) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int64()
+			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int32()
+			outputCol := outputVec.Decimal()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -5187,13 +6478,28 @@ func (c *castDateInt4Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r apd.Decimal
 
-							shifted := v >> uint(31)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+							_s := strings.TrimSpace(string(v))
+							_, res, err := tree.ExactCtx.SetString(&r, _s)
+							if res != 0 || err != nil {
+								colexecerror.ExpectedError(tree.MakeParseError(_s, types.Decimal, err))
+							}
+							switch r.Form {
+							case apd.NaNSignaling:
+								r.Form = apd.NaN
+								r.Negative = false
+							case apd.NaN:
+								r.Negative = false
+							case apd.Finite:
+								if r.IsZero() && r.Negative {
+									r.Negative = false
+								}
+							}
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int32(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -5203,7 +6509,6 @@ func (c *castDateInt4Op) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -5211,15 +6516,29 @@ func (c *castDateInt4Op) Next() coldata.Batch {
 							if true && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r apd.Decimal
 
-							shifted := v >> uint(31)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+							_s := strings.TrimSpace(string(v))
+							_, res, err := tree.ExactCtx.SetString(&r, _s)
+							if res != 0 || err != nil {
+								colexecerror.ExpectedError(tree.MakeParseError(_s, types.Decimal, err))
+							}
+							switch r.Form {
+							case apd.NaNSignaling:
+								r.Form = apd.NaN
+								r.Negative = false
+							case apd.NaN:
+								r.Negative = false
+							case apd.Finite:
+								if r.IsZero() && r.Negative {
+									r.Negative = false
+								}
+							}
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int32(v)
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -5239,13 +6558,28 @@ func (c *castDateInt4Op) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r apd.Decimal
 
-							shifted := v >> uint(31)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+							_s := strings.TrimSpace(string(v))
+							_, res, err := tree.ExactCtx.SetString(&r, _s)
+							if res != 0 || err != nil {
+								colexecerror.ExpectedError(tree.MakeParseError(_s, types.Decimal, err))
+							}
+							switch r.Form {
+							case apd.NaNSignaling:
+								r.Form = apd.NaN
+								r.Negative = false
+							case apd.NaN:
+								r.Negative = false
+							case apd.Finite:
+								if r.IsZero() && r.Negative {
+									r.Negative = false
+								}
+							}
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int32(v)
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -5255,7 +6589,6 @@ func (c *castDateInt4Op) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -5263,15 +6596,29 @@ func (c *castDateInt4Op) Next() coldata.Batch {
 							if false && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int32
+							var r apd.Decimal
 
-							shifted := v >> uint(31)
-							if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) {
-								colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+							_s := strings.TrimSpace(string(v))
+							_, res, err := tree.ExactCtx.SetString(&r, _s)
+							if res != 0 || err != nil {
+								colexecerror.ExpectedError(tree.MakeParseError(_s, types.Decimal, err))
+							}
+							switch r.Form {
+							case apd.NaNSignaling:
+								r.Form = apd.NaN
+								r.Negative = false
+							case apd.NaN:
+								r.Negative = false
+							case apd.Finite:
+								if r.IsZero() && r.Negative {
+									r.Negative = false
+								}
+							}
+
+							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = int32(v)
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -5284,14 +6631,14 @@ func (c *castDateInt4Op) Next() coldata.Batch {
 	return batch
 }
 
-type castDateIntOp struct {
+type castStringFloatOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDateIntOp{}
-var _ colexecop.ClosableOperator = &castDateIntOp{}
+var _ colexecop.ResettableOperator = &castStringFloatOp{}
+var _ colexecop.ClosableOperator = &castStringFloatOp{}
 
-func (c *castDateIntOp) Next() coldata.Batch {
+func (c *castStringFloatOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -5305,9 +6652,9 @@ func (c *castDateIntOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int64()
+			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Int64()
+			outputCol := outputVec.Float64()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -5323,8 +6670,15 @@ func (c *castDateIntOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int64
-							r = int64(v)
+							var r float64
+
+							_s := string(v)
+							var _err error
+							r, _err = strconv.ParseFloat(strings.TrimSpace(_s), 64)
+							if _err != nil {
+								colexecerror.ExpectedError(tree.MakeParseError(_s, toType, _err))
+							}
+
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -5333,7 +6687,6 @@ func (c *castDateIntOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -5341,10 +6694,16 @@ func (c *castDateIntOp) Next() coldata.Batch {
 							if true && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int64
-							r = int64(v)
+							var r float64
+
+							_s := string(v)
+							var _err error
+							r, _err = strconv.ParseFloat(strings.TrimSpace(_s), 64)
+							if _err != nil {
+								colexecerror.ExpectedError(tree.MakeParseError(_s, toType, _err))
+							}
+
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -5363,8 +6722,15 @@ func (c *castDateIntOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r int64
-							r = int64(v)
+							var r float64
+
+							_s := string(v)
+							var _err error
+							r, _err = strconv.ParseFloat(strings.TrimSpace(_s), 64)
+							if _err != nil {
+								colexecerror.ExpectedError(tree.MakeParseError(_s, toType, _err))
+							}
+
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -5373,7 +6739,6 @@ func (c *castDateIntOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -5381,10 +6746,16 @@ func (c *castDateIntOp) Next() coldata.Batch {
 							if false && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r int64
-							r = int64(v)
+							var r float64
+
+							_s := string(v)
+							var _err error
+							r, _err = strconv.ParseFloat(strings.TrimSpace(_s), 64)
+							if _err != nil {
+								colexecerror.ExpectedError(tree.MakeParseError(_s, toType, _err))
+							}
+
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
@@ -5396,14 +6767,14 @@ func (c *castDateIntOp) Next() coldata.Batch {
 	return batch
 }
 
-type castDateFloatOp struct {
+type castStringInt2Op struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDateFloatOp{}
-var _ colexecop.ClosableOperator = &castDateFloatOp{}
+var _ colexecop.ResettableOperator = &castStringInt2Op{}
+var _ colexecop.ClosableOperator = &castStringInt2Op{}
 
-func (c *castDateFloatOp) Next() coldata.Batch {
+func (c *castStringInt2Op) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -5417,9 +6788,9 @@ func (c *castDateFloatOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int64()
+			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Float64()
+			outputCol := outputVec.Int16()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -5435,9 +6806,22 @@ func (c *castDateFloatOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r float64
+							var r int16
 
-							r = float64(v)
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+
+								shifted := _i >> uint(15)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+								r = int16(_i)
+
+							}
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -5447,7 +6831,6 @@ func (c *castDateFloatOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -5455,11 +6838,23 @@ func (c *castDateFloatOp) Next() coldata.Batch {
 							if true && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r float64
+							var r int16
 
-							r = float64(v)
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+
+								shifted := _i >> uint(15)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+								r = int16(_i)
+
+							}
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -5479,9 +6874,22 @@ func (c *castDateFloatOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r float64
+							var r int16
 
-							r = float64(v)
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+
+								shifted := _i >> uint(15)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+								r = int16(_i)
+
+							}
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -5491,7 +6899,6 @@ func (c *castDateFloatOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -5499,11 +6906,23 @@ func (c *castDateFloatOp) Next() coldata.Batch {
 							if false && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r float64
+							var r int16
 
-							r = float64(v)
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+
+								shifted := _i >> uint(15)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt2OutOfRange)
+								}
+								r = int16(_i)
+
+							}
 
 							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
@@ -5516,14 +6935,14 @@ func (c *castDateFloatOp) Next() coldata.Batch {
 	return batch
 }
 
-type castDateDecimalOp struct {
+type castStringInt4Op struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castDateDecimalOp{}
-var _ colexecop.ClosableOperator = &castDateDecimalOp{}
+var _ colexecop.ResettableOperator = &castStringInt4Op{}
+var _ colexecop.ClosableOperator = &castStringInt4Op{}
 
-func (c *castDateDecimalOp) Next() coldata.Batch {
+func (c *castStringInt4Op) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -5537,9 +6956,9 @@ func (c *castDateDecimalOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.Int64()
+			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Decimal()
+			outputCol := outputVec.Int32()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -5555,12 +6974,21 @@ func (c *castDateDecimalOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
+							var r int32
 
-							r.SetInt64(int64(v))
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+
+								shifted := _i >> uint(31)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+								}
+								r = int32(_i)
 
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -5571,7 +6999,6 @@ func (c *castDateDecimalOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -5579,14 +7006,22 @@ func (c *castDateDecimalOp) Next() coldata.Batch {
 							if true && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
+							var r int32
 
-							r.SetInt64(int64(v))
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+
+								shifted := _i >> uint(31)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+								}
+								r = int32(_i)
 
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
 							}
 
 							//gcassert:bce
@@ -5607,12 +7042,21 @@ func (c *castDateDecimalOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
+							var r int32
 
-							r.SetInt64(int64(v))
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+
+								shifted := _i >> uint(31)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+								}
+								r = int32(_i)
 
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -5623,7 +7067,6 @@ func (c *castDateDecimalOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
-						_ = inputCol.Get(n - 1)
 						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
@@ -5631,14 +7074,22 @@ func (c *castDateDecimalOp) Next() coldata.Batch {
 							if false && inputNulls.NullAt(tupleIdx) {
 								continue
 							}
-							//gcassert:bce
 							v := inputCol.Get(tupleIdx)
-							var r apd.Decimal
+							var r int32
 
-							r.SetInt64(int64(v))
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+
+								shifted := _i >> uint(31)
+								if (_i >= 0 && shifted > 0) || (_i < 0 && shifted < -1) {
+									colexecerror.ExpectedError(tree.ErrInt4OutOfRange)
+								}
+								r = int32(_i)
 
-							if err := tree.LimitDecimalWidth(&r, int(toType.Precision()), int(toType.Scale())); err != nil {
-								colexecerror.ExpectedError(err)
 							}
 
 							//gcassert:bce
@@ -5652,14 +7103,14 @@ func (c *castDateDecimalOp) Next() coldata.Batch {
 	return batch
 }
 
-type castBytesUuidOp struct {
+type castStringIntOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castBytesUuidOp{}
-var _ colexecop.ClosableOperator = &castBytesUuidOp{}
+var _ colexecop.ResettableOperator = &castStringIntOp{}
+var _ colexecop.ClosableOperator = &castStringIntOp{}
 
-func (c *castBytesUuidOp) Next() coldata.Batch {
+func (c *castStringIntOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -5675,7 +7126,7 @@ func (c *castBytesUuidOp) Next() coldata.Batch {
 		[]coldata.Vec{outputVec}, func() {
 			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Bytes()
+			outputCol := outputVec.Int64()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -5691,13 +7142,16 @@ func (c *castBytesUuidOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r int64
 
-							_uuid, err := uuid.FromBytes(v)
-							if err != nil {
-								colexecerror.ExpectedError(err)
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+								r = int64(_i)
 							}
-							r = _uuid.GetBytes()
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -5707,6 +7161,7 @@ func (c *castBytesUuidOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
+						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
 							tupleIdx = i
@@ -5714,14 +7169,18 @@ func (c *castBytesUuidOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r int64
 
-							_uuid, err := uuid.FromBytes(v)
-							if err != nil {
-								colexecerror.ExpectedError(err)
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+								r = int64(_i)
 							}
-							r = _uuid.GetBytes()
 
+							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -5739,13 +7198,16 @@ func (c *castBytesUuidOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r int64
 
-							_uuid, err := uuid.FromBytes(v)
-							if err != nil {
-								colexecerror.ExpectedError(err)
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+								r = int64(_i)
 							}
-							r = _uuid.GetBytes()
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -5755,6 +7217,7 @@ func (c *castBytesUuidOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
+						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
 							tupleIdx = i
@@ -5762,14 +7225,18 @@ func (c *castBytesUuidOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r int64
 
-							_uuid, err := uuid.FromBytes(v)
-							if err != nil {
-								colexecerror.ExpectedError(err)
+							{
+								_s := string(v)
+								_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+								if err != nil {
+									colexecerror.ExpectedError(tree.MakeParseError(_s, toType, err))
+								}
+								r = int64(_i)
 							}
-							r = _uuid.GetBytes()
 
+							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -5780,14 +7247,14 @@ func (c *castBytesUuidOp) Next() coldata.Batch {
 	return batch
 }
 
-type castStringBoolOp struct {
+type castStringIntervalOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castStringBoolOp{}
-var _ colexecop.ClosableOperator = &castStringBoolOp{}
+var _ colexecop.ResettableOperator = &castStringIntervalOp{}
+var _ colexecop.ClosableOperator = &castStringIntervalOp{}
 
-func (c *castStringBoolOp) Next() coldata.Batch {
+func (c *castStringIntervalOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -5803,7 +7270,7 @@ func (c *castStringBoolOp) Next() coldata.Batch {
 		[]coldata.Vec{outputVec}, func() {
 			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Bool()
+			outputCol := outputVec.Interval()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -5819,10 +7286,14 @@ func (c *castStringBoolOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r bool
+							var r duration.Duration
 
-							var err error
-							r, err = tree.ParseBool(string(v))
+							_itm, err := toType.IntervalTypeMetadata()
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							_intervalStyle := evalCtx.GetIntervalStyle()
+							r, err = tree.ParseIntervalWithTypeMetadata(_intervalStyle, string(v), _itm)
 							if err != nil {
 								colexecerror.ExpectedError(err)
 							}
@@ -5843,10 +7314,14 @@ func (c *castStringBoolOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r bool
-
-							var err error
-							r, err = tree.ParseBool(string(v))
+							var r duration.Duration
+
+							_itm, err := toType.IntervalTypeMetadata()
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							_intervalStyle := evalCtx.GetIntervalStyle()
+							r, err = tree.ParseIntervalWithTypeMetadata(_intervalStyle, string(v), _itm)
 							if err != nil {
 								colexecerror.ExpectedError(err)
 							}
@@ -5869,10 +7344,14 @@ func (c *castStringBoolOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r bool
+							var r duration.Duration
 
-							var err error
-							r, err = tree.ParseBool(string(v))
+							_itm, err := toType.IntervalTypeMetadata()
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							_intervalStyle := evalCtx.GetIntervalStyle()
+							r, err = tree.ParseIntervalWithTypeMetadata(_intervalStyle, string(v), _itm)
 							if err != nil {
 								colexecerror.ExpectedError(err)
 							}
@@ -5893,10 +7372,14 @@ func (c *castStringBoolOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r bool
+							var r duration.Duration
 
-							var err error
-							r, err = tree.ParseBool(string(v))
+							_itm, err := toType.IntervalTypeMetadata()
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							_intervalStyle := evalCtx.GetIntervalStyle()
+							r, err = tree.ParseIntervalWithTypeMetadata(_intervalStyle, string(v), _itm)
 							if err != nil {
 								colexecerror.ExpectedError(err)
 							}
@@ -5912,14 +7395,14 @@ func (c *castStringBoolOp) Next() coldata.Batch {
 	return batch
 }
 
-type castStringBytesOp struct {
+type castStringJsonbOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castStringBytesOp{}
-var _ colexecop.ClosableOperator = &castStringBytesOp{}
+var _ colexecop.ResettableOperator = &castStringJsonbOp{}
+var _ colexecop.ClosableOperator = &castStringJsonbOp{}
 
-func (c *castStringBytesOp) Next() coldata.Batch {
+func (c *castStringJsonbOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -5935,7 +7418,7 @@ func (c *castStringBytesOp) Next() coldata.Batch {
 		[]coldata.Vec{outputVec}, func() {
 			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Bytes()
+			outputCol := outputVec.JSON()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -5951,12 +7434,12 @@ func (c *castStringBytesOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r json.JSON
 
 							var err error
-							r, err = lex.DecodeRawBytesToByteArrayAuto(v)
+							r, err = json.ParseJSON(string(v))
 							if err != nil {
-								colexecerror.ExpectedError(err)
+								colexecerror.ExpectedError(pgerror.Wrapf(err, pgcode.Syntax, "could not parse JSON"))
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -5974,12 +7457,12 @@ func (c *castStringBytesOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r json.JSON
 
 							var err error
-							r, err = lex.DecodeRawBytesToByteArrayAuto(v)
+							r, err = json.ParseJSON(string(v))
 							if err != nil {
-								colexecerror.ExpectedError(err)
+								colexecerror.ExpectedError(pgerror.Wrapf(err, pgcode.Syntax, "could not parse JSON"))
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -5999,12 +7482,12 @@ func (c *castStringBytesOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r json.JSON
 
 							var err error
-							r, err = lex.DecodeRawBytesToByteArrayAuto(v)
+							r, err = json.ParseJSON(string(v))
 							if err != nil {
-								colexecerror.ExpectedError(err)
+								colexecerror.ExpectedError(pgerror.Wrapf(err, pgcode.Syntax, "could not parse JSON"))
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -6022,12 +7505,12 @@ func (c *castStringBytesOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r json.JSON
 
 							var err error
-							r, err = lex.DecodeRawBytesToByteArrayAuto(v)
+							r, err = json.ParseJSON(string(v))
 							if err != nil {
-								colexecerror.ExpectedError(err)
+								colexecerror.ExpectedError(pgerror.Wrapf(err, pgcode.Syntax, "could not parse JSON"))
 							}
 
 							outputCol.Set(tupleIdx, r)
@@ -6248,14 +7731,14 @@ func (c *castStringStringOp) Next() coldata.Batch {
 	return batch
 }
 
-type castStringUuidOp struct {
+type castStringTimestampOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castStringUuidOp{}
-var _ colexecop.ClosableOperator = &castStringUuidOp{}
+var _ colexecop.ResettableOperator = &castStringTimestampOp{}
+var _ colexecop.ClosableOperator = &castStringTimestampOp{}
 
-func (c *castStringUuidOp) Next() coldata.Batch {
+func (c *castStringTimestampOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -6271,7 +7754,7 @@ func (c *castStringUuidOp) Next() coldata.Batch {
 		[]coldata.Vec{outputVec}, func() {
 			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
-			outputCol := outputVec.Bytes()
+			outputCol := outputVec.Timestamp()
 			outputNulls := outputVec.Nulls()
 			if inputVec.MaybeHasNulls() {
 				outputNulls.Copy(inputNulls)
@@ -6287,13 +7770,19 @@ func (c *castStringUuidOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r time.Time
 
-							_uuid, err := uuid.FromString(string(v))
+							_roundTo := tree.TimeFamilyPrecisionToRoundDuration(toType.Precision())
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_t, _, err := pgdate.ParseTimestampWithoutTimezone(_now, _dateStyle, string(v))
 							if err != nil {
 								colexecerror.ExpectedError(err)
 							}
-							r = _uuid.GetBytes()
+							r = _t.Round(_roundTo)
+							if r.After(tree.MaxSupportedTime) || r.Before(tree.MinSupportedTime) {
+								colexecerror.ExpectedError(tree.NewTimestampExceedsBoundsError(r))
+							}
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -6303,6 +7792,7 @@ func (c *castStringUuidOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
+						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
 							tupleIdx = i
@@ -6310,14 +7800,21 @@ func (c *castStringUuidOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r time.Time
 
-							_uuid, err := uuid.FromString(string(v))
+							_roundTo := tree.TimeFamilyPrecisionToRoundDuration(toType.Precision())
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_t, _, err := pgdate.ParseTimestampWithoutTimezone(_now, _dateStyle, string(v))
 							if err != nil {
 								colexecerror.ExpectedError(err)
 							}
-							r = _uuid.GetBytes()
+							r = _t.Round(_roundTo)
+							if r.After(tree.MaxSupportedTime) || r.Before(tree.MinSupportedTime) {
+								colexecerror.ExpectedError(tree.NewTimestampExceedsBoundsError(r))
+							}
 
+							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -6335,13 +7832,19 @@ func (c *castStringUuidOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r time.Time
 
-							_uuid, err := uuid.FromString(string(v))
+							_roundTo := tree.TimeFamilyPrecisionToRoundDuration(toType.Precision())
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_t, _, err := pgdate.ParseTimestampWithoutTimezone(_now, _dateStyle, string(v))
 							if err != nil {
 								colexecerror.ExpectedError(err)
 							}
-							r = _uuid.GetBytes()
+							r = _t.Round(_roundTo)
+							if r.After(tree.MaxSupportedTime) || r.Before(tree.MinSupportedTime) {
+								colexecerror.ExpectedError(tree.NewTimestampExceedsBoundsError(r))
+							}
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -6351,6 +7854,7 @@ func (c *castStringUuidOp) Next() coldata.Batch {
 						var evalCtx *eval.Context = c.evalCtx
 						// Silence unused warning.
 						_ = evalCtx
+						_ = outputCol.Get(n - 1)
 						var tupleIdx int
 						for i := 0; i < n; i++ {
 							tupleIdx = i
@@ -6358,14 +7862,21 @@ func (c *castStringUuidOp) Next() coldata.Batch {
 								continue
 							}
 							v := inputCol.Get(tupleIdx)
-							var r []byte
+							var r time.Time
 
-							_uuid, err := uuid.FromString(string(v))
+							_roundTo := tree.TimeFamilyPrecisionToRoundDuration(toType.Precision())
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_t, _, err := pgdate.ParseTimestampWithoutTimezone(_now, _dateStyle, string(v))
 							if err != nil {
 								colexecerror.ExpectedError(err)
 							}
-							r = _uuid.GetBytes()
+							r = _t.Round(_roundTo)
+							if r.After(tree.MaxSupportedTime) || r.Before(tree.MinSupportedTime) {
+								colexecerror.ExpectedError(tree.NewTimestampExceedsBoundsError(r))
+							}
 
+							//gcassert:bce
 							outputCol.Set(tupleIdx, r)
 						}
 					}
@@ -6376,14 +7887,14 @@ func (c *castStringUuidOp) Next() coldata.Batch {
 	return batch
 }
 
-type castJsonbStringOp struct {
+type castStringTimestamptzOp struct {
 	castOpBase
 }
 
-var _ colexecop.ResettableOperator = &castJsonbStringOp{}
-var _ colexecop.ClosableOperator = &castJsonbStringOp{}
+var _ colexecop.ResettableOperator = &castStringTimestamptzOp{}
+var _ colexecop.ClosableOperator = &castStringTimestamptzOp{}
 
-func (c *castJsonbStringOp) Next() coldata.Batch {
+func (c *castStringTimestamptzOp) Next() coldata.Batch {
 	batch := c.Input.Next()
 	n := batch.Length()
 	if n == 0 {
@@ -6397,7 +7908,163 @@ func (c *castJsonbStringOp) Next() coldata.Batch {
 	_ = toType
 	c.allocator.PerformOperation(
 		[]coldata.Vec{outputVec}, func() {
-			inputCol := inputVec.JSON()
+			inputCol := inputVec.Bytes()
+			inputNulls := inputVec.Nulls()
+			outputCol := outputVec.Timestamp()
+			outputNulls := outputVec.Nulls()
+			if inputVec.MaybeHasNulls() {
+				outputNulls.Copy(inputNulls)
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r time.Time
+
+							_roundTo := tree.TimeFamilyPrecisionToRoundDuration(toType.Precision())
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_t, _, err := pgdate.ParseTimestamp(_now, _dateStyle, string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							r = _t.Round(_roundTo)
+							if r.After(tree.MaxSupportedTime) || r.Before(tree.MinSupportedTime) {
+								colexecerror.ExpectedError(tree.NewTimestampExceedsBoundsError(r))
+							}
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if true && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r time.Time
+
+							_roundTo := tree.TimeFamilyPrecisionToRoundDuration(toType.Precision())
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_t, _, err := pgdate.ParseTimestamp(_now, _dateStyle, string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							r = _t.Round(_roundTo)
+							if r.After(tree.MaxSupportedTime) || r.Before(tree.MinSupportedTime) {
+								colexecerror.ExpectedError(tree.NewTimestampExceedsBoundsError(r))
+							}
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			} else {
+				if sel != nil {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = sel[i]
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r time.Time
+
+							_roundTo := tree.TimeFamilyPrecisionToRoundDuration(toType.Precision())
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_t, _, err := pgdate.ParseTimestamp(_now, _dateStyle, string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							r = _t.Round(_roundTo)
+							if r.After(tree.MaxSupportedTime) || r.Before(tree.MinSupportedTime) {
+								colexecerror.ExpectedError(tree.NewTimestampExceedsBoundsError(r))
+							}
+
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				} else {
+					{
+						var evalCtx *eval.Context = c.evalCtx
+						// Silence unused warning.
+						_ = evalCtx
+						_ = outputCol.Get(n - 1)
+						var tupleIdx int
+						for i := 0; i < n; i++ {
+							tupleIdx = i
+							if false && inputNulls.NullAt(tupleIdx) {
+								continue
+							}
+							v := inputCol.Get(tupleIdx)
+							var r time.Time
+
+							_roundTo := tree.TimeFamilyPrecisionToRoundDuration(toType.Precision())
+							_now := evalCtx.GetRelativeParseTime()
+							_dateStyle := evalCtx.GetDateStyle()
+							_t, _, err := pgdate.ParseTimestamp(_now, _dateStyle, string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
+							}
+							r = _t.Round(_roundTo)
+							if r.After(tree.MaxSupportedTime) || r.Before(tree.MinSupportedTime) {
+								colexecerror.ExpectedError(tree.NewTimestampExceedsBoundsError(r))
+							}
+
+							//gcassert:bce
+							outputCol.Set(tupleIdx, r)
+						}
+					}
+				}
+			}
+		},
+	)
+	return batch
+}
+
+type castStringUuidOp struct {
+	castOpBase
+}
+
+var _ colexecop.ResettableOperator = &castStringUuidOp{}
+var _ colexecop.ClosableOperator = &castStringUuidOp{}
+
+func (c *castStringUuidOp) Next() coldata.Batch {
+	batch := c.Input.Next()
+	n := batch.Length()
+	if n == 0 {
+		return coldata.ZeroBatch
+	}
+	sel := batch.Selection()
+	inputVec := batch.ColVec(c.colIdx)
+	outputVec := batch.ColVec(c.outputIdx)
+	toType := outputVec.Type()
+	// Remove unused warnings.
+	_ = toType
+	c.allocator.PerformOperation(
+		[]coldata.Vec{outputVec}, func() {
+			inputCol := inputVec.Bytes()
 			inputNulls := inputVec.Nulls()
 			outputCol := outputVec.Bytes()
 			outputNulls := outputVec.Nulls()
@@ -6417,16 +8084,11 @@ func (c *castJsonbStringOp) Next() coldata.Batch {
 							v := inputCol.Get(tupleIdx)
 							var r []byte
 
-							_string := v.String()
-							switch toType.Oid() {
-							case oid.T_char:
-								// "char" is supposed to truncate long values.
-								_string = util.TruncateString(_string, 1)
-							case oid.T_bpchar:
-								// bpchar types truncate trailing whitespace.
-								_string = strings.TrimRight(_string, " ")
+							_uuid, err := uuid.FromString(string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = []byte(_string)
+							r = _uuid.GetBytes()
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -6445,16 +8107,11 @@ func (c *castJsonbStringOp) Next() coldata.Batch {
 							v := inputCol.Get(tupleIdx)
 							var r []byte
 
-							_string := v.String()
-							switch toType.Oid() {
-							case oid.T_char:
-								// "char" is supposed to truncate long values.
-								_string = util.TruncateString(_string, 1)
-							case oid.T_bpchar:
-								// bpchar types truncate trailing whitespace.
-								_string = strings.TrimRight(_string, " ")
+							_uuid, err := uuid.FromString(string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = []byte(_string)
+							r = _uuid.GetBytes()
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -6475,16 +8132,11 @@ func (c *castJsonbStringOp) Next() coldata.Batch {
 							v := inputCol.Get(tupleIdx)
 							var r []byte
 
-							_string := v.String()
-							switch toType.Oid() {
-							case oid.T_char:
-								// "char" is supposed to truncate long values.
-								_string = util.TruncateString(_string, 1)
-							case oid.T_bpchar:
-								// bpchar types truncate trailing whitespace.
-								_string = strings.TrimRight(_string, " ")
+							_uuid, err := uuid.FromString(string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = []byte(_string)
+							r = _uuid.GetBytes()
 
 							outputCol.Set(tupleIdx, r)
 						}
@@ -6503,16 +8155,11 @@ func (c *castJsonbStringOp) Next() coldata.Batch {
 							v := inputCol.Get(tupleIdx)
 							var r []byte
 
-							_string := v.String()
-							switch toType.Oid() {
-							case oid.T_char:
-								// "char" is supposed to truncate long values.
-								_string = util.TruncateString(_string, 1)
-							case oid.T_bpchar:
-								// bpchar types truncate trailing whitespace.
-								_string = strings.TrimRight(_string, " ")
+							_uuid, err := uuid.FromString(string(v))
+							if err != nil {
+								colexecerror.ExpectedError(err)
 							}
-							r = []byte(_string)
+							r = _uuid.GetBytes()
 
 							outputCol.Set(tupleIdx, r)
 						}
diff --git a/pkg/sql/colexec/colexecbase/cast_tmpl.go b/pkg/sql/colexec/colexecbase/cast_tmpl.go
index 5aa39a3544ed..aac8c39b058e 100644
--- a/pkg/sql/colexec/colexecbase/cast_tmpl.go
+++ b/pkg/sql/colexec/colexecbase/cast_tmpl.go
@@ -35,12 +35,15 @@ import (
 	"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
 	"github.com/cockroachdb/cockroach/pkg/sql/colmem"
 	"github.com/cockroachdb/cockroach/pkg/sql/lex"
+	"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
+	"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
 	"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
 	"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
 	"github.com/cockroachdb/cockroach/pkg/sql/types"
 	"github.com/cockroachdb/cockroach/pkg/util"
 	"github.com/cockroachdb/cockroach/pkg/util/duration"
 	"github.com/cockroachdb/cockroach/pkg/util/json"
+	"github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
 	"github.com/cockroachdb/cockroach/pkg/util/uuid"
 	"github.com/cockroachdb/errors"
 	"github.com/lib/pq/oid"
@@ -55,6 +58,9 @@ var (
 	_ = uuid.FromBytes
 	_ = oid.T_name
 	_ = util.TruncateString
+	_ = pgcode.Syntax
+	_ = pgdate.ParseTimestamp
+	_ = pgerror.Wrapf
 )
 
 // {{/*
@@ -75,7 +81,7 @@ const _TYPE_WIDTH = 0
 // "castOp" template in the scope of this value's "callsite".
 const _GENERATE_CAST_OP = 0
 
-func _CAST(to, from, fromCol, toType interface{}) {
+func _CAST(to, from, evalCtx, toType interface{}) {
 	colexecerror.InternalError(errors.AssertionFailedf(""))
 }
 
diff --git a/pkg/sql/colexec/execgen/cmd/execgen/cast_gen_util.go b/pkg/sql/colexec/execgen/cmd/execgen/cast_gen_util.go
index c584ef189109..5a4a587ad69c 100644
--- a/pkg/sql/colexec/execgen/cmd/execgen/cast_gen_util.go
+++ b/pkg/sql/colexec/execgen/cmd/execgen/cast_gen_util.go
@@ -42,53 +42,63 @@ var nativeCastInfos = []supportedNativeCastInfo{
 	{types.Bool, types.Int4, boolToIntOrFloat},
 	{types.Bool, types.Int, boolToIntOrFloat},
 
+	{types.Bytes, types.Uuid, bytesToUUID},
+
+	{types.Date, types.Decimal, intToDecimal},
+	{types.Date, types.Float, intToFloat},
+	// Dates are represented as int64, and we currently mistakenly support dates
+	// outside of the range (#40354), so the casts from dates and from ints turn
+	// out to be the same.
+	// TODO(yuzefovich): add the checks for these casts that dates are finite.
+	{types.Date, types.Int2, getIntToIntCastFunc(64 /* fromWidth */, 16 /* toWidth */)},
+	{types.Date, types.Int4, getIntToIntCastFunc(64 /* fromWidth */, 32 /* toWidth */)},
+	{types.Date, types.Int, getIntToIntCastFunc(64 /* fromWidth */, anyWidth)},
+
 	{types.Decimal, types.Bool, decimalToBool},
+	{types.Decimal, types.Decimal, decimalToDecimal},
+	{types.Decimal, types.Float, decimalToFloat},
 	{types.Decimal, types.Int2, getDecimalToIntCastFunc(16)},
 	{types.Decimal, types.Int4, getDecimalToIntCastFunc(32)},
 	{types.Decimal, types.Int, getDecimalToIntCastFunc(anyWidth)},
-	{types.Decimal, types.Float, decimalToFloat},
-	{types.Decimal, types.Decimal, decimalToDecimal},
 
-	{types.Int2, types.Int4, getIntToIntCastFunc(16, 32)},
-	{types.Int2, types.Int, getIntToIntCastFunc(16, anyWidth)},
+	{types.Float, types.Bool, numToBool},
+	{types.Float, types.Decimal, floatToDecimal},
+	{types.Float, types.Int2, floatToInt(16, 64 /* floatWidth */)},
+	{types.Float, types.Int4, floatToInt(32, 64 /* floatWidth */)},
+	{types.Float, types.Int, floatToInt(anyWidth, 64 /* floatWidth */)},
+
 	{types.Int2, types.Bool, numToBool},
 	{types.Int2, types.Decimal, intToDecimal},
 	{types.Int2, types.Float, intToFloat},
-	{types.Int4, types.Int2, getIntToIntCastFunc(32, 16)},
-	{types.Int4, types.Int, getIntToIntCastFunc(32, anyWidth)},
+	{types.Int2, types.Int4, getIntToIntCastFunc(16, 32)},
+	{types.Int2, types.Int, getIntToIntCastFunc(16, anyWidth)},
 	{types.Int4, types.Bool, numToBool},
 	{types.Int4, types.Decimal, intToDecimal},
 	{types.Int4, types.Float, intToFloat},
-	{types.Int, types.Int2, getIntToIntCastFunc(anyWidth, 16)},
-	{types.Int, types.Int4, getIntToIntCastFunc(anyWidth, 32)},
+	{types.Int4, types.Int2, getIntToIntCastFunc(32, 16)},
+	{types.Int4, types.Int, getIntToIntCastFunc(32, anyWidth)},
 	{types.Int, types.Bool, numToBool},
 	{types.Int, types.Decimal, intToDecimal},
 	{types.Int, types.Float, intToFloat},
+	{types.Int, types.Int2, getIntToIntCastFunc(anyWidth, 16)},
+	{types.Int, types.Int4, getIntToIntCastFunc(anyWidth, 32)},
 
-	{types.Float, types.Bool, numToBool},
-	{types.Float, types.Decimal, floatToDecimal},
-	{types.Float, types.Int2, floatToInt(16, 64 /* floatWidth */)},
-	{types.Float, types.Int4, floatToInt(32, 64 /* floatWidth */)},
-	{types.Float, types.Int, floatToInt(anyWidth, 64 /* floatWidth */)},
-
-	// Dates are represented as int64, and we currently mistakenly support dates
-	// outside of the range (#40354), so the casts from dates and from ints turn
-	// out to be the same.
-	// TODO(yuzefovich): add the checks for these casts that dates are finite.
-	{types.Date, types.Int2, getIntToIntCastFunc(64 /* fromWidth */, 16 /* toWidth */)},
-	{types.Date, types.Int4, getIntToIntCastFunc(64 /* fromWidth */, 32 /* toWidth */)},
-	{types.Date, types.Int, getIntToIntCastFunc(64 /* fromWidth */, anyWidth)},
-	{types.Date, types.Float, intToFloat},
-	{types.Date, types.Decimal, intToDecimal},
-
-	{types.Bytes, types.Uuid, bytesToUUID},
+	{types.Jsonb, types.String, jsonToString},
 
 	{types.String, types.Bool, stringToBool},
 	{types.String, types.Bytes, stringToBytes},
+	{types.String, types.Date, stringToDate},
+	{types.String, types.Decimal, stringToDecimal},
+	{types.String, types.Float, stringToFloat},
+	{types.String, types.Int2, getStringToIntCastFunc(16)},
+	{types.String, types.Int4, getStringToIntCastFunc(32)},
+	{types.String, types.Int, getStringToIntCastFunc(anyWidth)},
+	{types.String, types.Interval, stringToInterval},
+	{types.String, types.Jsonb, stringToJSON},
 	{types.String, types.String, stringToString},
+	{types.String, types.Timestamp, getStringToTimestampCastFunc(true /* withoutTimezone */)},
+	{types.String, types.TimestampTZ, getStringToTimestampCastFunc(false /* withoutTimezone */)},
 	{types.String, types.Uuid, stringToUUID},
-
-	{types.Jsonb, types.String, jsonToString},
 }
 
 type supportedNativeCastInfo struct {
@@ -107,10 +117,38 @@ func boolToIntOrFloat(to, from, _, _ string) string {
 	return fmt.Sprintf(convStr, to, from)
 }
 
+func bytesToUUID(to, from, _, _ string) string {
+	convStr := `
+		_uuid, err := uuid.FromBytes(%[2]s)
+		if err != nil {
+			colexecerror.ExpectedError(err)
+		}
+		%[1]s = _uuid.GetBytes()
+	`
+	return fmt.Sprintf(convStr, to, from)
+}
+
 func decimalToBool(to, from, _, _ string) string {
 	return fmt.Sprintf("%[1]s = %[2]s.Sign() != 0", to, from)
 }
 
+func decimalToDecimal(to, from, _, toType string) string {
+	return toDecimal(fmt.Sprintf(`%[1]s.Set(&%[2]s)`, to, from), to, toType)
+}
+
+func decimalToFloat(to, from, _, _ string) string {
+	convStr := `
+		{
+			f, err := %[2]s.Float64()
+			if err != nil {
+				colexecerror.ExpectedError(tree.ErrFloatOutOfRange)
+			}
+			%[1]s = f
+		}
+`
+	return fmt.Sprintf(convStr, to, from)
+}
+
 func getDecimalToIntCastFunc(toIntWidth int32) castFunc {
 	if toIntWidth == anyWidth {
 		toIntWidth = 64
@@ -153,23 +191,6 @@ func getDecimalToIntCastFunc(toIntWidth int32) castFunc {
 	}
 }
 
-func decimalToFloat(to, from, _, _ string) string {
-	convStr := `
-		{
-			f, err := %[2]s.Float64()
-			if err != nil {
-				colexecerror.ExpectedError(tree.ErrFloatOutOfRange)
-			}
-			%[1]s = f
-		}
-`
-	return fmt.Sprintf(convStr, to, from)
-}
-
-func decimalToDecimal(to, from, _, toType string) string {
-	return toDecimal(fmt.Sprintf(`%[1]s.Set(&%[2]s)`, to, from), to, toType)
-}
-
 // toDecimal returns the templated code that performs the cast to a decimal. It
 // first will execute whatever is passed in 'conv' (the main conversion) and
 // then will perform the rounding of 'to' variable according to 'toType'.
@@ -183,6 +204,51 @@ func toDecimal(conv, to, toType string) string {
 	return fmt.Sprintf(convStr, conv, to, toType)
 }
 
+func numToBool(to, from, _, _ string) string {
+	convStr := `
+		%[1]s = %[2]s != 0
+	`
+	return fmt.Sprintf(convStr, to, from)
+}
+
+func floatToDecimal(to, from, _, toType string) string {
+	convStr := `
+		if _, err := %[1]s.SetFloat64(float64(%[2]s)); err != nil {
+			colexecerror.ExpectedError(err)
+		}
+	`
+	return toDecimal(fmt.Sprintf(convStr, to, from), to, toType)
+}
+
+func floatToInt(intWidth, floatWidth int32) func(string, string, string, string) string {
+	return func(to, from, _, _ string) string {
+		convStr := `
+			if math.IsNaN(float64(%[2]s)) || %[2]s <= float%[4]d(math.MinInt%[3]d) || %[2]s >= float%[4]d(math.MaxInt%[3]d) {
+				colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+			}
+			%[1]s = int%[3]d(%[2]s)
+		`
+		if intWidth == anyWidth {
+			intWidth = 64
+		}
+		return fmt.Sprintf(convStr, to, from, intWidth, floatWidth)
+	}
+}
+
+func intToDecimal(to, from, _, toType string) string {
+	conv := `
+		%[1]s.SetInt64(int64(%[2]s))
+	`
+	return toDecimal(fmt.Sprintf(conv, to, from), to, toType)
+}
+
+func intToFloat(to, from, _, _ string) string {
+	convStr := `
+		%[1]s = float64(%[2]s)
+	`
+	return fmt.Sprintf(convStr, to, from)
+}
+
 // getIntToIntCastFunc returns a castFunc between integers of any widths.
 func getIntToIntCastFunc(fromWidth, toWidth int32) castFunc {
 	if fromWidth == anyWidth {
@@ -220,81 +286,144 @@ func getIntToIntCastFunc(fromWidth, toWidth int32) castFunc {
 	}
 }
 
-func numToBool(to, from, _, _ string) string {
+func jsonToString(to, from, _, toType string) string {
 	convStr := `
-		%[1]s = %[2]s != 0
+		_string := %[2]s.String()
+		switch %[3]s.Oid() {
+		case oid.T_char:
+			// "char" is supposed to truncate long values.
+			_string = util.TruncateString(_string, 1)
+		case oid.T_bpchar:
+			// bpchar types truncate trailing whitespace.
+			_string = strings.TrimRight(_string, " ")
+		}
+		%[1]s = []byte(_string)
 	`
-	return fmt.Sprintf(convStr, to, from)
+	return fmt.Sprintf(convStr, to, from, toType)
 }
 
-func intToDecimal(to, from, _, toType string) string {
-	conv := `
-		%[1]s.SetInt64(int64(%[2]s))
+func stringToBool(to, from, _, _ string) string {
+	convStr := `
+		var err error
+		%[1]s, err = tree.ParseBool(string(%[2]s))
+		if err != nil {
+			colexecerror.ExpectedError(err)
+		}
 	`
-	return toDecimal(fmt.Sprintf(conv, to, from), to, toType)
+	return fmt.Sprintf(convStr, to, from)
 }
 
-func intToFloat(to, from, _, _ string) string {
+func stringToBytes(to, from, _, _ string) string {
 	convStr := `
-		%[1]s = float64(%[2]s)
+		var err error
+		%[1]s, err = lex.DecodeRawBytesToByteArrayAuto(%[2]s)
+		if err != nil {
+			colexecerror.ExpectedError(err)
+		}
 	`
 	return fmt.Sprintf(convStr, to, from)
 }
 
-func floatToDecimal(to, from, _, toType string) string {
+func stringToDate(to, from, evalCtx, _ string) string {
 	convStr := `
-		if _, err := %[1]s.SetFloat64(float64(%[2]s)); err != nil {
+		_now := %[3]s.GetRelativeParseTime()
+		_dateStyle := %[3]s.GetDateStyle()
+		_d, _, err := pgdate.ParseDate(_now, _dateStyle, string(%[2]s))
+		if err != nil {
 			colexecerror.ExpectedError(err)
 		}
-	`
+		%[1]s = _d.UnixEpochDays()
+`
+	return fmt.Sprintf(convStr, to, from, evalCtx)
+}
+
+func stringToDecimal(to, from, _, toType string) string {
+	convStr := `
+		_s := strings.TrimSpace(string(%[2]s))
+		_, res, err := tree.ExactCtx.SetString(&%[1]s, _s)
+		if res != 0 || err != nil {
+			colexecerror.ExpectedError(tree.MakeParseError(_s, types.Decimal, err))
+		}
+		switch %[1]s.Form {
+		case apd.NaNSignaling:
+			%[1]s.Form = apd.NaN
+			%[1]s.Negative = false
+		case apd.NaN:
+			%[1]s.Negative = false
+		case apd.Finite:
+			if %[1]s.IsZero() && %[1]s.Negative {
+				%[1]s.Negative = false
+			}
+		}
+`
 	return toDecimal(fmt.Sprintf(convStr, to, from), to, toType)
 }
 
-func floatToInt(intWidth, floatWidth int32) func(string, string, string, string) string {
-	return func(to, from, _, _ string) string {
+func stringToFloat(to, from, _, toType string) string {
+	convStr := `
+		_s := string(%[2]s)
+		var _err error
+		%[1]s, _err = strconv.ParseFloat(strings.TrimSpace(_s), 64)
+		if _err != nil {
+			colexecerror.ExpectedError(tree.MakeParseError(_s, %[3]s, _err))
+		}
+	`
+	return fmt.Sprintf(convStr, to, from, toType)
+}
+
+func getStringToIntCastFunc(toIntWidth int32) castFunc {
+	if toIntWidth == anyWidth {
+		toIntWidth = 64
+	}
+	return func(to, from, evalCtx, toType string) string {
+		// convStr is a format string expecting three arguments:
+		// 1. the code snippet that performs an assigment of int64 local
+		//    variable named '_i' to the result, possibly performing the bounds
+		//    checks
+		// 2. the original value variable name
+		// 3. the name of the global variable storing the type we're casting to.
 		convStr := `
-			if math.IsNaN(float64(%[2]s)) || %[2]s <= float%[4]d(math.MinInt%[3]d) || %[2]s >= float%[4]d(math.MaxInt%[3]d) {
-				colexecerror.ExpectedError(tree.ErrIntOutOfRange)
+		{
+			_s := string(%[2]s)
+			_i, err := strconv.ParseInt(strings.TrimSpace(_s), 0, 64)
+			if err != nil {
+				colexecerror.ExpectedError(tree.MakeParseError(_s, %[3]s, err))
 			}
-			%[1]s = int%[3]d(%[2]s)
-		`
-		if intWidth == anyWidth {
-			intWidth = 64
+			%[1]s
 		}
-		return fmt.Sprintf(convStr, to, from, intWidth, floatWidth)
+	`
+		return fmt.Sprintf(
+			convStr,
+			getIntToIntCastFunc(64 /* fromWidth */, toIntWidth)(to, "_i" /* from */, evalCtx, toType),
+			from,
+			toType,
+		)
 	}
 }
 
-func bytesToUUID(to, from, _, _ string) string {
+func stringToInterval(to, from, evalCtx, toType string) string {
 	convStr := `
-		_uuid, err := uuid.FromBytes(%[2]s)
+		_itm, err := %[4]s.IntervalTypeMetadata()
 		if err != nil {
 			colexecerror.ExpectedError(err)
 		}
-		%[1]s = _uuid.GetBytes()
-	`
-	return fmt.Sprintf(convStr, to, from)
-}
-
-func stringToBool(to, from, _, _ string) string {
-	convStr := `
-		var err error
-		%[1]s, err = tree.ParseBool(string(%[2]s))
+		_intervalStyle := %[3]s.GetIntervalStyle()
+		%[1]s, err = tree.ParseIntervalWithTypeMetadata(_intervalStyle, string(%[2]s), _itm)
 		if err != nil {
 			colexecerror.ExpectedError(err)
 		}
-	`
-	return fmt.Sprintf(convStr, to, from)
+`
+	return fmt.Sprintf(convStr, to, from, evalCtx, toType)
 }
 
-func stringToBytes(to, from, _, _ string) string {
+func stringToJSON(to, from, _, _ string) string {
 	convStr := `
 		var err error
-		%[1]s, err = lex.DecodeRawBytesToByteArrayAuto(%[2]s)
+		%[1]s, err = json.ParseJSON(string(%[2]s))
 		if err != nil {
-			colexecerror.ExpectedError(err)
+			colexecerror.ExpectedError(pgerror.Wrapf(err, pgcode.Syntax, "could not parse JSON"))
 		}
-	`
+`
 	return fmt.Sprintf(convStr, to, from)
 }
 
@@ -329,6 +458,32 @@ func stringToString(to, from, _, toType string) string {
 	return fmt.Sprintf(convStr, to, from, toType)
 }
 
+func getStringToTimestampCastFunc(withoutTimezone bool) func(_, _, _, _ string) string {
+	return func(to, from, evalCtx, toType string) string {
+		var parseTimestampKind string
+		if withoutTimezone {
+			parseTimestampKind = "WithoutTimezone"
+		}
+		// TODO(yuzefovich): consider introducing "prologue" sections to the
+		// cast template so that we could do some operations (e.g. _roundTo
+		// value below) only once per batch.
+		convStr := `
+		_roundTo := tree.TimeFamilyPrecisionToRoundDuration(%[4]s.Precision())
+		_now := %[3]s.GetRelativeParseTime()
+		_dateStyle := %[3]s.GetDateStyle()
+		_t, _, err := pgdate.ParseTimestamp%[5]s(_now, _dateStyle, string(%[2]s))
+		if err != nil {
+			colexecerror.ExpectedError(err)
+		}
+		%[1]s = _t.Round(_roundTo)
+		if %[1]s.After(tree.MaxSupportedTime) || %[1]s.Before(tree.MinSupportedTime) {
+			colexecerror.ExpectedError(tree.NewTimestampExceedsBoundsError(%[1]s))
+		}
+`
+		return fmt.Sprintf(convStr, to, from, evalCtx, toType, parseTimestampKind)
+	}
+}
+
 func stringToUUID(to, from, _, _ string) string {
 	convStr := `
 		_uuid, err := uuid.FromString(string(%[2]s))
@@ -340,22 +495,6 @@ func stringToUUID(to, from, _, _ string) string {
 	return fmt.Sprintf(convStr, to, from)
 }
 
-func jsonToString(to, from, _, toType string) string {
-	convStr := `
-		_string := %[2]s.String()
-		switch %[3]s.Oid() {
-		case oid.T_char:
-			// "char" is supposed to truncate long values.
-			_string = util.TruncateString(_string, 1)
-		case oid.T_bpchar:
-			// bpchar types truncate trailing whitespace.
-			_string = strings.TrimRight(_string, " ")
-		}
-		%[1]s = []byte(_string)
-	`
-	return fmt.Sprintf(convStr, to, from, toType)
-}
-
 // getDatumToNativeCastFunc returns a castFunc for casting datum-backed value
 // to a value of the specified physical representation (i.e. to natively
 // supported type). The returned castFunc assumes that there is a converter
diff --git a/pkg/sql/conn_executor_prepare.go b/pkg/sql/conn_executor_prepare.go
index 6dac885c538b..6a3d56b03752 100644
--- a/pkg/sql/conn_executor_prepare.go
+++ b/pkg/sql/conn_executor_prepare.go
@@ -105,12 +105,14 @@ func (ex *connExecutor) addPreparedStmt(
 	}
 
 	if len(prepared.TypeHints) > pgwirebase.MaxPreparedStatementArgs {
+		prepared.memAcc.Close(ctx)
 		return nil, pgwirebase.NewProtocolViolationErrorf(
 			"more than %d arguments to prepared statement: %d",
 			pgwirebase.MaxPreparedStatementArgs, len(prepared.TypeHints))
 	}
 
 	if err := prepared.memAcc.Grow(ctx, int64(len(name))); err != nil {
+		prepared.memAcc.Close(ctx)
 		return nil, err
 	}
 	ex.extraTxnState.prepStmtsNamespace.prepStmts[name] = prepared
@@ -135,16 +137,13 @@ func (ex *connExecutor) addPreparedStmt(
 //
 // placeholderHints may contain partial type information for placeholders.
 // prepare will populate the missing types. It can be nil.
-//
-// The PreparedStatement is returned (or nil if there are no results). The
-// returned PreparedStatement needs to be close()d once its no longer in use.
 func (ex *connExecutor) prepare(
 	ctx context.Context,
 	stmt Statement,
 	placeholderHints tree.PlaceholderTypes,
 	rawTypeHints []oid.Oid,
 	origin PreparedStatementOrigin,
-) (*PreparedStatement, error) {
+) (_ *PreparedStatement, retErr error) {
 
 	prepared := &PreparedStatement{
 		memAcc:   ex.sessionMon.MakeBoundAccount(),
@@ -153,6 +152,12 @@ func (ex *connExecutor) prepare(
 		createdAt: timeutil.Now(),
 		origin:    origin,
 	}
+	defer func() {
+		// Make sure to close the memory account if an error is returned.
+		if retErr != nil {
+			prepared.memAcc.Close(ctx)
+		}
+	}()
 
 	if stmt.AST == nil {
 		return prepared, nil
diff --git a/pkg/sql/opt/indexrec/testdata/geospatial-index-candidates-recommendations b/pkg/sql/opt/indexrec/testdata/geospatial-index-candidates-recommendations
index 54d2dcaf408b..c288a1da5b17 100644
--- a/pkg/sql/opt/indexrec/testdata/geospatial-index-candidates-recommendations
+++ b/pkg/sql/opt/indexrec/testdata/geospatial-index-candidates-recommendations
@@ -34,7 +34,7 @@ select
  ├── immutable
  ├── cost: 1054.09
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 941.837778
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -77,7 +77,7 @@ select
  │         ├── cost: 250.706667
  │         ├── key: (11)
  │         └── scan t2@_hyp_1
- │              ├── columns: rowid:11!null k:14!null
+ │              ├── columns: rowid:11!null geom1_inverted_key:14!null
  │              ├── inverted constraint: /14/11
  │              │    └── spans
  │              │         ├── ["B\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "B\xfd\x10\x00\x00\x00\x00\x00\x00\x00"]
@@ -136,7 +136,7 @@ select
  ├── immutable
  ├── cost: 1054.09
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 941.837778
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -179,7 +179,7 @@ select
  │         ├── cost: 250.706667
  │         ├── key: (11)
  │         └── scan t2@_hyp_1
- │              ├── columns: rowid:11!null k:14!null
+ │              ├── columns: rowid:11!null geom2_inverted_key:14!null
  │              ├── inverted constraint: /14/11
  │              │    └── spans
  │              │         ├── ["B\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "B\xfd\x10\x00\x00\x00\x00\x00\x00\x00"]
@@ -251,7 +251,7 @@ select
  ├── immutable
  ├── cost: 2246.15
  ├── scan t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    └── cost: 1236.12
  └── filters
       └── st_coveredby(geom1:4, geom2:8) [outer=(4,8), immutable, constraints=(/4: (/NULL - ]; /8: (/NULL - ])]
@@ -282,7 +282,7 @@ select
  ├── immutable
  ├── cost: 12054.09
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 941.837778
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -325,7 +325,7 @@ select
  │         ├── cost: 250.706667
  │         ├── key: (11)
  │         └── scan t2@_hyp_1
- │              ├── columns: rowid:11!null k:14!null
+ │              ├── columns: rowid:11!null geom2_inverted_key:14!null
  │              ├── inverted constraint: /14/11
  │              │    └── spans
  │              │         ├── ["B\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "B\xfd\x10\x00\x00\x00\x00\x00\x00\x00"]
@@ -383,7 +383,7 @@ select
  ├── immutable
  ├── cost: 101246.15
  ├── scan t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    └── cost: 1236.12
  └── filters
       └── st_containsproperly(geom1:4, geom2:8) [outer=(4,8), immutable, constraints=(/4: (/NULL - ]; /8: (/NULL - ])]
@@ -406,7 +406,7 @@ select
  ├── immutable
  ├── cost: 101246.15
  ├── scan t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    └── cost: 1236.12
  └── filters
       └── st_crosses(geom1:4, geom2:8) [outer=(4,8), immutable, constraints=(/4: (/NULL - ]; /8: (/NULL - ])]
@@ -444,7 +444,7 @@ select
  ├── immutable
  ├── cost: 12054.09
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 941.837778
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -487,7 +487,7 @@ select
  │         ├── cost: 250.706667
  │         ├── key: (11)
  │         └── scan t2@_hyp_1
- │              ├── columns: rowid:11!null k:14!null
+ │              ├── columns: rowid:11!null geom1_inverted_key:14!null
  │              ├── inverted constraint: /14/11
  │              │    └── spans
  │              │         ├── ["B\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "B\xfd\x10\x00\x00\x00\x00\x00\x00\x00"]
@@ -546,7 +546,7 @@ select
  ├── immutable
  ├── cost: 1058.09
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 945.837778
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -590,7 +590,7 @@ select
  │         ├── cost: 254.706667
  │         ├── key: (11)
  │         └── scan t2@_hyp_1
- │              ├── columns: rowid:11!null k:14!null
+ │              ├── columns: rowid:11!null geom1_inverted_key:14!null
  │              ├── inverted constraint: /14/11
  │              │    └── spans
  │              │         ├── ["B\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "B\xfd\x10\x00\x00\x00\x00\x00\x00\x00"]
@@ -649,7 +649,7 @@ select
  ├── immutable
  ├── cost: 101246.15
  ├── scan t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    └── cost: 1236.12
  └── filters
       └── st_overlaps(geom1:4, geom2:8) [outer=(4,8), immutable, constraints=(/4: (/NULL - ]; /8: (/NULL - ])]
@@ -693,7 +693,7 @@ select
  ├── immutable
  ├── cost: 926.978889
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 925.837778
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -732,7 +732,7 @@ select
  │         ├── cost: 234.706667
  │         ├── key: (11)
  │         └── scan t2@_hyp_1
- │              ├── columns: rowid:11!null k:14!null
+ │              ├── columns: rowid:11!null geog2_inverted_key:14!null
  │              ├── inverted constraint: /14/11
  │              │    └── spans
  │              │         ├── ["B\xfdL\x00\x00\x00\x00\x00\x00\x00", "B\xfdL\x00\x00\x00\x00\x00\x00\x00"]
@@ -786,7 +786,7 @@ select
  ├── immutable
  ├── cost: 1246.15
  ├── scan t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    └── cost: 1236.12
  └── filters
       └── st_dwithinexclusive(geog1:5, geog2:9, 10.0) [outer=(5,9), immutable, constraints=(/5: (/NULL - ]; /9: (/NULL - ])]
@@ -824,7 +824,7 @@ select
  ├── immutable
  ├── cost: 1221.85
  ├── scan t2@_hyp_1
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6!null bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6!null bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── constraint: /6/11: (/NULL - ]
  │    └── cost: 1211.92
  └── filters
@@ -855,7 +855,7 @@ select
  ├── immutable
  ├── cost: 946.988889
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 945.837778
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -899,7 +899,7 @@ select
  │         ├── cost: 254.706667
  │         ├── key: (11)
  │         └── scan t2@_hyp_3
- │              ├── columns: rowid:11!null k:14!null
+ │              ├── columns: rowid:11!null geom1_inverted_key:14!null
  │              ├── inverted constraint: /14/11
  │              │    └── spans
  │              │         ├── ["B\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "B\xfd\x10\x00\x00\x00\x00\x00\x00\x00"]
@@ -1055,7 +1055,7 @@ select
  ├── immutable
  ├── cost: 23165.2111
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 941.837778
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -1098,7 +1098,7 @@ select
  │         ├── cost: 250.706667
  │         ├── key: (11)
  │         └── scan t2@_hyp_1
- │              ├── columns: rowid:11!null k:14!null
+ │              ├── columns: rowid:11!null geom1_inverted_key:14!null
  │              ├── inverted constraint: /14/11
  │              │    └── spans
  │              │         ├── ["B\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "B\xfd\x10\x00\x00\x00\x00\x00\x00\x00"]
@@ -1162,7 +1162,7 @@ select
  ├── cost: 133.301112
  ├── fd: ()-->(1)
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 22.148889
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -1205,7 +1205,7 @@ select
  │         ├── cost: 15.2177778
  │         ├── key: (11)
  │         └── scan t2@_hyp_3
- │              ├── columns: rowid:11!null k:15!null
+ │              ├── columns: rowid:11!null geom1_inverted_key:15!null
  │              ├── constraint: /1: [/2 - /2]
  │              ├── inverted constraint: /15/11
  │              │    └── spans
@@ -1267,7 +1267,7 @@ select
  ├── immutable
  ├── cost: 12054.1
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 941.837778
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -1310,7 +1310,7 @@ select
  │         ├── cost: 250.706667
  │         ├── key: (11)
  │         └── scan t2@_hyp_2
- │              ├── columns: rowid:11!null k:14!null
+ │              ├── columns: rowid:11!null geom1_inverted_key:14!null
  │              ├── inverted constraint: /14/11
  │              │    └── spans
  │              │         ├── ["B\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "B\xfd\x10\x00\x00\x00\x00\x00\x00\x00"]
@@ -1349,7 +1349,7 @@ select
  │              └── fd: (11)-->(14)
  └── filters
       ├── st_overlaps(geom1:4, '01010000C00000000000000000000000000000000000000000000000000000000000000000') [outer=(4), immutable, constraints=(/4: (/NULL - ])]
-      └── t2.k:1 > 2 [outer=(1), constraints=(/1: [/3 - ]; tight)]
+      └── k:1 > 2 [outer=(1), constraints=(/1: [/3 - ]; tight)]
 
 # 4.Geospatial inverted + EQ + EQ: combined index (inverted + equal, inverted +
 # equal group, equal group candidates)
@@ -1385,7 +1385,7 @@ inner-join (zigzag t2@_hyp_1 t2@_hyp_2)
  ├── fd: ()-->(1,2)
  └── filters
       ├── st_overlaps(geom1:4, '01010000C00000000000000000000000000000000000000000000000000000000000000000') [outer=(4), immutable, constraints=(/4: (/NULL - ])]
-      ├── t2.k:1 = 2 [outer=(1), constraints=(/1: [/2 - /2]; tight), fd=()-->(1)]
+      ├── k:1 = 2 [outer=(1), constraints=(/1: [/2 - /2]; tight), fd=()-->(1)]
       └── i:2 = 3 [outer=(2), constraints=(/2: [/3 - /3]; tight), fd=()-->(2)]
 
 # 5. Geospatial inverted + R + R: combined index (inverted + equal, inverted +
@@ -1411,7 +1411,7 @@ select
  ├── immutable
  ├── cost: 12054.11
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 941.837778
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -1454,7 +1454,7 @@ select
  │         ├── cost: 250.706667
  │         ├── key: (11)
  │         └── scan t2@_hyp_3
- │              ├── columns: rowid:11!null k:14!null
+ │              ├── columns: rowid:11!null geom1_inverted_key:14!null
  │              ├── inverted constraint: /14/11
  │              │    └── spans
  │              │         ├── ["B\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "B\xfd\x10\x00\x00\x00\x00\x00\x00\x00"]
@@ -1493,7 +1493,7 @@ select
  │              └── fd: (11)-->(14)
  └── filters
       ├── st_overlaps(geom1:4, '01010000C00000000000000000000000000000000000000000000000000000000000000000') [outer=(4), immutable, constraints=(/4: (/NULL - ])]
-      ├── t2.k:1 > 2 [outer=(1), constraints=(/1: [/3 - ]; tight)]
+      ├── k:1 > 2 [outer=(1), constraints=(/1: [/3 - ]; tight)]
       └── i:2 > 3 [outer=(2), constraints=(/2: [/4 - ]; tight)]
 
 # 6. Geospatial inverted + EQ + R: combined index (inverted + equal, equal +
@@ -1607,7 +1607,7 @@ select
  ├── cost: 22.1900001
  ├── fd: ()-->(1)
  ├── index-join t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── cost: 22.148889
  │    └── inverted-filter
  │         ├── columns: rowid:11!null
@@ -1651,7 +1651,7 @@ select
  │         ├── cost: 15.2177778
  │         ├── key: (11)
  │         └── scan t2@_hyp_3
- │              ├── columns: rowid:11!null k:15!null
+ │              ├── columns: rowid:11!null geom1_inverted_key:15!null
  │              ├── constraint: /1: [/2 - /2]
  │              ├── inverted constraint: /15/11
  │              │    └── spans
@@ -1716,7 +1716,7 @@ select
  ├── immutable
  ├── cost: 420.726667
  ├── scan t2@_hyp_3
- │    ├── columns: t2.k:1!null i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1!null i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    ├── constraint: /1/11: [/3 - ]
  │    └── cost: 417.353333
  └── filters
@@ -1743,11 +1743,11 @@ select
  ├── immutable
  ├── cost: 2246.16
  ├── scan t2
- │    ├── columns: t2.k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
+ │    ├── columns: k:1 i:2 s:3 geom1:4 geog1:5 bbox1:6 bbox2:7 geom2:8 geog2:9 inet1:10
  │    └── cost: 1236.12
  └── filters
       ├── st_covers(geom1:4, geom2:8) [outer=(4,8), immutable, constraints=(/4: (/NULL - ]; /8: (/NULL - ])]
-      └── (~t2.k:1) = 1 [outer=(1), immutable]
+      └── (~k:1) = 1 [outer=(1), immutable]
 
 # 16. Geospatial Inverted + J + EQ: combined index between join + equal,
 # inverted + equal
diff --git a/pkg/sql/opt/indexrec/testdata/index-candidates-recommendations b/pkg/sql/opt/indexrec/testdata/index-candidates-recommendations
index fd3722e6f09d..601fcb3c89b5 100644
--- a/pkg/sql/opt/indexrec/testdata/index-candidates-recommendations
+++ b/pkg/sql/opt/indexrec/testdata/index-candidates-recommendations
@@ -1208,7 +1208,7 @@ project
  ├── immutable
  ├── cost: 807.393333
  └── index-join t4
-      ├── columns: t4.k:1 f:3 j:4
+      ├── columns: k:1 f:3 j:4
       ├── immutable
       ├── cost: 806.262222
       └── scan t4@_hyp_1
@@ -1237,7 +1237,7 @@ project
  ├── immutable
  ├── cost: 805.048889
  └── index-join t4
-      ├── columns: t4.k:1 f:3 j:4!null
+      ├── columns: k:1 f:3 j:4!null
       ├── immutable
       ├── cost: 803.928889
       └── scan t4@_hyp_1
@@ -1266,11 +1266,11 @@ project
  ├── immutable
  ├── cost: 816.998889
  └── select
-      ├── columns: t4.k:1 f:3 j:4
+      ├── columns: k:1 f:3 j:4
       ├── immutable
       ├── cost: 813.645556
       ├── index-join t4
-      │    ├── columns: t4.k:1 f:3 j:4
+      │    ├── columns: k:1 f:3 j:4
       │    ├── cost: 812.504444
       │    └── inverted-filter
       │         ├── columns: rowid:6!null
@@ -1282,7 +1282,7 @@ project
       │         ├── cost: 134.706667
       │         ├── key: (6)
       │         └── scan t4@_hyp_1
-      │              ├── columns: rowid:6!null k:9!null
+      │              ├── columns: rowid:6!null j_inverted_key:9!null
       │              ├── inverted constraint: /9/6
       │              │    └── spans
       │              │         ├── ["7\x00\x019", "7\x00\x019"]
@@ -1345,12 +1345,12 @@ project
  ├── cost: 22.1100001
  ├── fd: ()-->(1)
  └── select
-      ├── columns: t4.k:1!null f:3 j:4
+      ├── columns: k:1!null f:3 j:4
       ├── immutable
       ├── cost: 22.0566667
       ├── fd: ()-->(1)
       ├── index-join t4
-      │    ├── columns: t4.k:1 f:3 j:4
+      │    ├── columns: k:1 f:3 j:4
       │    ├── cost: 22.0155556
       │    └── inverted-filter
       │         ├── columns: rowid:6!null
@@ -1362,7 +1362,7 @@ project
       │         ├── cost: 15.2177778
       │         ├── key: (6)
       │         └── scan t4@_hyp_3
-      │              ├── columns: rowid:6!null k:10!null
+      │              ├── columns: rowid:6!null j_inverted_key:10!null
       │              ├── constraint: /1: [/1 - /1]
       │              ├── inverted constraint: /10/6
       │              │    └── spans
@@ -1402,7 +1402,7 @@ project
  ├── cost: 10.7379279
  ├── fd: ()-->(1)
  └── inner-join (zigzag t4@_hyp_1 t4@_hyp_2)
-      ├── columns: t4.k:1!null i:2!null f:3 j:4
+      ├── columns: k:1!null i:2!null f:3 j:4
       ├── eq columns: [6] = [6]
       ├── left fixed columns: [1] = [1]
       ├── right fixed columns: [2] = [2]
@@ -1411,7 +1411,7 @@ project
       ├── fd: ()-->(1,2)
       └── filters
            ├── j:4 <@ '{"foo": "1"}' [outer=(4), immutable]
-           ├── t4.k:1 = 1 [outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)]
+           ├── k:1 = 1 [outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)]
            └── i:2 = 2 [outer=(2), constraints=(/2: [/2 - /2]; tight), fd=()-->(2)]
 
 index-candidates
@@ -1431,12 +1431,12 @@ project
  ├── immutable
  ├── cost: 1125.07
  └── select
-      ├── columns: t4.k:1 f:3 j:4
+      ├── columns: k:1 f:3 j:4
       ├── immutable
       ├── cost: 1124.95
       ├── fd: ()-->(4)
       ├── scan t4
-      │    ├── columns: t4.k:1 f:3 j:4
+      │    ├── columns: k:1 f:3 j:4
       │    └── cost: 1114.92
       └── filters
            └── j:4 IS NULL [outer=(4), immutable, constraints=(/4: [/NULL - /NULL]; tight), fd=()-->(4)]
@@ -1457,11 +1457,11 @@ project
  ├── columns: k:1 f:3
  ├── cost: 1125.07
  └── select
-      ├── columns: t4.k:1 f:3 a:5
+      ├── columns: k:1 f:3 a:5
       ├── cost: 1124.95
       ├── fd: ()-->(5)
       ├── scan t4
-      │    ├── columns: t4.k:1 f:3 a:5
+      │    ├── columns: k:1 f:3 a:5
       │    └── cost: 1114.92
       └── filters
            └── a:5 IS NULL [outer=(5), constraints=(/5: [/NULL - /NULL]; tight), fd=()-->(5)]
@@ -1485,7 +1485,7 @@ project
  ├── immutable
  ├── cost: 805.048889
  └── index-join t4
-      ├── columns: t4.k:1 f:3 a:5!null
+      ├── columns: k:1 f:3 a:5!null
       ├── immutable
       ├── cost: 803.928889
       └── scan t4@_hyp_1
@@ -1511,11 +1511,11 @@ project
  ├── columns: k:1 f:3
  ├── cost: 1125.07
  └── select
-      ├── columns: t4.k:1 f:3 a:5!null
+      ├── columns: k:1 f:3 a:5!null
       ├── cost: 1124.95
       ├── fd: ()-->(5)
       ├── scan t4
-      │    ├── columns: t4.k:1 f:3 a:5
+      │    ├── columns: k:1 f:3 a:5
       │    └── cost: 1114.92
       └── filters
            └── a:5 = ARRAY[1] [outer=(5), constraints=(/5: [/ARRAY[1] - /ARRAY[1]]; tight), fd=()-->(5)]
@@ -1541,7 +1541,7 @@ project
  ├── immutable
  ├── cost: 21.9920001
  └── index-join t4
-      ├── columns: t4.k:1 i:2!null f:3 a:5!null
+      ├── columns: k:1 i:2!null f:3 a:5!null
       ├── immutable
       ├── cost: 21.9610001
       ├── fd: ()-->(2)
@@ -1573,11 +1573,11 @@ project
  ├── immutable
  ├── cost: 381.103333
  └── select
-      ├── columns: t4.k:1 f:3!null a:5!null
+      ├── columns: k:1 f:3!null a:5!null
       ├── immutable
       ├── cost: 380.716667
       ├── scan t4@_hyp_1
-      │    ├── columns: t4.k:1 f:3!null a:5
+      │    ├── columns: k:1 f:3!null a:5
       │    ├── constraint: /3/6: [/3.0000000000000004 - ]
       │    └── cost: 377.353333
       └── filters
@@ -1775,3 +1775,82 @@ project
 exec-ddl
 DROP INDEX t1@expr
 ----
+
+# Regression test for #83965.
+exec-ddl
+CREATE TABLE t83965a (c INT);
+----
+
+exec-ddl
+CREATE TABLE t83965b (c INT);
+----
+
+exec-ddl
+CREATE TABLE t83965c (c INT);
+----
+
+exec-ddl
+CREATE TABLE t83965d (c INT);
+----
+
+exec-ddl
+CREATE TABLE t83965e (c VARBIT(1)[]);
+----
+
+exec-ddl
+CREATE TABLE t83965f (c INT);
+----
+
+index-recommendations
+SELECT bool_and(t83965e.c < t83965e.c) FROM t83965f, t83965a, t83965c, t83965b, t83965d, t83965e
+----
+No index recommendations.
+--
+Optimal Plan.
+scalar-group-by
+ ├── columns: bool_and:26
+ ├── cardinality: [1 - 1]
+ ├── cost: 4.00225225e+16
+ ├── key: ()
+ ├── fd: ()-->(26)
+ ├── project
+ │    ├── columns: column25:25
+ │    ├── cost: 3.00225225e+16
+ │    ├── inner-join (cross)
+ │    │    ├── columns: t83965e.c:21
+ │    │    ├── cost: 1.00225225e+16
+ │    │    ├── inner-join (cross)
+ │    │    │    ├── columns: t83965e.c:21
+ │    │    │    ├── cost: 1.00225225e+13
+ │    │    │    ├── inner-join (cross)
+ │    │    │    │    ├── columns: t83965e.c:21
+ │    │    │    │    ├── cost: 1.00225268e+10
+ │    │    │    │    ├── inner-join (cross)
+ │    │    │    │    │    ├── columns: t83965e.c:21
+ │    │    │    │    │    ├── cost: 10025690.6
+ │    │    │    │    │    ├── inner-join (cross)
+ │    │    │    │    │    │    ├── columns: t83965e.c:21
+ │    │    │    │    │    │    ├── cost: 12128.7063
+ │    │    │    │    │    │    ├── scan t83965d
+ │    │    │    │    │    │    │    └── cost: 1044.22
+ │    │    │    │    │    │    ├── scan t83965e
+ │    │    │    │    │    │    │    ├── columns: t83965e.c:21
+ │    │    │    │    │    │    │    └── cost: 1054.32
+ │    │    │    │    │    │    └── filters (true)
+ │    │    │    │    │    ├── scan t83965b
+ │    │    │    │    │    │    └── cost: 1044.22
+ │    │    │    │    │    └── filters (true)
+ │    │    │    │    ├── scan t83965c
+ │    │    │    │    │    └── cost: 1044.22
+ │    │    │    │    └── filters (true)
+ │    │    │    ├── scan t83965a
+ │    │    │    │    └── cost: 1044.22
+ │    │    │    └── filters (true)
+ │    │    ├── scan t83965f
+ │    │    │    └── cost: 1044.22
+ │    │    └── filters (true)
+ │    └── projections
+ │         └── (t83965e.c:21 IS NOT DISTINCT FROM CAST(NULL AS VARBIT(1)[])) AND CAST(NULL AS BOOL) [as=column25:25, outer=(21)]
+ └── aggregations
+      └── bool-and [as=bool_and:26, outer=(25)]
+           └── column25:25
diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go
index f9ff3025a00e..060aa495d9e8 100644
--- a/pkg/sql/opt/metadata.go
+++ b/pkg/sql/opt/metadata.go
@@ -610,14 +610,15 @@ func (md *Metadata) QualifiedAlias(colID ColumnID, fullyQualify bool, catalog ca
 // TableMeta instance stores.
 func (md *Metadata) UpdateTableMeta(tables map[cat.StableID]cat.Table) {
 	for i := range md.tables {
-		if tab, ok := tables[md.tables[i].Table.ID()]; ok {
+		oldTable := md.tables[i].Table
+		if newTable, ok := tables[oldTable.ID()]; ok {
 			// If there are any inverted hypothetical indexes, the hypothetical table
 			// will have extra inverted columns added. Add any new inverted columns to
 			// the metadata.
-			for j, n := md.tables[i].Table.ColumnCount(), tab.ColumnCount(); j < n; j++ {
-				md.AddColumn(string(tab.Column(i).ColName()), types.Bytes)
+			for j, n := oldTable.ColumnCount(), newTable.ColumnCount(); j < n; j++ {
+				md.AddColumn(string(newTable.Column(j).ColName()), types.Bytes)
 			}
-			md.tables[i].Table = tab
+			md.tables[i].Table = newTable
 		}
 	}
 }
diff --git a/pkg/sql/prepared_stmt.go b/pkg/sql/prepared_stmt.go
index d7513cc26c27..cc820d8bb503 100644
--- a/pkg/sql/prepared_stmt.go
+++ b/pkg/sql/prepared_stmt.go
@@ -157,8 +157,12 @@ func (ex *connExecutor) makePreparedPortal(
 func (p PreparedPortal) accountForCopy(
 	ctx context.Context, prepStmtsNamespaceMemAcc *mon.BoundAccount, portalName string,
 ) error {
+	if err := prepStmtsNamespaceMemAcc.Grow(ctx, p.size(portalName)); err != nil {
+		return err
+	}
+	// Only increment the reference if we're going to keep it.
 	p.Stmt.incRef(ctx)
-	return prepStmtsNamespaceMemAcc.Grow(ctx, p.size(portalName))
+	return nil
 }
 
 // close closes this portal.
diff --git a/pkg/sql/sem/eval/cast.go b/pkg/sql/sem/eval/cast.go
index 5aa766950f8f..9f13ee21932d 100644
--- a/pkg/sql/sem/eval/cast.go
+++ b/pkg/sql/sem/eval/cast.go
@@ -159,7 +159,9 @@ func performCastWithoutPrecisionTruncation(
 		case *tree.DDecimal:
 			return tree.MakeDBool(v.Sign() != 0), nil
 		case *tree.DString:
-			return tree.ParseDBool(strings.TrimSpace(string(*v)))
+			// No need to trim the spaces explicitly since ParseDBool does that
+			// itself.
+			return tree.ParseDBool(string(*v))
 		case *tree.DCollatedString:
 			return tree.ParseDBool(v.Contents)
 		case *tree.DJSON:
diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go
index 527a311427f6..60cd0a61800c 100644
--- a/pkg/sql/sem/tree/datum.go
+++ b/pkg/sql/sem/tree/datum.go
@@ -1046,7 +1046,7 @@ func (d *DDecimal) SetString(s string) error {
 	//_, res, err := HighPrecisionCtx.SetString(&d.Decimal, s)
 	_, res, err := ExactCtx.SetString(&d.Decimal, s)
 	if res != 0 || err != nil {
-		return MakeParseError(s, types.Decimal, nil)
+		return MakeParseError(s, types.Decimal, err)
 	}
 	switch d.Form {
 	case apd.NaNSignaling:
@@ -2260,8 +2260,7 @@ func ParseDTime(
 
 	t, dependsOnContext, err := pgdate.ParseTimeWithoutTimezone(now, dateStyle(ctx), s)
 	if err != nil {
-		// Build our own error message to avoid exposing the dummy date.
-		return nil, false, MakeParseError(s, types.Time, nil)
+		return nil, false, MakeParseError(s, types.Time, err)
 	}
 	return MakeDTime(timeofday.FromTime(t).Round(precision)), dependsOnContext, nil
 }
@@ -2532,6 +2531,16 @@ func (d *DTimeTZ) Size() uintptr {
 	return unsafe.Sizeof(*d)
 }
 
+// NewTimestampExceedsBoundsError returns a new "exceeds supported timestamp
+// bounds" error for the given timestamp, with the correct pgcode.
+func NewTimestampExceedsBoundsError(t time.Time) error {
+	return pgerror.Newf(
+		pgcode.InvalidTimeZoneDisplacementValue,
+		"timestamp %q exceeds supported timestamp bounds",
+		t.Format(time.RFC3339),
+	)
+}
+
 // DTimestamp is the timestamp Datum.
 type DTimestamp struct {
 	// Time always has UTC location.
@@ -2542,9 +2551,7 @@ type DTimestamp struct {
 func MakeDTimestamp(t time.Time, precision time.Duration) (*DTimestamp, error) {
 	ret := t.Round(precision)
 	if ret.After(MaxSupportedTime) || ret.Before(MinSupportedTime) {
-		return nil, pgerror.Newf(
-			pgcode.InvalidTimeZoneDisplacementValue,
-			"timestamp %q exceeds supported timestamp bounds", ret.Format(time.RFC3339))
+		return nil, NewTimestampExceedsBoundsError(ret)
 	}
 	return &DTimestamp{Time: ret}, nil
 }
@@ -2827,9 +2834,7 @@ type DTimestampTZ struct {
 func MakeDTimestampTZ(t time.Time, precision time.Duration) (*DTimestampTZ, error) {
 	ret := t.Round(precision)
 	if ret.After(MaxSupportedTime) || ret.Before(MinSupportedTime) {
-		return nil, pgerror.Newf(
-			pgcode.InvalidTimeZoneDisplacementValue,
-			"timestamp %q exceeds supported timestamp bounds", ret.Format(time.RFC3339))
+		return nil, NewTimestampExceedsBoundsError(ret)
 	}
 	return &DTimestampTZ{Time: ret}, nil
 }
@@ -3023,9 +3028,8 @@ func MustBeDInterval(e Expr) *DInterval {
 
 // NewDInterval creates a new DInterval.
 func NewDInterval(d duration.Duration, itm types.IntervalTypeMetadata) *DInterval {
-	ret := &DInterval{Duration: d}
-	truncateDInterval(ret, itm)
-	return ret
+	truncateInterval(&d, itm)
+	return &DInterval{Duration: d}
 }
 
 // ParseDInterval parses and returns the *DInterval Datum value represented by the provided
@@ -3034,28 +3038,28 @@ func ParseDInterval(style duration.IntervalStyle, s string) (*DInterval, error)
 	return ParseDIntervalWithTypeMetadata(style, s, types.DefaultIntervalTypeMetadata)
 }
 
-// truncateDInterval truncates the input DInterval downward to the nearest
+// truncateInterval truncates the input interval downward to the nearest
 // interval quantity specified by the DurationField input.
 // If precision is set for seconds, this will instead round at the second layer.
-func truncateDInterval(d *DInterval, itm types.IntervalTypeMetadata) {
+func truncateInterval(d *duration.Duration, itm types.IntervalTypeMetadata) {
 	switch itm.DurationField.DurationType {
 	case types.IntervalDurationType_YEAR:
-		d.Duration.Months = d.Duration.Months - d.Duration.Months%12
-		d.Duration.Days = 0
-		d.Duration.SetNanos(0)
+		d.Months = d.Months - d.Months%12
+		d.Days = 0
+		d.SetNanos(0)
 	case types.IntervalDurationType_MONTH:
-		d.Duration.Days = 0
-		d.Duration.SetNanos(0)
+		d.Days = 0
+		d.SetNanos(0)
 	case types.IntervalDurationType_DAY:
-		d.Duration.SetNanos(0)
+		d.SetNanos(0)
 	case types.IntervalDurationType_HOUR:
-		d.Duration.SetNanos(d.Duration.Nanos() - d.Duration.Nanos()%time.Hour.Nanoseconds())
+		d.SetNanos(d.Nanos() - d.Nanos()%time.Hour.Nanoseconds())
 	case types.IntervalDurationType_MINUTE:
-		d.Duration.SetNanos(d.Duration.Nanos() - d.Duration.Nanos()%time.Minute.Nanoseconds())
+		d.SetNanos(d.Nanos() - d.Nanos()%time.Minute.Nanoseconds())
 	case types.IntervalDurationType_SECOND, types.IntervalDurationType_UNSET:
 		if itm.PrecisionIsSet || itm.Precision > 0 {
 			prec := TimeFamilyPrecisionToRoundDuration(itm.Precision)
-			d.Duration.SetNanos(time.Duration(d.Duration.Nanos()).Round(prec).Nanoseconds())
+			d.SetNanos(time.Duration(d.Nanos()).Round(prec).Nanoseconds())
 		}
 	}
 }
@@ -3066,17 +3070,29 @@ func truncateDInterval(d *DInterval, itm types.IntervalTypeMetadata) {
 func ParseDIntervalWithTypeMetadata(
 	style duration.IntervalStyle, s string, itm types.IntervalTypeMetadata,
 ) (*DInterval, error) {
-	d, err := parseDInterval(style, s, itm)
+	d, err := ParseIntervalWithTypeMetadata(style, s, itm)
 	if err != nil {
 		return nil, err
 	}
-	truncateDInterval(d, itm)
+	return &DInterval{Duration: d}, nil
+}
+
+// ParseIntervalWithTypeMetadata is the same as ParseDIntervalWithTypeMetadata
+// but returns a duration.Duration.
+func ParseIntervalWithTypeMetadata(
+	style duration.IntervalStyle, s string, itm types.IntervalTypeMetadata,
+) (duration.Duration, error) {
+	d, err := parseInterval(style, s, itm)
+	if err != nil {
+		return d, err
+	}
+	truncateInterval(&d, itm)
 	return d, nil
 }
 
-func parseDInterval(
+func parseInterval(
 	style duration.IntervalStyle, s string, itm types.IntervalTypeMetadata,
-) (*DInterval, error) {
+) (duration.Duration, error) {
 	// At this time the only supported interval formats are:
 	// - SQL standard.
 	// - Postgres compatible.
@@ -3086,34 +3102,34 @@ func parseDInterval(
 
 	// If it's a blank string, exit early.
 	if len(s) == 0 {
-		return nil, MakeParseError(s, types.Interval, nil)
+		return duration.Duration{}, MakeParseError(s, types.Interval, nil)
 	}
 	if s[0] == 'P' {
 		// If it has a leading P we're most likely working with an iso8601
 		// interval.
 		dur, err := iso8601ToDuration(s)
 		if err != nil {
-			return nil, MakeParseError(s, types.Interval, err)
+			return duration.Duration{}, MakeParseError(s, types.Interval, err)
 		}
-		return &DInterval{Duration: dur}, nil
+		return dur, nil
 	}
 	if strings.IndexFunc(s, unicode.IsLetter) == -1 {
 		// If it has no letter, then we're most likely working with a SQL standard
 		// interval, as both postgres and golang have letter(s) and iso8601 has been tested.
 		dur, err := sqlStdToDuration(s, itm)
 		if err != nil {
-			return nil, MakeParseError(s, types.Interval, err)
+			return duration.Duration{}, MakeParseError(s, types.Interval, err)
 		}
-		return &DInterval{Duration: dur}, nil
+		return dur, nil
 	}
 
 	// We're either a postgres string or a Go duration.
 	// Our postgres syntax parser also supports golang, so just use that for both.
 	dur, err := parseDuration(style, s, itm)
 	if err != nil {
-		return nil, MakeParseError(s, types.Interval, err)
+		return duration.Duration{}, MakeParseError(s, types.Interval, err)
 	}
-	return &DInterval{Duration: dur}, nil
+	return dur, nil
 }
 
 // ResolvedType implements the TypedExpr interface.
diff --git a/pkg/sql/sem/tree/interval_test.go b/pkg/sql/sem/tree/interval_test.go
index 148de5412c05..5523fecb07a2 100644
--- a/pkg/sql/sem/tree/interval_test.go
+++ b/pkg/sql/sem/tree/interval_test.go
@@ -195,11 +195,11 @@ func TestValidSQLIntervalSyntax(t *testing.T) {
 					}
 
 					// Test that a Datum recognizes the format.
-					di, err := parseDInterval(styleVal, test.input, test.itm)
+					di, err := parseInterval(styleVal, test.input, test.itm)
 					if err != nil {
 						t.Fatalf(`%q: unrecognized as datum: %v`, test.input, err)
 					}
-					s3 := di.Duration.String()
+					s3 := di.String()
 					if s3 != test.output {
 						t.Fatalf(`%q: as datum, got "%s", expected "%s"`, test.input, s3, test.output)
 					}
@@ -462,11 +462,11 @@ func TestPGIntervalSyntax(t *testing.T) {
 					}
 
 					// Test that a Datum recognizes the format.
-					di, err := parseDInterval(styleVal, test.input, test.itm)
+					di, err := parseInterval(styleVal, test.input, test.itm)
 					if err != nil {
 						t.Fatalf(`%q: unrecognized as datum: %v`, test.input, err)
 					}
-					s3 := di.Duration.String()
+					s3 := di.String()
 					if s3 != expected {
 						t.Fatalf(`%q: as datum, got "%s", expected "%s"`, test.input, s3, expected)
 					}
@@ -585,11 +585,11 @@ func TestISO8601IntervalSyntax(t *testing.T) {
 					}
 
 					// Test that a Datum recognizes the format.
-					di, err := parseDInterval(duration.IntervalStyle(duration.IntervalStyle_value[style]), test.input, test.itm)
+					di, err := parseInterval(duration.IntervalStyle(duration.IntervalStyle_value[style]), test.input, test.itm)
 					if err != nil {
 						t.Fatalf(`%q: unrecognized as datum: %v`, test.input, err)
 					}
-					s3 := di.Duration.String()
+					s3 := di.String()
 					if s3 != test.output {
 						t.Fatalf(`%q: as datum, got "%s", expected "%s"`, test.input, s3, test.output)
 					}
@@ -597,11 +597,11 @@ func TestISO8601IntervalSyntax(t *testing.T) {
 
 				// Test that ISO 8601 output format also round-trips
 				s4 := dur.ISO8601String()
-				di2, err := parseDInterval(duration.IntervalStyle(duration.IntervalStyle_value[style]), s4, test.itm)
+				di2, err := parseInterval(duration.IntervalStyle(duration.IntervalStyle_value[style]), s4, test.itm)
 				if err != nil {
 					t.Fatalf(`%q: ISO8601String "%s" unrecognized as datum: %v`, test.input, s4, err)
 				}
-				s5 := di2.Duration.String()
+				s5 := di2.String()
 				if s != s5 {
 					t.Fatalf(`%q: repr "%s" does not round-trip, got %s instead`, test.input, s4, s5)
 
diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel
index 85831e80cc84..fdb8b61eb0e1 100644
--- a/pkg/storage/BUILD.bazel
+++ b/pkg/storage/BUILD.bazel
@@ -104,6 +104,7 @@ go_test(
         "disk_map_test.go",
         "engine_key_test.go",
         "engine_test.go",
+        "external_helpers_test.go",
         "intent_interleaving_iter_test.go",
         "intent_reader_writer_test.go",
         "main_test.go",
diff --git a/pkg/storage/external_helpers_test.go b/pkg/storage/external_helpers_test.go
new file mode 100644
index 000000000000..71c05dfcb610
--- /dev/null
+++ b/pkg/storage/external_helpers_test.go
@@ -0,0 +1,21 @@
+// 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 storage_test
+
+import (
+	"github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap"
+	"github.com/cockroachdb/cockroach/pkg/storage"
+)
+
+func init() {
+	storage.TestingUserDescID = bootstrap.TestingUserDescID
+	storage.TestingUserTableDataMin = bootstrap.TestingUserTableDataMin
+}
diff --git a/pkg/storage/multi_iterator.go b/pkg/storage/multi_iterator.go
index 5a4d248f12fa..0f4c2f00561a 100644
--- a/pkg/storage/multi_iterator.go
+++ b/pkg/storage/multi_iterator.go
@@ -20,6 +20,8 @@ import (
 const invalidIdxSentinel = -1
 
 // multiIterator multiplexes iteration over a number of SimpleMVCCIterators.
+//
+// TODO (msbutler): remove the multiIterator and replace all uses with PebbleSSTIterator
 type multiIterator struct {
 	iters []SimpleMVCCIterator
 	// The index into `iters` of the iterator currently being pointed at.
diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go
index 1498426de504..05cd64b933cd 100644
--- a/pkg/storage/mvcc_test.go
+++ b/pkg/storage/mvcc_test.go
@@ -26,7 +26,6 @@ import (
 	"github.com/cockroachdb/cockroach/pkg/keys"
 	"github.com/cockroachdb/cockroach/pkg/roachpb"
 	"github.com/cockroachdb/cockroach/pkg/settings/cluster"
-	"github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap"
 	"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
 	"github.com/cockroachdb/cockroach/pkg/testutils"
 	"github.com/cockroachdb/cockroach/pkg/testutils/skip"
@@ -4374,13 +4373,16 @@ func TestFindSplitKey(t *testing.T) {
 	}
 }
 
+// Injected via `external_helpers_test.go`.
+var TestingUserDescID func(offset uint32) uint32
+
 // TestFindValidSplitKeys verifies split keys are located such that
 // they avoid splits through invalid key ranges.
 func TestFindValidSplitKeys(t *testing.T) {
 	defer leaktest.AfterTest(t)()
 	defer log.Scope(t).Close(t)
 
-	userID := bootstrap.TestingUserDescID(0)
+	userID := TestingUserDescID(0)
 	// Manually creates rows corresponding to the schema:
 	// CREATE TABLE t (id1 STRING, id2 STRING, ... PRIMARY KEY (id1, id2, ...))
 	addTablePrefix := func(prefix roachpb.Key, id uint32, rowVals ...string) roachpb.Key {
@@ -4574,7 +4576,7 @@ func TestFindValidSplitKeys(t *testing.T) {
 				addColFam(tablePrefix(userID, "b"), 1),
 				addColFam(tablePrefix(userID, "c"), 1),
 			},
-			rangeStart: keys.SystemSQLCodec.TablePrefix(bootstrap.TestingUserDescID(0)),
+			rangeStart: keys.SystemSQLCodec.TablePrefix(TestingUserDescID(0)),
 			expSplit:   tablePrefix(userID, "b"),
 			expError:   false,
 		},
@@ -5014,6 +5016,9 @@ func (it *seekLTTrackingIterator) SeekLT(k MVCCKey) {
 	it.MVCCIterator.SeekLT(k)
 }
 
+// Injected via `external_helpers_test.go`.
+var TestingUserTableDataMin func() roachpb.Key
+
 // TestMVCCGarbageCollectUsesSeekLTAppropriately ensures that the garbage
 // collection only utilizes SeekLT if there are enough undeleted versions.
 func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) {
@@ -5037,7 +5042,7 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) {
 		batch := engine.NewBatch()
 		defer batch.Close()
 		it := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{
-			UpperBound: bootstrap.TestingUserTableDataMin(),
+			UpperBound: TestingUserTableDataMin(),
 			LowerBound: keys.MaxKey,
 		})
 		defer it.Close()