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()