diff --git a/core/dataloader/csvdataloader.go b/core/dataloader/csvdataloader.go index 6a6995da50..bcc9b24ecd 100644 --- a/core/dataloader/csvdataloader.go +++ b/core/dataloader/csvdataloader.go @@ -32,7 +32,7 @@ type CsvDataLoader struct { results LoadDataResults partialRecord string rowInserter sql.RowInserter - colTypes []types.DoltgresType + colTypes []*types.DoltgresType sch sql.Schema removeHeader bool delimiter string @@ -69,7 +69,7 @@ func NewCsvDataLoader(ctx *sql.Context, table sql.InsertableTable, delimiter str // LoadChunk implements the DataLoader interface func (cdl *CsvDataLoader) LoadChunk(ctx *sql.Context, data *bufio.Reader) error { - combinedReader := newStringPrefixReader(cdl.partialRecord, data) + combinedReader := NewStringPrefixReader(cdl.partialRecord, data) cdl.partialRecord = "" reader, err := newCsvReaderWithDelimiter(combinedReader, cdl.delimiter) diff --git a/core/dataloader/csvreader.go b/core/dataloader/csvreader.go index 8cdd9d497f..3c75f66cda 100644 --- a/core/dataloader/csvreader.go +++ b/core/dataloader/csvreader.go @@ -67,7 +67,7 @@ type csvReader struct { fieldsPerRecord int } -// newCsvReader creates a csvReader from a given ReadCloser. +// NewCsvReader creates a csvReader from a given ReadCloser. // // The interpretation of the bytes of the supplied reader is a little murky. If // there is a UTF8, UTF16LE or UTF16BE BOM as the first bytes read, then the @@ -75,7 +75,7 @@ type csvReader struct { // encoding. If we are not in any of those marked encodings, then some of the // bytes go uninterpreted until we get to the SQL layer. It is currently the // case that newlines must be encoded as a '0xa' byte. -func newCsvReader(r io.ReadCloser) (*csvReader, error) { +func NewCsvReader(r io.ReadCloser) (*csvReader, error) { return newCsvReaderWithDelimiter(r, ",") } diff --git a/core/dataloader/dataloader.go b/core/dataloader/dataloader.go index 641285c073..bc164fb47d 100644 --- a/core/dataloader/dataloader.go +++ b/core/dataloader/dataloader.go @@ -51,11 +51,11 @@ type LoadDataResults struct { // getColumnTypes examines |sch| and returns a slice of DoltgresTypes in the order of the schema's columns. If any // columns in the schema are not DoltgresType instances, an error is returned. -func getColumnTypes(sch sql.Schema) ([]types.DoltgresType, error) { - colTypes := make([]types.DoltgresType, len(sch)) +func getColumnTypes(sch sql.Schema) ([]*types.DoltgresType, error) { + colTypes := make([]*types.DoltgresType, len(sch)) for i, col := range sch { var ok bool - colTypes[i], ok = col.Type.(types.DoltgresType) + colTypes[i], ok = col.Type.(*types.DoltgresType) if !ok { return nil, fmt.Errorf("unsupported column type: name: %s, type: %T", col.Name, col.Type) } diff --git a/core/dataloader/string_prefix_reader.go b/core/dataloader/string_prefix_reader.go index 2cb167e32c..efe993cc73 100644 --- a/core/dataloader/string_prefix_reader.go +++ b/core/dataloader/string_prefix_reader.go @@ -27,9 +27,9 @@ type stringPrefixReader struct { var _ io.ReadCloser = (*stringPrefixReader)(nil) -// newStringPrefixReader creates a new stringPrefixReader that first returns the data in |prefix| and +// NewStringPrefixReader creates a new stringPrefixReader that first returns the data in |prefix| and // then returns data from |reader|. -func newStringPrefixReader(prefix string, reader io.Reader) *stringPrefixReader { +func NewStringPrefixReader(prefix string, reader io.Reader) *stringPrefixReader { return &stringPrefixReader{ prefix: prefix, reader: reader, diff --git a/core/dataloader/tabdataloader.go b/core/dataloader/tabdataloader.go index 87c6496103..b0e0dcbf86 100644 --- a/core/dataloader/tabdataloader.go +++ b/core/dataloader/tabdataloader.go @@ -34,7 +34,7 @@ type TabularDataLoader struct { results LoadDataResults partialLine string rowInserter sql.RowInserter - colTypes []types.DoltgresType + colTypes []*types.DoltgresType sch sql.Schema delimiterChar string nullChar string diff --git a/core/typecollection/merge.go b/core/typecollection/merge.go index f46effc04e..68db5deb48 100644 --- a/core/typecollection/merge.go +++ b/core/typecollection/merge.go @@ -21,15 +21,14 @@ import ( "github.com/dolthub/doltgresql/server/types" ) -// Merge handles merging sequences on our root and their root. +// Merge handles merging types on our root and their root. func Merge(ctx context.Context, ourCollection, theirCollection, ancCollection *TypeCollection) (*TypeCollection, error) { mergedCollection := ourCollection.Clone() - err := theirCollection.IterateTypes(func(schema string, theirType *types.Type) error { + err := theirCollection.IterateTypes(func(schema string, theirType *types.DoltgresType) error { // If we don't have the type, then we simply add it mergedType, exists := mergedCollection.GetType(schema, theirType.Name) if !exists { - newSeq := *theirType - return mergedCollection.CreateType(schema, &newSeq) + return mergedCollection.CreateType(schema, theirType) } // Different types with the same name cannot be merged. (e.g.: 'domain' type and 'base' type with the same name) diff --git a/core/typecollection/serialization.go b/core/typecollection/serialization.go index 9dd1112fa0..7d601f8470 100644 --- a/core/typecollection/serialization.go +++ b/core/typecollection/serialization.go @@ -19,8 +19,6 @@ import ( "fmt" "sync" - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/doltgresql/server/types" "github.com/dolthub/doltgresql/utils" ) @@ -34,7 +32,7 @@ func (pgs *TypeCollection) Serialize(ctx context.Context) ([]byte, error) { pgs.mutex.Lock() defer pgs.mutex.Unlock() - // Write all the Types to the writer + // Write all the types to the writer writer := utils.NewWriter(256) writer.VariableUint(0) // Version schemaMapKeys := utils.GetMapKeysSorted(pgs.schemaMap) @@ -46,42 +44,8 @@ func (pgs *TypeCollection) Serialize(ctx context.Context) ([]byte, error) { writer.VariableUint(uint64(len(nameMapKeys))) for _, nameMapKey := range nameMapKeys { typ := nameMap[nameMapKey] - writer.Uint32(typ.Oid) - writer.String(typ.Name) - writer.String(typ.Owner) - writer.Int16(typ.Length) - writer.Bool(typ.PassedByVal) - writer.String(string(typ.TypType)) - writer.String(string(typ.TypCategory)) - writer.Bool(typ.IsPreferred) - writer.Bool(typ.IsDefined) - writer.String(typ.Delimiter) - writer.Uint32(typ.RelID) - writer.String(typ.SubscriptFunc) - writer.Uint32(typ.Elem) - writer.Uint32(typ.Array) - writer.String(typ.InputFunc) - writer.String(typ.OutputFunc) - writer.String(typ.ReceiveFunc) - writer.String(typ.SendFunc) - writer.String(typ.ModInFunc) - writer.String(typ.ModOutFunc) - writer.String(typ.AnalyzeFunc) - writer.String(string(typ.Align)) - writer.String(string(typ.Storage)) - writer.Bool(typ.NotNull) - writer.Uint32(typ.BaseTypeOID) - writer.Int32(typ.TypMod) - writer.Int32(typ.NDims) - writer.Uint32(typ.Collation) - writer.String(typ.DefaulBin) - writer.String(typ.Default) - writer.String(typ.Acl) - writer.VariableUint(uint64(len(typ.Checks))) - for _, check := range typ.Checks { - writer.String(check.Name) - writer.String(check.CheckExpression) - } + data := typ.Serialize() + writer.ByteSlice(data) } } @@ -93,11 +57,11 @@ func (pgs *TypeCollection) Serialize(ctx context.Context) ([]byte, error) { func Deserialize(ctx context.Context, data []byte) (*TypeCollection, error) { if len(data) == 0 { return &TypeCollection{ - schemaMap: make(map[string]map[string]*types.Type), + schemaMap: make(map[string]map[string]*types.DoltgresType), mutex: &sync.RWMutex{}, }, nil } - schemaMap := make(map[string]map[string]*types.Type) + schemaMap := make(map[string]map[string]*types.DoltgresType) reader := utils.NewReader(data) version := reader.VariableUint() if version != 0 { @@ -109,51 +73,15 @@ func Deserialize(ctx context.Context, data []byte) (*TypeCollection, error) { for i := uint64(0); i < numOfSchemas; i++ { schemaName := reader.String() numOfTypes := reader.VariableUint() - nameMap := make(map[string]*types.Type) + nameMap := make(map[string]*types.DoltgresType) for j := uint64(0); j < numOfTypes; j++ { - typ := &types.Type{Schema: schemaName} - typ.Oid = reader.Uint32() - typ.Name = reader.String() - typ.Owner = reader.String() - typ.Length = reader.Int16() - typ.PassedByVal = reader.Bool() - typ.TypType = types.TypeType(reader.String()) - typ.TypCategory = types.TypeCategory(reader.String()) - typ.IsPreferred = reader.Bool() - typ.IsDefined = reader.Bool() - typ.Delimiter = reader.String() - typ.RelID = reader.Uint32() - typ.SubscriptFunc = reader.String() - typ.Elem = reader.Uint32() - typ.Array = reader.Uint32() - typ.InputFunc = reader.String() - typ.OutputFunc = reader.String() - typ.ReceiveFunc = reader.String() - typ.SendFunc = reader.String() - typ.ModInFunc = reader.String() - typ.ModOutFunc = reader.String() - typ.AnalyzeFunc = reader.String() - typ.Align = types.TypeAlignment(reader.String()) - typ.Storage = types.TypeStorage(reader.String()) - typ.NotNull = reader.Bool() - typ.BaseTypeOID = reader.Uint32() - typ.TypMod = reader.Int32() - typ.NDims = reader.Int32() - typ.Collation = reader.Uint32() - typ.DefaulBin = reader.String() - typ.Default = reader.String() - typ.Acl = reader.String() - numOfChecks := reader.VariableUint() - for k := uint64(0); k < numOfChecks; k++ { - checkName := reader.String() - checkExpr := reader.String() - typ.Checks = append(typ.Checks, &sql.CheckDefinition{ - Name: checkName, - CheckExpression: checkExpr, - Enforced: true, - }) + typData := reader.ByteSlice() + typ, err := types.DeserializeType(typData) + if err != nil { + return nil, err } - nameMap[typ.Name] = typ + dt := typ.(*types.DoltgresType) + nameMap[dt.Name] = dt } schemaMap[schemaName] = nameMap } diff --git a/core/typecollection/typecollection.go b/core/typecollection/typecollection.go index 4fab3d90e9..f24d11672c 100644 --- a/core/typecollection/typecollection.go +++ b/core/typecollection/typecollection.go @@ -21,15 +21,15 @@ import ( "github.com/dolthub/doltgresql/server/types" ) -// TypeCollection contains a collection of Types. +// TypeCollection contains a collection of types. type TypeCollection struct { - schemaMap map[string]map[string]*types.Type + schemaMap map[string]map[string]*types.DoltgresType mutex *sync.RWMutex } -// GetType returns the Type with the given schema and name. -// Returns nil if the Type cannot be found. -func (pgs *TypeCollection) GetType(schName, typName string) (*types.Type, bool) { +// GetType returns the type with the given schema and name. +// Returns nil if the type cannot be found. +func (pgs *TypeCollection) GetType(schName, typName string) (*types.DoltgresType, bool) { pgs.mutex.RLock() defer pgs.mutex.RUnlock() @@ -41,9 +41,9 @@ func (pgs *TypeCollection) GetType(schName, typName string) (*types.Type, bool) return nil, false } -// GetDomainType returns a domain Type with the given schema and name. -// Returns nil if the Type cannot be found. It checks for type of Type for domain type. -func (pgs *TypeCollection) GetDomainType(schName, typName string) (*types.Type, bool) { +// GetDomainType returns a domain type with the given schema and name. +// Returns nil if the type cannot be found. It checks for domain type. +func (pgs *TypeCollection) GetDomainType(schName, typName string) (*types.DoltgresType, bool) { pgs.mutex.RLock() defer pgs.mutex.RUnlock() @@ -57,14 +57,14 @@ func (pgs *TypeCollection) GetDomainType(schName, typName string) (*types.Type, // GetAllTypes returns a map containing all types in the collection, grouped by the schema they're contained in. // Each type array is also sorted by the type name. -func (pgs *TypeCollection) GetAllTypes() (typesMap map[string][]*types.Type, schemaNames []string, totalCount int) { +func (pgs *TypeCollection) GetAllTypes() (typesMap map[string][]*types.DoltgresType, schemaNames []string, totalCount int) { pgs.mutex.RLock() defer pgs.mutex.RUnlock() - typesMap = make(map[string][]*types.Type) + typesMap = make(map[string][]*types.DoltgresType) for schemaName, nameMap := range pgs.schemaMap { schemaNames = append(schemaNames, schemaName) - typs := make([]*types.Type, 0, len(nameMap)) + typs := make([]*types.DoltgresType, 0, len(nameMap)) for _, typ := range nameMap { typs = append(typs, typ) } @@ -74,20 +74,22 @@ func (pgs *TypeCollection) GetAllTypes() (typesMap map[string][]*types.Type, sch }) typesMap[schemaName] = typs } + + // TODO: add built-in types sort.Slice(schemaNames, func(i, j int) bool { return schemaNames[i] < schemaNames[j] }) return } -// CreateType creates a new Type. -func (pgs *TypeCollection) CreateType(schema string, typ *types.Type) error { +// CreateType creates a new type. +func (pgs *TypeCollection) CreateType(schema string, typ *types.DoltgresType) error { pgs.mutex.Lock() defer pgs.mutex.Unlock() nameMap, ok := pgs.schemaMap[schema] if !ok { - nameMap = make(map[string]*types.Type) + nameMap = make(map[string]*types.DoltgresType) pgs.schemaMap[schema] = nameMap } if _, ok = nameMap[typ.Name]; ok { @@ -97,7 +99,7 @@ func (pgs *TypeCollection) CreateType(schema string, typ *types.Type) error { return nil } -// DropType drops an existing Type. +// DropType drops an existing type. func (pgs *TypeCollection) DropType(schName, typName string) error { pgs.mutex.Lock() defer pgs.mutex.Unlock() @@ -111,8 +113,8 @@ func (pgs *TypeCollection) DropType(schName, typName string) error { return types.ErrTypeDoesNotExist.New(typName) } -// IterateTypes iterates over all Types in the collection. -func (pgs *TypeCollection) IterateTypes(f func(schema string, typ *types.Type) error) error { +// IterateTypes iterates over all types in the collection. +func (pgs *TypeCollection) IterateTypes(f func(schema string, typ *types.DoltgresType) error) error { pgs.mutex.Lock() defer pgs.mutex.Unlock() @@ -132,17 +134,16 @@ func (pgs *TypeCollection) Clone() *TypeCollection { defer pgs.mutex.Unlock() newCollection := &TypeCollection{ - schemaMap: make(map[string]map[string]*types.Type), + schemaMap: make(map[string]map[string]*types.DoltgresType), mutex: &sync.RWMutex{}, } for schema, nameMap := range pgs.schemaMap { if len(nameMap) == 0 { continue } - clonedNameMap := make(map[string]*types.Type) + clonedNameMap := make(map[string]*types.DoltgresType) for key, typ := range nameMap { - newType := *typ - clonedNameMap[key] = &newType + clonedNameMap[key] = typ } newCollection.schemaMap[schema] = clonedNameMap } diff --git a/server/analyzer/add_implicit_prefix_lengths.go b/server/analyzer/add_implicit_prefix_lengths.go index eed284bf5b..f3f65e558c 100644 --- a/server/analyzer/add_implicit_prefix_lengths.go +++ b/server/analyzer/add_implicit_prefix_lengths.go @@ -22,6 +22,7 @@ import ( "github.com/dolthub/go-mysql-server/sql/analyzer" "github.com/dolthub/go-mysql-server/sql/plan" "github.com/dolthub/go-mysql-server/sql/transform" + "github.com/lib/pq/oid" pgtypes "github.com/dolthub/doltgresql/server/types" ) @@ -72,7 +73,7 @@ func AddImplicitPrefixLengths(_ *sql.Context, _ *analyzer.Analyzer, node sql.Nod if !ok { return nil, false, fmt.Errorf("indexed column %s not found in schema", index.Columns[i].Name) } - if _, ok := col.Type.(pgtypes.TextType); ok && index.Columns[i].Length == 0 { + if dt, ok := col.Type.(*pgtypes.DoltgresType); ok && dt.OID == uint32(oid.T_text) && index.Columns[i].Length == 0 { index.Columns[i].Length = defaultIndexPrefixLength indexModified = true } @@ -97,7 +98,7 @@ func AddImplicitPrefixLengths(_ *sql.Context, _ *analyzer.Analyzer, node sql.Nod if !ok { return nil, false, fmt.Errorf("indexed column %s not found in schema", newColumns[i].Name) } - if _, ok := col.Type.(pgtypes.TextType); ok && newColumns[i].Length == 0 { + if dt, ok := col.Type.(*pgtypes.DoltgresType); ok && dt.OID == uint32(oid.T_text) && newColumns[i].Length == 0 { newColumns[i].Length = defaultIndexPrefixLength indexModified = true } diff --git a/server/analyzer/assign_insert_casts.go b/server/analyzer/assign_insert_casts.go index 6cfcc36f85..4f6ee70dd4 100644 --- a/server/analyzer/assign_insert_casts.go +++ b/server/analyzer/assign_insert_casts.go @@ -38,16 +38,16 @@ func AssignInsertCasts(ctx *sql.Context, a *analyzer.Analyzer, node sql.Node, sc // First we'll make a map for each column, so that it's easier to match a name to a type. We also ensure that the // types use Doltgres types, as casts rely on them. At this point, we shouldn't have any GMS types floating around // anymore, so no need to include a lot of additional code to handle them. - destinationNameToType := make(map[string]pgtypes.DoltgresType) + destinationNameToType := make(map[string]*pgtypes.DoltgresType) for _, col := range insertInto.Destination.Schema() { - colType, ok := col.Type.(pgtypes.DoltgresType) + colType, ok := col.Type.(*pgtypes.DoltgresType) if !ok { return nil, transform.NewTree, fmt.Errorf("INSERT: non-Doltgres type found in destination: %s", col.Type.String()) } destinationNameToType[strings.ToLower(col.Name)] = colType } // Create the destination type slice that will match each inserted column - destinationTypes := make([]pgtypes.DoltgresType, len(insertInto.ColumnNames)) + destinationTypes := make([]*pgtypes.DoltgresType, len(insertInto.ColumnNames)) for i, colName := range insertInto.ColumnNames { destinationTypes[i], ok = destinationNameToType[strings.ToLower(colName)] if !ok { @@ -64,9 +64,9 @@ func AssignInsertCasts(ctx *sql.Context, a *analyzer.Analyzer, node sql.Node, sc // Null ColumnDefaultValues or empty DefaultValues are not properly typed in TypeSanitizer, so we must handle them here colExprType := colExpr.Type() if colExprType == nil || colExprType == types.Null { - colExprType = pgtypes.UnknownType{} + colExprType = pgtypes.Unknown } - fromColType, ok := colExprType.(pgtypes.DoltgresType) + fromColType, ok := colExprType.(*pgtypes.DoltgresType) if !ok { return nil, transform.NewTree, fmt.Errorf("INSERT: non-Doltgres type found in values source: %s", fromColType.String()) } @@ -84,7 +84,7 @@ func AssignInsertCasts(ctx *sql.Context, a *analyzer.Analyzer, node sql.Node, sc sourceSchema := insertInto.Source.Schema() projections := make([]sql.Expression, len(sourceSchema)) for i, col := range sourceSchema { - fromColType, ok := col.Type.(pgtypes.DoltgresType) + fromColType, ok := col.Type.(*pgtypes.DoltgresType) if !ok { return nil, transform.NewTree, fmt.Errorf("INSERT: non-Doltgres type found in source: %s", fromColType.String()) } diff --git a/server/analyzer/assign_update_casts.go b/server/analyzer/assign_update_casts.go index 920496d4ea..1065e6705a 100644 --- a/server/analyzer/assign_update_casts.go +++ b/server/analyzer/assign_update_casts.go @@ -88,11 +88,11 @@ func assignUpdateFieldCasts(updateExprs []sql.Expression) ([]sql.Expression, err if !ok { return nil, fmt.Errorf("UPDATE: assumption that expression is always SetField is incorrect: %T", updateExpr) } - fromType, ok := setField.RightChild.Type().(pgtypes.DoltgresType) + fromType, ok := setField.RightChild.Type().(*pgtypes.DoltgresType) if !ok { return nil, fmt.Errorf("UPDATE: non-Doltgres type found in source: %s", setField.RightChild.String()) } - toType, ok := setField.LeftChild.Type().(pgtypes.DoltgresType) + toType, ok := setField.LeftChild.Type().(*pgtypes.DoltgresType) if !ok { return nil, fmt.Errorf("UPDATE: non-Doltgres type found in destination: %s", setField.LeftChild.String()) } diff --git a/server/analyzer/domain.go b/server/analyzer/domain.go index 3080821c70..bd45812276 100644 --- a/server/analyzer/domain.go +++ b/server/analyzer/domain.go @@ -51,17 +51,17 @@ func resolveDomainTypeAndLoadCheckConstraints(ctx *sql.Context, a *analyzer.Anal checks := c.Checks() var same = transform.SameTree for _, col := range schema { - if domainType, ok := col.Type.(pgtypes.DomainType); ok { + if dt, ok := col.Type.(*pgtypes.DoltgresType); ok && dt.TypType == pgtypes.TypeType_Domain { // assign column nullable - col.Nullable = !domainType.NotNull + col.Nullable = !dt.NotNull // get domain default value and assign to the column default value - defVal, err := getDefault(ctx, a, domainType.DefaultExpr, col.Source, col.Type, col.Nullable) + defVal, err := getDefault(ctx, a, dt.Default, col.Source, col.Type, col.Nullable) if err != nil { return nil, transform.SameTree, err } col.Default = defVal // get domain checks - colChecks, err := getCheckConstraints(ctx, a, col.Name, col.Source, domainType.Checks) + colChecks, err := getCheckConstraints(ctx, a, col.Name, col.Source, dt.Checks) if err != nil { return nil, transform.SameTree, err } diff --git a/server/analyzer/init.go b/server/analyzer/init.go index d3b49594fd..06e0289f18 100644 --- a/server/analyzer/init.go +++ b/server/analyzer/init.go @@ -32,6 +32,7 @@ const ( ruleId_AddImplicitPrefixLengths ruleId_InsertContextRootFinalizer ruleId_ResolveType + ruleId_OptimizeFunctions ) // Init adds additional rules to the analyzer to handle Doltgres-specific functionality. @@ -61,8 +62,10 @@ func Init() { analyzer.Rule{Id: ruleId_ReplaceDropTable, Apply: ReplaceDropTable}, ) - // The auto-commit rule writes the contents of the context, so we need to insert our finalizer before that + // The auto-commit rule writes the contents of the context, so we need to insert our finalizer before that. + // We also should optimize functions last, since other rules may change the underlying expressions, potentially changing their return types. analyzer.OnceAfterAll = insertAnalyzerRules(analyzer.OnceAfterAll, analyzer.BacktickDefaulColumnValueNamesId, false, + analyzer.Rule{Id: ruleId_OptimizeFunctions, Apply: OptimizeFunctions}, analyzer.Rule{Id: ruleId_InsertContextRootFinalizer, Apply: InsertContextRootFinalizer}) } diff --git a/server/analyzer/optimize_functions.go b/server/analyzer/optimize_functions.go new file mode 100644 index 0000000000..6eab93d1a3 --- /dev/null +++ b/server/analyzer/optimize_functions.go @@ -0,0 +1,42 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package analyzer + +import ( + "github.com/dolthub/go-mysql-server/sql" + "github.com/dolthub/go-mysql-server/sql/analyzer" + "github.com/dolthub/go-mysql-server/sql/plan" + "github.com/dolthub/go-mysql-server/sql/transform" + + "github.com/dolthub/doltgresql/server/functions/framework" + + pgtransform "github.com/dolthub/doltgresql/server/transform" +) + +// OptimizeFunctions replaces all functions that fit specific criteria with their optimized variants. +func OptimizeFunctions(ctx *sql.Context, a *analyzer.Analyzer, node sql.Node, scope *plan.Scope, selector analyzer.RuleSelector, qFlags *sql.QueryFlags) (sql.Node, transform.TreeIdentity, error) { + // This is supposed to be one of the last rules to run. Subqueries break that assumption, so we skip this rule in such cases. + if scope != nil && scope.CurrentNodeIsFromSubqueryExpression { + return node, transform.SameTree, nil + } + return pgtransform.NodeExprsWithNodeWithOpaque(node, func(node sql.Node, expr sql.Expression) (sql.Expression, transform.TreeIdentity, error) { + if compiledFunction, ok := expr.(*framework.CompiledFunction); ok { + if quickFunction := compiledFunction.GetQuickFunction(); quickFunction != nil { + return quickFunction, transform.NewTree, nil + } + } + return expr, transform.SameTree, nil + }) +} diff --git a/server/analyzer/resolve_type.go b/server/analyzer/resolve_type.go index a88785836d..26d9073d3f 100644 --- a/server/analyzer/resolve_type.go +++ b/server/analyzer/resolve_type.go @@ -15,89 +15,74 @@ package analyzer import ( - "fmt" - "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/analyzer" "github.com/dolthub/go-mysql-server/sql/plan" "github.com/dolthub/go-mysql-server/sql/transform" "github.com/dolthub/doltgresql/core" - "github.com/dolthub/doltgresql/postgres/parser/sem/tree" "github.com/dolthub/doltgresql/server/types" ) // ResolveType replaces types.ResolvableType to appropriate types.DoltgresType. func ResolveType(ctx *sql.Context, a *analyzer.Analyzer, node sql.Node, scope *plan.Scope, selector analyzer.RuleSelector, qFlags *sql.QueryFlags) (sql.Node, transform.TreeIdentity, error) { return transform.Node(node, func(node sql.Node) (sql.Node, transform.TreeIdentity, error) { + var same = transform.SameTree switch n := node.(type) { - case sql.SchemaTarget: - switch n.(type) { - case *plan.AlterPK, *plan.AddColumn, *plan.ModifyColumn, *plan.CreateTable, *plan.DropColumn: - // DDL nodes must resolve any new column type, continue to logic below - // TODO: add nodes that use unresolved types like domain (e.g.: casting in SELECT) - default: - // other node types are not altering the schema and therefore don't need resolution of column type - return node, transform.SameTree, nil - } - - var same = transform.SameTree + case *plan.CreateTable: for _, col := range n.TargetSchema() { - if rt, ok := col.Type.(types.ResolvableType); ok { - dt, err := resolveResolvableType(ctx, rt.Typ) + if rt, ok := col.Type.(*types.DoltgresType); ok && !rt.IsResolvedType() { + dt, err := resolveType(ctx, rt) if err != nil { - return nil, transform.SameTree, err + return nil, transform.NewTree, err } same = transform.NewTree col.Type = dt } } return node, same, nil + case *plan.AddColumn: + col := n.Column() + if rt, ok := col.Type.(*types.DoltgresType); ok && !rt.IsResolvedType() { + dt, err := resolveType(ctx, rt) + if err != nil { + return nil, transform.NewTree, err + } + same = transform.NewTree + col.Type = dt + } + return node, same, nil + case *plan.ModifyColumn: + col := n.NewColumn() + if rt, ok := col.Type.(*types.DoltgresType); ok && !rt.IsResolvedType() { + dt, err := resolveType(ctx, rt) + if err != nil { + return nil, transform.NewTree, err + } + same = transform.NewTree + col.Type = dt + } + return node, same, nil default: + // TODO: add nodes that use unresolved types like domain return node, transform.SameTree, nil } }) } -// resolveResolvableType resolves any type that is unresolved yet. -func resolveResolvableType(ctx *sql.Context, typ tree.ResolvableTypeReference) (types.DoltgresType, error) { - switch t := typ.(type) { - case *tree.UnresolvedObjectName: - domain := t.ToTableName() - return resolveDomainType(ctx, string(domain.SchemaName), string(domain.ObjectName)) - default: - // TODO: add other types that need resolution at analyzer stage. - return nil, fmt.Errorf("the given type %T is not yet supported", typ) - } -} - -// resolveDomainType resolves DomainType from given schema and domain name. -func resolveDomainType(ctx *sql.Context, schema, domainName string) (types.DoltgresType, error) { - schema, err := core.GetSchemaName(ctx, nil, schema) +// resolveType resolves any type that is unresolved yet. (e.g.: domain types) +func resolveType(ctx *sql.Context, typ *types.DoltgresType) (*types.DoltgresType, error) { + schema, err := core.GetSchemaName(ctx, nil, typ.Schema) if err != nil { return nil, err } - domains, err := core.GetTypesCollectionFromContext(ctx) + typs, err := core.GetTypesCollectionFromContext(ctx) if err != nil { return nil, err } - domain, exists := domains.GetDomainType(schema, domainName) + resolvedTyp, exists := typs.GetType(schema, typ.Name) if !exists { - return nil, types.ErrTypeDoesNotExist.New(domainName) + return nil, types.ErrTypeDoesNotExist.New(typ.Name) } - - // TODO: need to resolve OID for non build-in type - asType, ok := types.OidToBuildInDoltgresType[domain.BaseTypeOID] - if !ok { - return nil, fmt.Errorf(`cannot resolve base type for "%s" domain type`, domainName) - } - - return types.DomainType{ - Schema: schema, - Name: domainName, - AsType: asType, - DefaultExpr: domain.Default, - NotNull: domain.NotNull, - Checks: domain.Checks, - }, nil + return resolvedTyp, nil } diff --git a/server/analyzer/serial.go b/server/analyzer/serial.go index 2043fb2469..2f725aae75 100644 --- a/server/analyzer/serial.go +++ b/server/analyzer/serial.go @@ -41,24 +41,21 @@ func ReplaceSerial(ctx *sql.Context, a *analyzer.Analyzer, node sql.Node, scope var ctSequences []*pgnodes.CreateSequence for _, col := range createTable.PkSchema().Schema { - if doltgresType, ok := col.Type.(pgtypes.DoltgresType); ok { - isSerial := false - var maxValue int64 - switch doltgresType.BaseID() { - case pgtypes.DoltgresTypeBaseID_Int16Serial: - isSerial = true - col.Type = pgtypes.Int16 - maxValue = 32767 - case pgtypes.DoltgresTypeBaseID_Int32Serial: - isSerial = true - col.Type = pgtypes.Int32 - maxValue = 2147483647 - case pgtypes.DoltgresTypeBaseID_Int64Serial: - isSerial = true - col.Type = pgtypes.Int64 - maxValue = 9223372036854775807 - } - if isSerial { + if doltgresType, ok := col.Type.(*pgtypes.DoltgresType); ok { + if doltgresType.IsSerial { + var maxValue int64 + switch doltgresType.Name { + case "smallserial": + col.Type = pgtypes.Int16 + maxValue = 32767 + case "serial": + col.Type = pgtypes.Int32 + maxValue = 2147483647 + case "bigserial": + col.Type = pgtypes.Int64 + maxValue = 9223372036854775807 + } + baseSequenceName := fmt.Sprintf("%s_%s_seq", createTable.Name(), col.Name) sequenceName := baseSequenceName schemaName, err := core.GetSchemaName(ctx, createTable.Db, "") @@ -104,7 +101,7 @@ func ReplaceSerial(ctx *sql.Context, a *analyzer.Analyzer, node sql.Node, scope } ctSequences = append(ctSequences, pgnodes.NewCreateSequence(false, "", &sequences.Sequence{ Name: sequenceName, - DataTypeOID: col.Type.(pgtypes.DoltgresType).OID(), + DataTypeOID: col.Type.(*pgtypes.DoltgresType).OID, Persistence: sequences.Persistence_Permanent, Start: 1, Current: 1, diff --git a/server/analyzer/type_sanitizer.go b/server/analyzer/type_sanitizer.go index e88517109f..eb7fd521ae 100644 --- a/server/analyzer/type_sanitizer.go +++ b/server/analyzer/type_sanitizer.go @@ -48,13 +48,13 @@ func TypeSanitizer(ctx *sql.Context, a *analyzer.Analyzer, node sql.Node, scope return pgexprs.NewGMSCast(expr), transform.NewTree, nil case sql.FunctionExpression: // Compiled functions are Doltgres functions. We're only concerned with GMS functions. - if _, ok := expr.(*framework.CompiledFunction); !ok { + if _, ok := expr.(framework.Function); !ok { // Some aggregation functions cannot be wrapped due to expectations in the analyzer, so we exclude them here. switch expr.FunctionName() { case "Count", "CountDistinct", "GroupConcat", "JSONObjectAgg", "Sum": default: // Some GMS functions wrap Doltgres parameters, so we'll only handle those that return GMS types - if _, ok := expr.Type().(pgtypes.DoltgresType); !ok { + if _, ok := expr.Type().(*pgtypes.DoltgresType); !ok { return pgexprs.NewGMSCast(expr), transform.NewTree, nil } } @@ -63,11 +63,11 @@ func TypeSanitizer(ctx *sql.Context, a *analyzer.Analyzer, node sql.Node, scope // Due to how interfaces work, we sometimes pass (*ColumnDefaultValue)(nil), so we have to check for it if expr != nil && expr.Expr != nil { defaultExpr := expr.Expr - if _, ok := defaultExpr.Type().(pgtypes.DoltgresType); !ok { + if _, ok := defaultExpr.Type().(*pgtypes.DoltgresType); !ok { defaultExpr = pgexprs.NewGMSCast(defaultExpr) } - defaultExprType := defaultExpr.Type().(pgtypes.DoltgresType) - outType, ok := expr.OutType.(pgtypes.DoltgresType) + defaultExprType := defaultExpr.Type().(*pgtypes.DoltgresType) + outType, ok := expr.OutType.(*pgtypes.DoltgresType) if !ok { return nil, transform.NewTree, fmt.Errorf("default values must have a non-GMS OutType: `%s`", expr.OutType.String()) } @@ -85,7 +85,7 @@ func TypeSanitizer(ctx *sql.Context, a *analyzer.Analyzer, node sql.Node, scope // typeSanitizerLiterals handles literal expressions for TypeSanitizer. func typeSanitizerLiterals(gmsLiteral *expression.Literal) (sql.Expression, transform.TreeIdentity, error) { // GMS may resolve Doltgres literals and then stick them in GMS literals, so we have to account for that here - if doltgresType, ok := gmsLiteral.Type().(pgtypes.DoltgresType); ok { + if doltgresType, ok := gmsLiteral.Type().(*pgtypes.DoltgresType); ok { return pgexprs.NewUnsafeLiteral(gmsLiteral.Value(), doltgresType), transform.NewTree, nil } switch gmsLiteral.Type().Type() { diff --git a/server/ast/column_table_def.go b/server/ast/column_table_def.go index ac7d3368df..d07284c872 100644 --- a/server/ast/column_table_def.go +++ b/server/ast/column_table_def.go @@ -18,6 +18,7 @@ import ( "fmt" vitess "github.com/dolthub/vitess/go/vt/sqlparser" + "github.com/lib/pq/oid" "github.com/dolthub/doltgresql/postgres/parser/sem/tree" pgtypes "github.com/dolthub/doltgresql/server/types" @@ -98,15 +99,15 @@ func nodeColumnTableDef(ctx *Context, node *tree.ColumnTableDef) (*vitess.Column generatedStored = true } if node.IsSerial { - if resolvedType == nil { + if resolvedType.IsEmptyType() { return nil, fmt.Errorf("serial type was not resolvable") } - switch resolvedType.BaseID() { - case pgtypes.DoltgresTypeBaseID_Int16: + switch oid.Oid(resolvedType.OID) { + case oid.T_int2: resolvedType = pgtypes.Int16Serial - case pgtypes.DoltgresTypeBaseID_Int32: + case oid.T_int4: resolvedType = pgtypes.Int32Serial - case pgtypes.DoltgresTypeBaseID_Int64: + case oid.T_int8: resolvedType = pgtypes.Int64Serial default: return nil, fmt.Errorf(`type "%s" cannot be serial`, resolvedType.String()) diff --git a/server/ast/create_sequence.go b/server/ast/create_sequence.go index b5d7881cd1..0fe73b8ade 100644 --- a/server/ast/create_sequence.go +++ b/server/ast/create_sequence.go @@ -19,6 +19,7 @@ import ( "math" vitess "github.com/dolthub/vitess/go/vt/sqlparser" + "github.com/lib/pq/oid" "github.com/dolthub/doltgresql/core/sequences" "github.com/dolthub/doltgresql/postgres/parser/sem/tree" @@ -44,14 +45,14 @@ func nodeCreateSequence(ctx *Context, node *tree.CreateSequence) (vitess.Stateme if len(name.DbQualifier.String()) > 0 { return nil, fmt.Errorf("CREATE SEQUENCE is currently only supported for the current database") } - // Read all of the options and check whether they've been set (if not, we'll use the defaults) + // Read all options and check whether they've been set (if not, we'll use the defaults) minValueLimit := int64(math.MinInt64) maxValueLimit := int64(math.MaxInt64) increment := int64(1) var minValue int64 var maxValue int64 var start int64 - var dataType pgtypes.DoltgresType + var dataType *pgtypes.DoltgresType var ownerTableName string var ownerColumnName string minValueSet := false @@ -62,21 +63,21 @@ func nodeCreateSequence(ctx *Context, node *tree.CreateSequence) (vitess.Stateme for _, option := range node.Options { switch option.Name { case tree.SeqOptAs: - if dataType != nil { + if !dataType.IsEmptyType() { return nil, fmt.Errorf("conflicting or redundant options") } _, dataType, err = nodeResolvableTypeReference(ctx, option.AsType) if err != nil { return nil, err } - switch dataType.BaseID() { - case pgtypes.DoltgresTypeBaseID_Int16: + switch oid.Oid(dataType.OID) { + case oid.T_int2: minValueLimit = int64(math.MinInt16) maxValueLimit = int64(math.MaxInt16) - case pgtypes.DoltgresTypeBaseID_Int32: + case oid.T_int4: minValueLimit = int64(math.MinInt32) maxValueLimit = int64(math.MaxInt32) - case pgtypes.DoltgresTypeBaseID_Int64: + case oid.T_int8: minValueLimit = int64(math.MinInt64) maxValueLimit = int64(math.MaxInt64) default: @@ -140,7 +141,7 @@ func nodeCreateSequence(ctx *Context, node *tree.CreateSequence) (vitess.Stateme return nil, fmt.Errorf("unknown CREATE SEQUENCE option") } } - // Determine what all of the values should be based on what was set and what is inferred, as well as perform + // Determine what all values should be based on what was set and what is inferred, as well as perform // validation for options that make sense if minValueSet { if minValue < minValueLimit || minValue > maxValueLimit { @@ -172,14 +173,14 @@ func nodeCreateSequence(ctx *Context, node *tree.CreateSequence) (vitess.Stateme } else { start = maxValue } - if dataType == nil { + if dataType.IsEmptyType() { dataType = pgtypes.Int64 } - // Returns the stored procedure call with all of the options + // Returns the stored procedure call with all options return vitess.InjectedStatement{ Statement: pgnodes.NewCreateSequence(node.IfNotExists, name.SchemaQualifier.String(), &sequences.Sequence{ Name: name.Name.String(), - DataTypeOID: dataType.OID(), + DataTypeOID: dataType.OID, Persistence: sequences.Persistence_Permanent, Start: start, Current: start, diff --git a/server/ast/expr.go b/server/ast/expr.go index c94da8f335..cbfb4470c1 100644 --- a/server/ast/expr.go +++ b/server/ast/expr.go @@ -105,14 +105,14 @@ func nodeExpr(ctx *Context, node tree.Expr) (vitess.Expr, error) { return nil, fmt.Errorf("ANNOTATE_TYPE is not yet supported") case *tree.Array: unresolvedChildren := make([]vitess.Expr, len(node.Exprs)) - var coercedType pgtypes.DoltgresType + var coercedType *pgtypes.DoltgresType if node.HasResolvedType() { _, resolvedType, err := nodeResolvableTypeReference(ctx, node.ResolvedType()) if err != nil { return nil, err } - if arrayType, ok := resolvedType.(pgtypes.DoltgresArrayType); ok { - coercedType = arrayType + if resolvedType.IsArrayType() { + coercedType = resolvedType } else { return nil, fmt.Errorf("array has invalid resolved type") } @@ -250,7 +250,7 @@ func nodeExpr(ctx *Context, node tree.Expr) (vitess.Expr, error) { } // If we have the resolved type, then we've got a Doltgres type instead of a GMS type - if resolvedType != nil { + if !resolvedType.IsEmptyType() { cast, err := pgexprs.NewExplicitCastInjectable(resolvedType) if err != nil { return nil, err diff --git a/server/ast/resolvable_type_reference.go b/server/ast/resolvable_type_reference.go index 4964e5dda2..ba8d68c0ae 100755 --- a/server/ast/resolvable_type_reference.go +++ b/server/ast/resolvable_type_reference.go @@ -26,24 +26,26 @@ import ( ) // nodeResolvableTypeReference handles tree.ResolvableTypeReference nodes. -func nodeResolvableTypeReference(ctx *Context, typ tree.ResolvableTypeReference) (*vitess.ConvertType, pgtypes.DoltgresType, error) { +func nodeResolvableTypeReference(ctx *Context, typ tree.ResolvableTypeReference) (*vitess.ConvertType, *pgtypes.DoltgresType, error) { if typ == nil { + // TODO: use UNKNOWN? return nil, nil, nil } var columnTypeName string var columnTypeLength *vitess.SQLVal var columnTypeScale *vitess.SQLVal - var resolvedType pgtypes.DoltgresType + var resolvedType *pgtypes.DoltgresType + var err error switch columnType := typ.(type) { case *tree.ArrayTypeReference: return nil, nil, fmt.Errorf("the given array type is not yet supported") case *tree.OIDTypeReference: return nil, nil, fmt.Errorf("referencing types by their OID is not yet supported") case *tree.UnresolvedObjectName: - resolvedType = pgtypes.ResolvableType{ - Typ: typ, - } + tn := columnType.ToTableName() + columnTypeName = string(tn.ObjectName) + resolvedType = pgtypes.NewUnresolvedDoltgresType(string(tn.SchemaName), string(tn.ObjectName)) case *types.GeoMetadata: return nil, nil, fmt.Errorf("geometry types are not yet supported") case *types.T: @@ -53,7 +55,14 @@ func nodeResolvableTypeReference(ctx *Context, typ tree.ResolvableTypeReference) if err != nil { return nil, nil, err } - resolvedType = baseResolvedType.ToArrayType() + if baseResolvedType.IsResolvedType() { + // currently the built-in types will be resolved, so it can retrieve its array type + resolvedType = baseResolvedType.ToArrayType() + } else { + // TODO: handle array type of non-built-in types + baseResolvedType.TypCategory = pgtypes.TypeCategory_ArrayTypes + resolvedType = baseResolvedType + } } else if columnType.Family() == types.GeometryFamily { return nil, nil, fmt.Errorf("geometry types are not yet supported") } else if columnType.Family() == types.GeographyFamily { @@ -68,11 +77,14 @@ func nodeResolvableTypeReference(ctx *Context, typ tree.ResolvableTypeReference) width := uint32(columnType.Width()) if width > pgtypes.StringMaxLength { return nil, nil, fmt.Errorf("length for type bpchar cannot exceed %d", pgtypes.StringMaxLength) - } - if width == 0 { + } else if width == 0 { + // TODO: need to differentiate between definitions 'bpchar' (valid) and 'char(0)' (invalid) resolvedType = pgtypes.BpChar } else { - resolvedType = pgtypes.CharType{Length: width} + resolvedType, err = pgtypes.NewCharType(int32(width)) + if err != nil { + return nil, nil, err + } } case oid.T_char: width := uint32(columnType.Width()) @@ -107,9 +119,9 @@ func nodeResolvableTypeReference(ctx *Context, typ tree.ResolvableTypeReference) if columnType.Precision() == 0 && columnType.Scale() == 0 { resolvedType = pgtypes.Numeric } else { - resolvedType = pgtypes.NumericType{ - Precision: columnType.Precision(), - Scale: columnType.Scale(), + resolvedType, err = pgtypes.NewNumericTypeWithPrecisionAndScale(columnType.Precision(), columnType.Scale()) + if err != nil { + return nil, nil, err } } case oid.T_oid: @@ -136,8 +148,15 @@ func nodeResolvableTypeReference(ctx *Context, typ tree.ResolvableTypeReference) width := uint32(columnType.Width()) if width > pgtypes.StringMaxLength { return nil, nil, fmt.Errorf("length for type varchar cannot exceed %d", pgtypes.StringMaxLength) + } else if width == 0 { + // TODO: need to differentiate between definitions 'varchar' (valid) and 'varchar(0)' (invalid) + resolvedType = pgtypes.VarChar + } else { + resolvedType, err = pgtypes.NewVarCharType(int32(width)) + } + if err != nil { + return nil, nil, err } - resolvedType = pgtypes.VarCharType{MaxChars: width} case oid.T_xid: resolvedType = pgtypes.Xid default: diff --git a/server/auth/database.go b/server/auth/database.go index 858edbfa03..0c3258abb5 100644 --- a/server/auth/database.go +++ b/server/auth/database.go @@ -19,6 +19,8 @@ import ( "sync" "sync/atomic" + "github.com/dolthub/doltgresql/server/types" + "github.com/dolthub/dolt/go/libraries/doltcore/env" "github.com/dolthub/dolt/go/libraries/utils/filesys" ) @@ -178,4 +180,18 @@ func dbInitDefault() { panic(err) } SetRole(postgres) + typesInitDefault() +} + +// typesInitDefault adds owner to built-in types. +func typesInitDefault() { + postgresRole := GetRole("postgres") + allTypes := types.GetAllTypes() + for _, typ := range allTypes { + AddOwner(OwnershipKey{ + PrivilegeObject: PrivilegeObject_TYPE, + Schema: "pg_catalog", + Name: typ.Name, + }, postgresRole.ID()) + } } diff --git a/server/cast/bool.go b/server/cast/bool.go index 0880b2e3fb..9281750efd 100644 --- a/server/cast/bool.go +++ b/server/cast/bool.go @@ -32,7 +32,7 @@ func boolExplicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Bool, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val.(bool) { return int32(1), nil } else { @@ -47,7 +47,7 @@ func boolAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Bool, ToType: pgtypes.BpChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { str := "false" if val.(bool) { str = "true" @@ -58,7 +58,7 @@ func boolAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Bool, ToType: pgtypes.Name, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { str := "f" if val.(bool) { str = "t" @@ -69,7 +69,7 @@ func boolAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Bool, ToType: pgtypes.Text, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val.(bool) { return "true", nil } else { @@ -80,7 +80,7 @@ func boolAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Bool, ToType: pgtypes.VarChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { str := "false" if val.(bool) { str = "true" diff --git a/server/cast/char.go b/server/cast/char.go index 09041215b7..f13c40cdac 100644 --- a/server/cast/char.go +++ b/server/cast/char.go @@ -37,7 +37,7 @@ func charAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.BpChar, ToType: pgtypes.InternalChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return targetType.IoInput(ctx, val.(string)) }, }) @@ -48,7 +48,7 @@ func charExplicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.BpChar, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { out, err := strconv.ParseInt(strings.TrimSpace(val.(string)), 10, 32) if err != nil { return nil, fmt.Errorf("invalid input syntax for type %s: %q", targetType.String(), val.(string)) @@ -66,28 +66,28 @@ func charImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.BpChar, ToType: pgtypes.BpChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return targetType.IoInput(ctx, val.(string)) }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.BpChar, ToType: pgtypes.Name, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.BpChar, ToType: pgtypes.Text, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.BpChar, ToType: pgtypes.VarChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) diff --git a/server/cast/date.go b/server/cast/date.go index 757de3ad58..4c7d85fc31 100644 --- a/server/cast/date.go +++ b/server/cast/date.go @@ -33,14 +33,14 @@ func dateImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Date, ToType: pgtypes.Timestamp, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val.(time.Time), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Date, ToType: pgtypes.TimestampTZ, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val.(time.Time), nil }, }) diff --git a/server/cast/float32.go b/server/cast/float32.go index d20c088175..b30f15a1d4 100644 --- a/server/cast/float32.go +++ b/server/cast/float32.go @@ -36,7 +36,7 @@ func float32Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Float32, ToType: pgtypes.Int16, - Function: func(ctx *sql.Context, valInterface any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, valInterface any, targetType *pgtypes.DoltgresType) (any, error) { val := float32(math.RoundToEven(float64(valInterface.(float32)))) if val > 32767 || val < -32768 { return nil, fmt.Errorf("smallint out of range") @@ -47,7 +47,7 @@ func float32Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Float32, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, valInterface any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, valInterface any, targetType *pgtypes.DoltgresType) (any, error) { val := float32(math.RoundToEven(float64(valInterface.(float32)))) if val > 2147483647 || val < -2147483648 { return nil, fmt.Errorf("integer out of range") @@ -58,7 +58,7 @@ func float32Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Float32, ToType: pgtypes.Int64, - Function: func(ctx *sql.Context, valInterface any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, valInterface any, targetType *pgtypes.DoltgresType) (any, error) { val := float32(math.RoundToEven(float64(valInterface.(float32)))) if val > 9223372036854775807 || val < -9223372036854775808 { return nil, fmt.Errorf("bigint out of range") @@ -69,8 +69,8 @@ func float32Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Float32, ToType: pgtypes.Numeric, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { - return decimal.NewFromFloat(float64(val.(float32))), nil + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { + return pgtypes.GetNumericValueWithTypmod(decimal.NewFromFloat(float64(val.(float32))), targetType.GetAttTypMod()) }, }) } @@ -80,7 +80,7 @@ func float32Implicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Float32, ToType: pgtypes.Float64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return float64(val.(float32)), nil }, }) diff --git a/server/cast/float64.go b/server/cast/float64.go index e71deffab8..642614f5f2 100644 --- a/server/cast/float64.go +++ b/server/cast/float64.go @@ -35,14 +35,14 @@ func float64Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Float64, ToType: pgtypes.Float32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return float32(val.(float64)), nil }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Float64, ToType: pgtypes.Int16, - Function: func(ctx *sql.Context, valInterface any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, valInterface any, targetType *pgtypes.DoltgresType) (any, error) { val := math.RoundToEven(valInterface.(float64)) if val > 32767 || val < -32768 { return nil, fmt.Errorf("smallint out of range") @@ -53,7 +53,7 @@ func float64Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Float64, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, valInterface any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, valInterface any, targetType *pgtypes.DoltgresType) (any, error) { val := math.RoundToEven(valInterface.(float64)) if val > 2147483647 || val < -2147483648 { return nil, fmt.Errorf("integer out of range") @@ -64,7 +64,7 @@ func float64Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Float64, ToType: pgtypes.Int64, - Function: func(ctx *sql.Context, valInterface any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, valInterface any, targetType *pgtypes.DoltgresType) (any, error) { val := math.RoundToEven(valInterface.(float64)) if val > 9223372036854775807 || val < -9223372036854775808 { return nil, fmt.Errorf("bigint out of range") @@ -75,8 +75,8 @@ func float64Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Float64, ToType: pgtypes.Numeric, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { - return decimal.NewFromFloat(val.(float64)), nil + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { + return pgtypes.GetNumericValueWithTypmod(decimal.NewFromFloat(val.(float64)), targetType.GetAttTypMod()) }, }) } diff --git a/server/cast/int16.go b/server/cast/int16.go index 6be2ee6be0..932ed50328 100644 --- a/server/cast/int16.go +++ b/server/cast/int16.go @@ -32,63 +32,63 @@ func int16Implicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int16, ToType: pgtypes.Float32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return float32(val.(int16)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int16, ToType: pgtypes.Float64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return float64(val.(int16)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int16, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int32(val.(int16)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int16, ToType: pgtypes.Int64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int64(val.(int16)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int16, ToType: pgtypes.Numeric, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return decimal.NewFromInt(int64(val.(int16))), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int16, ToType: pgtypes.Oid, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return uint32(val.(int16)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int16, ToType: pgtypes.Regclass, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return uint32(val.(int16)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int16, ToType: pgtypes.Regproc, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return uint32(val.(int16)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int16, ToType: pgtypes.Regtype, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return uint32(val.(int16)), nil }, }) diff --git a/server/cast/int32.go b/server/cast/int32.go index 50954b4258..222f2c7d50 100644 --- a/server/cast/int32.go +++ b/server/cast/int32.go @@ -36,7 +36,7 @@ func int32Explicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int32, ToType: pgtypes.Bool, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val.(int32) != 0, nil }, }) @@ -47,7 +47,7 @@ func int32Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Int32, ToType: pgtypes.Int16, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val.(int32) > 32767 || val.(int32) < -32768 { return nil, fmt.Errorf("smallint out of range") } @@ -61,56 +61,56 @@ func int32Implicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int32, ToType: pgtypes.Float32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return float32(val.(int32)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int32, ToType: pgtypes.Float64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return float64(val.(int32)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int32, ToType: pgtypes.Int64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int64(val.(int32)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int32, ToType: pgtypes.Numeric, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return decimal.NewFromInt(int64(val.(int32))), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int32, ToType: pgtypes.Oid, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return uint32(val.(int32)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int32, ToType: pgtypes.Regclass, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return uint32(val.(int32)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int32, ToType: pgtypes.Regproc, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return uint32(val.(int32)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int32, ToType: pgtypes.Regtype, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return uint32(val.(int32)), nil }, }) diff --git a/server/cast/int64.go b/server/cast/int64.go index 637fc2b6f1..7a1f9cde24 100644 --- a/server/cast/int64.go +++ b/server/cast/int64.go @@ -35,7 +35,7 @@ func int64Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Int64, ToType: pgtypes.Int16, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val.(int64) > 32767 || val.(int64) < -32768 { return nil, fmt.Errorf("smallint out of range") } @@ -45,7 +45,7 @@ func int64Assignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Int64, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val.(int64) > 2147483647 || val.(int64) < -2147483648 { return nil, fmt.Errorf("integer out of range") } @@ -59,28 +59,28 @@ func int64Implicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int64, ToType: pgtypes.Float32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return float32(val.(int64)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int64, ToType: pgtypes.Float64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return float64(val.(int64)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int64, ToType: pgtypes.Numeric, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return decimal.NewFromInt(val.(int64)), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int64, ToType: pgtypes.Oid, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val.(int64) > pgtypes.MaxUint32 || val.(int64) < 0 { return nil, errOutOfRange.New(targetType.String()) } @@ -90,7 +90,7 @@ func int64Implicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int64, ToType: pgtypes.Regclass, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val.(int64) > pgtypes.MaxUint32 || val.(int64) < 0 { return nil, errOutOfRange.New(targetType.String()) } @@ -100,7 +100,7 @@ func int64Implicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int64, ToType: pgtypes.Regproc, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val.(int64) > pgtypes.MaxUint32 || val.(int64) < 0 { return nil, errOutOfRange.New(targetType.String()) } @@ -110,7 +110,7 @@ func int64Implicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Int64, ToType: pgtypes.Regtype, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val.(int64) > pgtypes.MaxUint32 || val.(int64) < 0 { return nil, errOutOfRange.New(targetType.String()) } diff --git a/server/cast/internal_char.go b/server/cast/internal_char.go index b1d598808a..30d9ceb064 100644 --- a/server/cast/internal_char.go +++ b/server/cast/internal_char.go @@ -36,14 +36,14 @@ func internalCharAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.InternalChar, ToType: pgtypes.BpChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return targetType.IoInput(ctx, val.(string)) }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.InternalChar, ToType: pgtypes.VarChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) @@ -54,7 +54,7 @@ func internalCharExplicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.InternalChar, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { s := val.(string) if len(s) == 0 { return int32(0), nil @@ -76,7 +76,7 @@ func internalCharImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.InternalChar, ToType: pgtypes.Text, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil }, }) diff --git a/server/cast/interval.go b/server/cast/interval.go index ca8f07957d..a8890e0d8e 100644 --- a/server/cast/interval.go +++ b/server/cast/interval.go @@ -36,7 +36,7 @@ func intervalAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Interval, ToType: pgtypes.Time, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { dur := val.(duration.Duration) // the month and day of the duration are excluded return timeofday.FromInt(dur.Nanos() / functions.NanosPerMicro).ToTime(), nil @@ -49,7 +49,7 @@ func intervalImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Interval, ToType: pgtypes.Interval, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val.(duration.Duration), nil }, }) diff --git a/server/cast/json.go b/server/cast/json.go index d24985c2aa..78afbf028b 100644 --- a/server/cast/json.go +++ b/server/cast/json.go @@ -31,7 +31,7 @@ func jsonAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Json, ToType: pgtypes.JsonB, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return targetType.IoInput(ctx, val.(string)) }, }) diff --git a/server/cast/jsonb.go b/server/cast/jsonb.go index 80077cb3ac..b630429e2a 100644 --- a/server/cast/jsonb.go +++ b/server/cast/jsonb.go @@ -35,7 +35,7 @@ func jsonbExplicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.JsonB, ToType: pgtypes.Bool, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { switch value := val.(pgtypes.JsonDocument).Value.(type) { case pgtypes.JsonValueObject: return nil, fmt.Errorf("cannot cast jsonb object to type %s", targetType.String()) @@ -57,7 +57,7 @@ func jsonbExplicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.JsonB, ToType: pgtypes.Float32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { switch value := val.(pgtypes.JsonDocument).Value.(type) { case pgtypes.JsonValueObject: return nil, fmt.Errorf("cannot cast jsonb object to type %s", targetType.String()) @@ -80,7 +80,7 @@ func jsonbExplicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.JsonB, ToType: pgtypes.Float64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { switch value := val.(pgtypes.JsonDocument).Value.(type) { case pgtypes.JsonValueObject: return nil, fmt.Errorf("cannot cast jsonb object to type %s", targetType.String()) @@ -103,7 +103,7 @@ func jsonbExplicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.JsonB, ToType: pgtypes.Int16, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { switch value := val.(pgtypes.JsonDocument).Value.(type) { case pgtypes.JsonValueObject: return nil, fmt.Errorf("cannot cast jsonb object to type %s", targetType.String()) @@ -129,7 +129,7 @@ func jsonbExplicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.JsonB, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { switch value := val.(pgtypes.JsonDocument).Value.(type) { case pgtypes.JsonValueObject: return nil, fmt.Errorf("cannot cast jsonb object to type %s", targetType.String()) @@ -155,7 +155,7 @@ func jsonbExplicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.JsonB, ToType: pgtypes.Int64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { switch value := val.(pgtypes.JsonDocument).Value.(type) { case pgtypes.JsonValueObject: return nil, fmt.Errorf("cannot cast jsonb object to type %s", targetType.String()) @@ -181,7 +181,7 @@ func jsonbExplicit() { framework.MustAddExplicitTypeCast(framework.TypeCast{ FromType: pgtypes.JsonB, ToType: pgtypes.Numeric, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { switch value := val.(pgtypes.JsonDocument).Value.(type) { case pgtypes.JsonValueObject: return nil, fmt.Errorf("cannot cast jsonb object to type %s", targetType.String()) @@ -207,7 +207,7 @@ func jsonbAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.JsonB, ToType: pgtypes.Json, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return pgtypes.JsonB.IoOutput(ctx, val) }, }) diff --git a/server/cast/name.go b/server/cast/name.go index 39f5ab5e2e..05747f1407 100644 --- a/server/cast/name.go +++ b/server/cast/name.go @@ -32,14 +32,14 @@ func nameAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Name, ToType: pgtypes.BpChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Name, ToType: pgtypes.VarChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) @@ -50,7 +50,7 @@ func nameImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Name, ToType: pgtypes.Text, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil }, }) diff --git a/server/cast/numeric.go b/server/cast/numeric.go index ee8045b4dd..fea084ff0a 100644 --- a/server/cast/numeric.go +++ b/server/cast/numeric.go @@ -35,7 +35,7 @@ func numericAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Numeric, ToType: pgtypes.Int16, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { d := val.(decimal.Decimal) if d.LessThan(pgtypes.NumericValueMinInt16) || d.GreaterThan(pgtypes.NumericValueMaxInt16) { return nil, fmt.Errorf("smallint out of range") @@ -46,7 +46,7 @@ func numericAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Numeric, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { d := val.(decimal.Decimal) if d.LessThan(pgtypes.NumericValueMinInt32) || d.GreaterThan(pgtypes.NumericValueMaxInt32) { return nil, fmt.Errorf("integer out of range") @@ -57,7 +57,7 @@ func numericAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Numeric, ToType: pgtypes.Int64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { d := val.(decimal.Decimal) if d.LessThan(pgtypes.NumericValueMinInt64) || d.GreaterThan(pgtypes.NumericValueMaxInt64) { return nil, fmt.Errorf("bigint out of range") @@ -72,7 +72,7 @@ func numericImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Numeric, ToType: pgtypes.Float32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { f, _ := val.(decimal.Decimal).Float64() return float32(f), nil }, @@ -80,7 +80,7 @@ func numericImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Numeric, ToType: pgtypes.Float64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { f, _ := val.(decimal.Decimal).Float64() return f, nil }, @@ -88,9 +88,8 @@ func numericImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Numeric, ToType: pgtypes.Numeric, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { - // TODO: handle precision and scale - return val, nil + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { + return pgtypes.GetNumericValueWithTypmod(val.(decimal.Decimal), targetType.GetAttTypMod()) }, }) } diff --git a/server/cast/oid.go b/server/cast/oid.go index 949a5bb122..eb2b1966eb 100644 --- a/server/cast/oid.go +++ b/server/cast/oid.go @@ -32,14 +32,14 @@ func oidAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Oid, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int32(val.(uint32)), nil }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Oid, ToType: pgtypes.Int64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int64(val.(uint32)), nil }, }) @@ -50,21 +50,21 @@ func oidImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Oid, ToType: pgtypes.Regclass, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Oid, ToType: pgtypes.Regproc, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Oid, ToType: pgtypes.Regtype, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil }, }) diff --git a/server/cast/regclass.go b/server/cast/regclass.go index b1f100db05..f646f3842d 100644 --- a/server/cast/regclass.go +++ b/server/cast/regclass.go @@ -32,14 +32,14 @@ func regclassAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Regclass, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int32(val.(uint32)), nil }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Regclass, ToType: pgtypes.Int64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int64(val.(uint32)), nil }, }) @@ -50,7 +50,7 @@ func regclassImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Regclass, ToType: pgtypes.Oid, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil }, }) diff --git a/server/cast/regproc.go b/server/cast/regproc.go index 77bac452a1..332dea8c2f 100644 --- a/server/cast/regproc.go +++ b/server/cast/regproc.go @@ -32,14 +32,14 @@ func regprocAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Regproc, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int32(val.(uint32)), nil }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Regproc, ToType: pgtypes.Int64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int64(val.(uint32)), nil }, }) @@ -50,7 +50,7 @@ func regprocImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Regproc, ToType: pgtypes.Oid, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil }, }) diff --git a/server/cast/regtype.go b/server/cast/regtype.go index 80f5d31fe6..97b848dfe9 100644 --- a/server/cast/regtype.go +++ b/server/cast/regtype.go @@ -32,14 +32,14 @@ func regtypeAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Regtype, ToType: pgtypes.Int32, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int32(val.(uint32)), nil }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Regtype, ToType: pgtypes.Int64, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return int64(val.(uint32)), nil }, }) @@ -50,7 +50,7 @@ func regtypeImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Regtype, ToType: pgtypes.Oid, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil }, }) diff --git a/server/cast/text.go b/server/cast/text.go index 60214de110..2ff5ff3620 100644 --- a/server/cast/text.go +++ b/server/cast/text.go @@ -32,14 +32,14 @@ func textAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Text, ToType: pgtypes.BpChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Text, ToType: pgtypes.InternalChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) @@ -50,28 +50,28 @@ func textImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Text, ToType: pgtypes.BpChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Text, ToType: pgtypes.Name, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Text, ToType: pgtypes.Regclass, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return targetType.IoInput(ctx, val.(string)) }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Text, ToType: pgtypes.VarChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) diff --git a/server/cast/time.go b/server/cast/time.go index f0679c48ad..1c077def46 100644 --- a/server/cast/time.go +++ b/server/cast/time.go @@ -34,7 +34,7 @@ func timeImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Time, ToType: pgtypes.Interval, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { t := val.(time.Time) dur := functions.GetIntervalDurationFromTimeComponents(0, 0, 0, int64(t.Hour()), int64(t.Minute()), int64(t.Second()), 0) return dur, nil diff --git a/server/cast/timestamp.go b/server/cast/timestamp.go index 4127444c8d..838455e605 100644 --- a/server/cast/timestamp.go +++ b/server/cast/timestamp.go @@ -36,14 +36,14 @@ func timestampAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Timestamp, ToType: pgtypes.Date, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return pgdate.MakeDateFromTime(val.(time.Time)) }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.Timestamp, ToType: pgtypes.Time, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return timeofday.FromTime(val.(time.Time)), nil }, }) @@ -54,14 +54,14 @@ func timestampImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Timestamp, ToType: pgtypes.Timestamp, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val.(time.Time), nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.Timestamp, ToType: pgtypes.TimestampTZ, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { // TODO: check return val.(time.Time), nil }, diff --git a/server/cast/timestamptz.go b/server/cast/timestamptz.go index 3261a17d5b..bf97754996 100644 --- a/server/cast/timestamptz.go +++ b/server/cast/timestamptz.go @@ -36,21 +36,21 @@ func timestampTZAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.TimestampTZ, ToType: pgtypes.Date, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return pgdate.MakeDateFromTime(val.(time.Time)) }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.TimestampTZ, ToType: pgtypes.Time, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return timeofday.FromTime(val.(time.Time)), nil }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.TimestampTZ, ToType: pgtypes.Timestamp, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { // TODO: check return val.(time.Time), nil }, @@ -58,7 +58,7 @@ func timestampTZAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.TimestampTZ, ToType: pgtypes.TimeTZ, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val.(time.Time), nil }, }) @@ -69,7 +69,7 @@ func timestampTZImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.TimestampTZ, ToType: pgtypes.TimestampTZ, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val.(time.Time), nil }, }) diff --git a/server/cast/timetz.go b/server/cast/timetz.go index 03827f50ee..52a1b4cedd 100644 --- a/server/cast/timetz.go +++ b/server/cast/timetz.go @@ -34,7 +34,7 @@ func timeTZAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.TimeTZ, ToType: pgtypes.Time, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val.(time.Time), nil }, }) @@ -45,7 +45,7 @@ func timeTZImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.TimeTZ, ToType: pgtypes.TimeTZ, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val.(time.Time), nil }, }) diff --git a/server/cast/utils.go b/server/cast/utils.go index 89bf231cc4..ce0269613c 100644 --- a/server/cast/utils.go +++ b/server/cast/utils.go @@ -19,6 +19,7 @@ import ( "strings" "unicode/utf8" + "github.com/lib/pq/oid" "gopkg.in/src-d/go-errors.v1" pgtypes "github.com/dolthub/doltgresql/server/types" @@ -29,34 +30,40 @@ var errOutOfRange = errors.NewKind("%s out of range") // handleStringCast handles casts to the string types that may have length restrictions. Returns an error if other types // are passed in. Will always return the correct string, even on error, as some contexts may ignore the error. -func handleStringCast(str string, targetType pgtypes.DoltgresType) (string, error) { - switch targetType := targetType.(type) { - case pgtypes.CharType: - if targetType.IsUnbounded() { +func handleStringCast(str string, targetType *pgtypes.DoltgresType) (string, error) { + tm := targetType.GetAttTypMod() + switch oid.Oid(targetType.OID) { + case oid.T_bpchar: + if tm == -1 { return str, nil + } + maxChars, err := pgtypes.GetTypModFromCharLength("char", tm) + if err != nil { + return "", err + } + length := uint32(maxChars) + str, runeLength := truncateString(str, length) + if runeLength > length { + return str, fmt.Errorf("value too long for type %s", targetType.String()) + } else if runeLength < length { + return str + strings.Repeat(" ", int(length-runeLength)), nil } else { - str, runeLength := truncateString(str, targetType.Length) - if runeLength > targetType.Length { - return str, fmt.Errorf("value too long for type %s", targetType.String()) - } else if runeLength < targetType.Length { - return str + strings.Repeat(" ", int(targetType.Length-runeLength)), nil - } else { - return str, nil - } + return str, nil } - case pgtypes.InternalCharType: + case oid.T_char: str, _ := truncateString(str, pgtypes.InternalCharLength) return str, nil - case pgtypes.NameType: + case oid.T_name: // Name seems to never throw an error, regardless of the context or how long the input is - str, _ := truncateString(str, targetType.Length) + str, _ := truncateString(str, uint32(targetType.TypLength)) return str, nil - case pgtypes.VarCharType: - if targetType.IsUnbounded() { + case oid.T_varchar: + if tm == -1 { return str, nil } - str, runeLength := truncateString(str, targetType.MaxChars) - if runeLength > targetType.MaxChars { + length := uint32(pgtypes.GetCharLengthFromTypmod(tm)) + str, runeLength := truncateString(str, length) + if runeLength > length { return str, fmt.Errorf("value too long for type %s", targetType.String()) } else { return str, nil diff --git a/server/cast/varchar.go b/server/cast/varchar.go index 16b109c34e..552f44dff8 100644 --- a/server/cast/varchar.go +++ b/server/cast/varchar.go @@ -32,14 +32,14 @@ func varcharAssignment() { framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.VarChar, ToType: pgtypes.BpChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) framework.MustAddAssignmentTypeCast(framework.TypeCast{ FromType: pgtypes.VarChar, ToType: pgtypes.InternalChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) @@ -50,28 +50,28 @@ func varcharImplicit() { framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.VarChar, ToType: pgtypes.BpChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.VarChar, ToType: pgtypes.Name, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.VarChar, ToType: pgtypes.Text, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil }, }) framework.MustAddImplicitTypeCast(framework.TypeCast{ FromType: pgtypes.VarChar, ToType: pgtypes.VarChar, - Function: func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + Function: func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return handleStringCast(val.(string), targetType) }, }) diff --git a/server/connection_data.go b/server/connection_data.go index d99d2293b6..c962895c39 100644 --- a/server/connection_data.go +++ b/server/connection_data.go @@ -116,8 +116,8 @@ func extractBindVarTypes(queryPlan sql.Node) ([]uint32, error) { switch e := expr.(type) { case *expression.BindVar: var typOid uint32 - if doltgresType, ok := e.Type().(pgtypes.DoltgresType); ok { - typOid = doltgresType.OID() + if doltgresType, ok := e.Type().(*pgtypes.DoltgresType); ok { + typOid = doltgresType.OID } else { // TODO: should remove usage non doltgres type typOid, err = VitessTypeToObjectID(e.Type().Type()) @@ -130,8 +130,8 @@ func extractBindVarTypes(queryPlan sql.Node) ([]uint32, error) { case *pgexprs.ExplicitCast: if bindVar, ok := e.Child().(*expression.BindVar); ok { var typOid uint32 - if doltgresType, ok := bindVar.Type().(pgtypes.DoltgresType); ok { - typOid = doltgresType.OID() + if doltgresType, ok := bindVar.Type().(*pgtypes.DoltgresType); ok { + typOid = doltgresType.OID } else { typOid, err = VitessTypeToObjectID(e.Type().Type()) if err != nil { diff --git a/server/connection_handler.go b/server/connection_handler.go index a603c03411..c6d6dac668 100644 --- a/server/connection_handler.go +++ b/server/connection_handler.go @@ -41,9 +41,7 @@ import ( "github.com/dolthub/doltgresql/postgres/parser/parser" "github.com/dolthub/doltgresql/postgres/parser/sem/tree" "github.com/dolthub/doltgresql/server/ast" - pgexprs "github.com/dolthub/doltgresql/server/expression" "github.com/dolthub/doltgresql/server/node" - pgtypes "github.com/dolthub/doltgresql/server/types" ) // ConnectionHandler is responsible for the entire lifecycle of a user connection: receiving messages they send, @@ -54,8 +52,8 @@ type ConnectionHandler struct { portals map[string]PortalData doltgresHandler *DoltgresHandler backend *pgproto3.Backend - pgTypeMap *pgtype.Map - waitForSync bool + + waitForSync bool // copyFromStdinState is set when this connection is in the COPY FROM STDIN mode, meaning it is waiting on // COPY DATA messages from the client to import data into tables. copyFromStdinState *copyFromStdinState @@ -96,6 +94,7 @@ func NewConnectionHandler(conn net.Conn, handler mysql.Handler) *ConnectionHandl sm: server.SessionManager(), readTimeout: 0, // cfg.ConnReadTimeout, encodeLoggedQuery: false, // cfg.EncodeLoggedQuery, + pgTypeMap: pgtype.NewMap(), } return &ConnectionHandler{ @@ -104,7 +103,6 @@ func NewConnectionHandler(conn net.Conn, handler mysql.Handler) *ConnectionHandl portals: portals, doltgresHandler: doltgresHandler, backend: pgproto3.NewBackend(conn, conn), - pgTypeMap: pgtype.NewMap(), } } @@ -541,12 +539,16 @@ func (h *ConnectionHandler) handleBind(message *pgproto3.Bind) error { return h.send(&pgproto3.BindComplete{}) } - bindVars, err := h.convertBindParameters(preparedData.BindVarTypes, message.ParameterFormatCodes, message.Parameters) - if err != nil { - return err - } - - analyzedPlan, fields, err := h.doltgresHandler.ComBind(context.Background(), h.mysqlConn, preparedData.Query.String, preparedData.Query.AST, bindVars) + analyzedPlan, fields, err := h.doltgresHandler.ComBind( + context.Background(), + h.mysqlConn, + preparedData.Query.String, + preparedData.Query.AST, + BindVariables{ + varTypes: preparedData.BindVarTypes, + formatCodes: message.ParameterFormatCodes, + parameters: message.Parameters, + }) if err != nil { return err } @@ -796,30 +798,6 @@ func (h *ConnectionHandler) deallocatePreparedStatement(name string, preparedSta }) } -// convertBindParameters handles the conversion from bind parameters to variable values. -func (h *ConnectionHandler) convertBindParameters(types []uint32, formatCodes []int16, values [][]byte) (map[string]sqlparser.Expr, error) { - bindings := make(map[string]sqlparser.Expr, len(values)) - for i := range values { - typ := types[i] - var bindVarString string - // We'll rely on a library to decode each format, which will deal with text and binary representations for us - if err := h.pgTypeMap.Scan(typ, formatCodes[i], values[i], &bindVarString); err != nil { - return nil, err - } - - pgTyp, ok := pgtypes.OidToBuildInDoltgresType[typ] - if !ok { - return nil, fmt.Errorf("unhandled oid type: %v", typ) - } - v, err := pgTyp.IoInput(nil, bindVarString) - if err != nil { - return nil, err - } - bindings[fmt.Sprintf("v%d", i+1)] = sqlparser.InjectedExpr{Expression: pgexprs.NewUnsafeLiteral(v, pgTyp)} - } - return bindings, nil -} - // query runs the given query and sends a CommandComplete message to the client func (h *ConnectionHandler) query(query ConvertedQuery) error { // |rowsAffected| gets altered by the callback below diff --git a/server/doltgres_handler.go b/server/doltgres_handler.go index b16a7d2aca..4cff7f8573 100644 --- a/server/doltgres_handler.go +++ b/server/doltgres_handler.go @@ -34,8 +34,10 @@ import ( "github.com/dolthub/vitess/go/mysql" "github.com/dolthub/vitess/go/vt/sqlparser" "github.com/jackc/pgx/v5/pgproto3" + "github.com/jackc/pgx/v5/pgtype" "github.com/sirupsen/logrus" + pgexprs "github.com/dolthub/doltgresql/server/expression" pgtypes "github.com/dolthub/doltgresql/server/types" ) @@ -49,6 +51,14 @@ func init() { } } +// BindVariables represents arrays of types, format codes and parameters +// used to convert given parameters to binding variables map. +type BindVariables struct { + varTypes []uint32 + formatCodes []int16 + parameters [][]byte +} + // Result represents a query result. type Result struct { Fields []pgproto3.FieldDescription `json:"fields"` @@ -72,12 +82,13 @@ type DoltgresHandler struct { sm *server.SessionManager readTimeout time.Duration encodeLoggedQuery bool + pgTypeMap *pgtype.Map } var _ Handler = &DoltgresHandler{} // ComBind implements the Handler interface. -func (h *DoltgresHandler) ComBind(ctx context.Context, c *mysql.Conn, query string, parsedQuery mysql.ParsedQuery, bindVars map[string]sqlparser.Expr) (mysql.BoundQuery, []pgproto3.FieldDescription, error) { +func (h *DoltgresHandler) ComBind(ctx context.Context, c *mysql.Conn, query string, parsedQuery mysql.ParsedQuery, bindVars BindVariables) (mysql.BoundQuery, []pgproto3.FieldDescription, error) { sqlCtx, err := h.sm.NewContextWithQuery(ctx, c, query) if err != nil { return nil, nil, err @@ -88,7 +99,12 @@ func (h *DoltgresHandler) ComBind(ctx context.Context, c *mysql.Conn, query stri return nil, nil, fmt.Errorf("parsedQuery must be a sqlparser.Statement, but got %T", parsedQuery) } - queryPlan, err := h.e.BoundQueryPlan(sqlCtx, query, stmt, bindVars) + bvs, err := h.convertBindParameters(sqlCtx, bindVars.varTypes, bindVars.formatCodes, bindVars.parameters) + if err != nil { + return nil, nil, err + } + + queryPlan, err := h.e.BoundQueryPlan(sqlCtx, query, stmt, bvs) if err != nil { return nil, nil, err } @@ -134,7 +150,7 @@ func (h *DoltgresHandler) ComPrepareParsed(ctx context.Context, c *mysql.Conn, q fields = []pgproto3.FieldDescription{ { Name: []byte("Rows"), - DataTypeOID: pgtypes.Int32.OID(), + DataTypeOID: pgtypes.Int32.OID, DataTypeSize: int16(pgtypes.Int32.MaxTextResponseByteLength(nil)), }, } @@ -196,6 +212,30 @@ func (h *DoltgresHandler) NewContext(ctx context.Context, c *mysql.Conn, query s return h.sm.NewContext(ctx, c, query) } +// convertBindParameters handles the conversion from bind parameters to variable values. +func (h *DoltgresHandler) convertBindParameters(ctx *sql.Context, types []uint32, formatCodes []int16, values [][]byte) (map[string]sqlparser.Expr, error) { + bindings := make(map[string]sqlparser.Expr, len(values)) + for i := range values { + typ := types[i] + var bindVarString string + // We'll rely on a library to decode each format, which will deal with text and binary representations for us + if err := h.pgTypeMap.Scan(typ, formatCodes[i], values[i], &bindVarString); err != nil { + return nil, err + } + + pgTyp, ok := pgtypes.OidToBuiltInDoltgresType[typ] + if !ok { + return nil, fmt.Errorf("unhandled oid type: %v", typ) + } + v, err := pgTyp.IoInput(ctx, bindVarString) + if err != nil { + return nil, err + } + bindings[fmt.Sprintf("v%d", i+1)] = sqlparser.InjectedExpr{Expression: pgexprs.NewUnsafeLiteral(v, pgTyp)} + } + return bindings, nil +} + var queryLoggingRegex = regexp.MustCompile(`[\r\n\t ]+`) func (h *DoltgresHandler) doQuery(ctx context.Context, c *mysql.Conn, query string, parsed sqlparser.Statement, analyzedPlan sql.Node, queryExec QueryExecutor, callback func(*Result) error) error { @@ -321,9 +361,11 @@ func schemaToFieldDescriptions(ctx *sql.Context, s sql.Schema) []pgproto3.FieldD fields := make([]pgproto3.FieldDescription, len(s)) for i, c := range s { var oid uint32 + var typmod = int32(-1) var err error - if doltgresType, ok := c.Type.(pgtypes.DoltgresType); ok { - oid = doltgresType.OID() + if doltgresType, ok := c.Type.(*pgtypes.DoltgresType); ok { + oid = doltgresType.OID + typmod = doltgresType.GetAttTypMod() // pg_attribute.atttypmod } else { oid, err = VitessTypeToObjectID(c.Type.Type()) if err != nil { @@ -342,7 +384,7 @@ func schemaToFieldDescriptions(ctx *sql.Context, s sql.Schema) []pgproto3.FieldD TableAttributeNumber: uint16(0), DataTypeOID: oid, DataTypeSize: int16(c.Type.MaxTextResponseByteLength(ctx)), - TypeModifier: int32(-1), // TODO: used for domain type, which we don't support yet + TypeModifier: typmod, Format: int16(0), } } @@ -420,8 +462,10 @@ func (h *DoltgresHandler) resultForDefaultIter(ctx *sql.Context, schema sql.Sche var rowChan = make(chan sql.Row, 512) pan2err := func() { - if recoveredPanic := recover(); recoveredPanic != nil { - returnErr = fmt.Errorf("DoltgresHandler caught panic: %v", recoveredPanic) + if HandlePanics { + if recoveredPanic := recover(); recoveredPanic != nil { + returnErr = fmt.Errorf("DoltgresHandler caught panic: %v", recoveredPanic) + } } } diff --git a/server/expression/any.go b/server/expression/any.go index 3c0c192d43..a43efc109e 100644 --- a/server/expression/any.go +++ b/server/expression/any.go @@ -19,6 +19,7 @@ import ( "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/plan" + "github.com/lib/pq/oid" "github.com/dolthub/doltgresql/server/functions/framework" pgtypes "github.com/dolthub/doltgresql/server/types" @@ -40,7 +41,7 @@ type subqueryAnyExpr struct { rightSub *plan.Subquery staticLiteral *Literal arrayLiterals []*Literal - compFuncs []*framework.CompiledFunction + compFuncs []framework.Function } // expressionAnyExpr represents the resolved comparison function for a sql.Expression. @@ -48,7 +49,7 @@ type expressionAnyExpr struct { rightExpr sql.Expression staticLiteral *Literal arrayLiteral *Literal - compFunc *framework.CompiledFunction + compFunc framework.Function } // NewAnyExpr creates a new AnyExpr expression. @@ -145,7 +146,7 @@ func (a *subqueryAnyExpr) eval(ctx *sql.Context, subOperator string, row sql.Row for i, rightValue := range rightValues { a.arrayLiterals[i].value = rightValue } - // Now we can loop over all of the comparison functions, as they'll reference their respective values + // Now we can loop over all comparison functions, as they'll reference their respective values for _, compFunc := range a.compFuncs { result, err := compFunc.Eval(ctx, row) if err != nil { @@ -276,9 +277,9 @@ func (a *AnyExpr) DebugString() string { // anySubqueryWithChildren resolves the comparison functions for a plan.Subquery. func anySubqueryWithChildren(anyExpr *AnyExpr, sub *plan.Subquery) (sql.Expression, error) { schema := sub.Query.Schema() - subTypes := make([]pgtypes.DoltgresType, len(schema)) + subTypes := make([]*pgtypes.DoltgresType, len(schema)) for i, col := range schema { - dgType, ok := col.Type.(pgtypes.DoltgresType) + dgType, ok := col.Type.(*pgtypes.DoltgresType) if !ok { return nil, fmt.Errorf("expected right child to be a DoltgresType but got `%T`", sub) } @@ -290,19 +291,19 @@ func anySubqueryWithChildren(anyExpr *AnyExpr, sub *plan.Subquery) (sql.Expressi return nil, err } - if leftType, ok := anyExpr.leftExpr.Type().(pgtypes.DoltgresType); ok { + if leftType, ok := anyExpr.leftExpr.Type().(*pgtypes.DoltgresType); ok { // Resolve comparison functions once and reuse the functions in Eval. staticLiteral := &Literal{typ: leftType} arrayLiterals := make([]*Literal, len(subTypes)) // Each expression may be a different type (which is valid), so we need a comparison function for each expression. - compFuncs := make([]*framework.CompiledFunction, len(subTypes)) + compFuncs := make([]framework.Function, len(subTypes)) for i, rightType := range subTypes { arrayLiterals[i] = &Literal{typ: rightType} compFuncs[i] = framework.GetBinaryFunction(op).Compile("internal_any_comparison", staticLiteral, arrayLiterals[i]) if compFuncs[i] == nil { return nil, fmt.Errorf("operator does not exist: %s = %s", leftType.String(), rightType.String()) } - if compFuncs[i].Type().(pgtypes.DoltgresType).BaseID() != pgtypes.DoltgresTypeBaseID_Bool { + if compFuncs[i].Type().(*pgtypes.DoltgresType).OID != uint32(oid.T_bool) { // This should never happen, but this is just to be safe return nil, fmt.Errorf("%T: found equality comparison that does not return a bool", anyExpr) } @@ -321,18 +322,17 @@ func anySubqueryWithChildren(anyExpr *AnyExpr, sub *plan.Subquery) (sql.Expressi // anyExpressionWithChildren resolves the comparison functions for a sql.Expression. func anyExpressionWithChildren(anyExpr *AnyExpr) (sql.Expression, error) { - arrType, ok := anyExpr.rightExpr.Type().(pgtypes.DoltgresArrayType) + arrType, ok := anyExpr.rightExpr.Type().(*pgtypes.DoltgresType) if !ok { return nil, fmt.Errorf("expected right child to be a DoltgresType but got `%T`", anyExpr.rightExpr) } - rightType := arrType.BaseType() - + rightType := arrType.ArrayBaseType() op, err := framework.GetOperatorFromString(anyExpr.subOperator) if err != nil { return nil, err } - if leftType, ok := anyExpr.leftExpr.Type().(pgtypes.DoltgresType); ok { + if leftType, ok := anyExpr.leftExpr.Type().(*pgtypes.DoltgresType); ok { // Resolve comparison function once and reuse the function in Eval. staticLiteral := &Literal{typ: leftType} arrayLiteral := &Literal{typ: rightType} @@ -340,7 +340,7 @@ func anyExpressionWithChildren(anyExpr *AnyExpr) (sql.Expression, error) { if compFunc == nil { return nil, fmt.Errorf("operator does not exist: %s = %s", leftType.String(), rightType.String()) } - if compFunc.Type().(pgtypes.DoltgresType).BaseID() != pgtypes.DoltgresTypeBaseID_Bool { + if compFunc.Type().(*pgtypes.DoltgresType).OID != uint32(oid.T_bool) { // This should never happen, but this is just to be safe return nil, fmt.Errorf("%T: found equality comparison that does not return a bool", anyExpr) } diff --git a/server/expression/array.go b/server/expression/array.go index a733234f2f..f07ed73f65 100644 --- a/server/expression/array.go +++ b/server/expression/array.go @@ -20,6 +20,7 @@ import ( "github.com/dolthub/go-mysql-server/sql" vitess "github.com/dolthub/vitess/go/vt/sqlparser" + "github.com/lib/pq/oid" "github.com/dolthub/doltgresql/server/functions/framework" pgtypes "github.com/dolthub/doltgresql/server/types" @@ -28,7 +29,7 @@ import ( // Array represents an ARRAY[...] expression. type Array struct { children []sql.Expression - coercedType pgtypes.DoltgresArrayType + coercedType *pgtypes.DoltgresType } var _ vitess.Injectable = (*Array)(nil) @@ -36,9 +37,15 @@ var _ sql.Expression = (*Array)(nil) // NewArray returns a new *Array. func NewArray(coercedType sql.Type) (*Array, error) { - var arrayCoercedType pgtypes.DoltgresArrayType - if dat, ok := coercedType.(pgtypes.DoltgresArrayType); ok { - arrayCoercedType = dat + var arrayCoercedType *pgtypes.DoltgresType + if dt, ok := coercedType.(*pgtypes.DoltgresType); ok { + if dt.IsEmptyType() { + // DoltgresType pointer can be nil + } else if dt.IsArrayType() { + arrayCoercedType = dt + } else if !dt.IsEmptyType() { + return nil, fmt.Errorf("cannot cast array to %s", coercedType.String()) + } } else if coercedType != nil { return nil, fmt.Errorf("cannot cast array to %s", coercedType.String()) } @@ -55,7 +62,7 @@ func (array *Array) Children() []sql.Expression { // Eval implements the sql.Expression interface. func (array *Array) Eval(ctx *sql.Context, row sql.Row) (any, error) { - resultTyp := array.coercedType.BaseType() + resultTyp := array.coercedType.ArrayBaseType() values := make([]any, len(array.children)) for i, expr := range array.children { val, err := expr.Eval(ctx, row) @@ -68,15 +75,15 @@ func (array *Array) Eval(ctx *sql.Context, row sql.Row) (any, error) { continue } - doltgresType, ok := expr.Type().(pgtypes.DoltgresType) + doltgresType, ok := expr.Type().(*pgtypes.DoltgresType) if !ok { return nil, fmt.Errorf("expected DoltgresType, but got %s", expr.Type().String()) } // We always cast the element, as there may be parameter restrictions in place - castFunc := framework.GetImplicitCast(doltgresType.BaseID(), resultTyp.BaseID()) + castFunc := framework.GetImplicitCast(doltgresType, resultTyp) if castFunc == nil { - if doltgresType.BaseID() == pgtypes.DoltgresTypeBaseID_Unknown { + if doltgresType.OID == uint32(oid.T_unknown) { castFunc = framework.UnknownLiteralCast } else { return nil, fmt.Errorf("cannot find cast function from %s to %s", doltgresType.String(), resultTyp.String()) @@ -157,21 +164,21 @@ func (array *Array) WithResolvedChildren(children []any) (any, error) { // getTargetType returns the evaluated type for this expression. // Returns the "anyarray" type if the type combination is invalid. -func (array *Array) getTargetType(children ...sql.Expression) (pgtypes.DoltgresArrayType, error) { - var childrenTypes []pgtypes.DoltgresTypeBaseID +func (array *Array) getTargetType(children ...sql.Expression) (*pgtypes.DoltgresType, error) { + var childrenTypes []*pgtypes.DoltgresType for _, child := range children { if child != nil { - childType, ok := child.Type().(pgtypes.DoltgresType) + childType, ok := child.Type().(*pgtypes.DoltgresType) if !ok { // We use "anyarray" as the indeterminate/invalid type return pgtypes.AnyArray, nil } - childrenTypes = append(childrenTypes, childType.BaseID()) + childrenTypes = append(childrenTypes, childType) } } targetType, err := framework.FindCommonType(childrenTypes) if err != nil { return nil, fmt.Errorf("ARRAY %s", err.Error()) } - return targetType.GetRepresentativeType().ToArrayType(), nil + return targetType.ToArrayType(), nil } diff --git a/server/expression/assignment_cast.go b/server/expression/assignment_cast.go index d257f210fe..10c50aa440 100644 --- a/server/expression/assignment_cast.go +++ b/server/expression/assignment_cast.go @@ -18,6 +18,7 @@ import ( "fmt" "github.com/dolthub/go-mysql-server/sql" + "github.com/lib/pq/oid" "github.com/dolthub/doltgresql/server/functions/framework" pgtypes "github.com/dolthub/doltgresql/server/types" @@ -26,14 +27,14 @@ import ( // AssignmentCast handles assignment casts. type AssignmentCast struct { expr sql.Expression - fromType pgtypes.DoltgresType - toType pgtypes.DoltgresType + fromType *pgtypes.DoltgresType + toType *pgtypes.DoltgresType } var _ sql.Expression = (*AssignmentCast)(nil) // NewAssignmentCast returns a new *AssignmentCast expression. -func NewAssignmentCast(expr sql.Expression, fromType pgtypes.DoltgresType, toType pgtypes.DoltgresType) *AssignmentCast { +func NewAssignmentCast(expr sql.Expression, fromType *pgtypes.DoltgresType, toType *pgtypes.DoltgresType) *AssignmentCast { toType = checkForDomainType(toType) fromType = checkForDomainType(fromType) return &AssignmentCast{ @@ -54,9 +55,9 @@ func (ac *AssignmentCast) Eval(ctx *sql.Context, row sql.Row) (any, error) { if err != nil || val == nil { return val, err } - castFunc := framework.GetAssignmentCast(ac.fromType.BaseID(), ac.toType.BaseID()) + castFunc := framework.GetAssignmentCast(ac.fromType, ac.toType) if castFunc == nil { - if ac.fromType.BaseID() == pgtypes.DoltgresTypeBaseID_Unknown { + if ac.fromType.OID == uint32(oid.T_unknown) { castFunc = framework.UnknownLiteralCast } else { return nil, fmt.Errorf("ASSIGNMENT_CAST: target is of type %s but expression is of type %s: %s", @@ -94,9 +95,9 @@ func (ac *AssignmentCast) WithChildren(children ...sql.Expression) (sql.Expressi return NewAssignmentCast(children[0], ac.fromType, ac.toType), nil } -func checkForDomainType(t pgtypes.DoltgresType) pgtypes.DoltgresType { - if dt, ok := t.(pgtypes.DomainType); ok { - t = dt.UnderlyingBaseType() +func checkForDomainType(t *pgtypes.DoltgresType) *pgtypes.DoltgresType { + if t.TypType == pgtypes.TypeType_Domain { + t = t.DomainUnderlyingBaseType() } return t } diff --git a/server/expression/binary_operator.go b/server/expression/binary_operator.go index 3ae06f3bb0..774b0127e3 100644 --- a/server/expression/binary_operator.go +++ b/server/expression/binary_operator.go @@ -27,7 +27,7 @@ import ( // BinaryOperator represents a VALUE OPERATOR VALUE expression. type BinaryOperator struct { operator framework.Operator - compiledFunc *framework.CompiledFunction + compiledFunc framework.Function } var _ vitess.Injectable = (*BinaryOperator)(nil) @@ -71,8 +71,16 @@ func (b *BinaryOperator) String() string { return fmt.Sprintf("? %s ?", b.operator.String()) } // We know that we'll always have two parameters here - return fmt.Sprintf("%s %s %s", - b.compiledFunc.Arguments[0].String(), b.operator.String(), b.compiledFunc.Arguments[1].String()) + switch f := b.compiledFunc.(type) { + case *framework.CompiledFunction: + return fmt.Sprintf("%s %s %s", + f.Arguments[0].String(), b.operator.String(), f.Arguments[1].String()) + case *framework.QuickFunction2: + return fmt.Sprintf("%s %s %s", + f.Arguments[0].String(), b.operator.String(), f.Arguments[1].String()) + default: + return fmt.Sprintf("unexpected binary operator function type: %T", b.compiledFunc) + } } // SwapParameters implements the expression.Equality interface. @@ -106,7 +114,7 @@ func (b *BinaryOperator) WithChildren(children ...sql.Expression) (sql.Expressio } return &BinaryOperator{ operator: b.operator, - compiledFunc: compiledFunc.(*framework.CompiledFunction), + compiledFunc: compiledFunc.(framework.Function), }, nil } @@ -143,11 +151,25 @@ func (b *BinaryOperator) Operator() framework.Operator { // Left implements the expression.BinaryExpression interface. func (b *BinaryOperator) Left() sql.Expression { // We know that we'll always have two parameters here - return b.compiledFunc.Arguments[0] + switch f := b.compiledFunc.(type) { + case *framework.CompiledFunction: + return f.Arguments[0] + case *framework.QuickFunction2: + return f.Arguments[0] + default: + return nil + } } // Right implements the expression.BinaryExpression interface. func (b *BinaryOperator) Right() sql.Expression { // We know that we'll always have two parameters here - return b.compiledFunc.Arguments[1] + switch f := b.compiledFunc.(type) { + case *framework.CompiledFunction: + return f.Arguments[1] + case *framework.QuickFunction2: + return f.Arguments[1] + default: + return nil + } } diff --git a/server/expression/explicit_cast.go b/server/expression/explicit_cast.go index 47839a0f20..db2bd2eff5 100644 --- a/server/expression/explicit_cast.go +++ b/server/expression/explicit_cast.go @@ -20,6 +20,7 @@ import ( "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/expression" vitess "github.com/dolthub/vitess/go/vt/sqlparser" + "github.com/lib/pq/oid" "github.com/dolthub/doltgresql/server/functions/framework" pgtypes "github.com/dolthub/doltgresql/server/types" @@ -28,7 +29,7 @@ import ( // ExplicitCast represents a VALUE::TYPE expression. type ExplicitCast struct { sqlChild sql.Expression - castToType pgtypes.DoltgresType + castToType *pgtypes.DoltgresType } var _ vitess.Injectable = (*ExplicitCast)(nil) @@ -36,7 +37,7 @@ var _ sql.Expression = (*ExplicitCast)(nil) // NewExplicitCastInjectable returns an incomplete *ExplicitCast that must be resolved through the vitess.Injectable interface. func NewExplicitCastInjectable(castToType sql.Type) (*ExplicitCast, error) { - pgtype, ok := castToType.(pgtypes.DoltgresType) + pgtype, ok := castToType.(*pgtypes.DoltgresType) if !ok { return nil, fmt.Errorf("cast expects a Doltgres type as the target type") } @@ -48,7 +49,7 @@ func NewExplicitCastInjectable(castToType sql.Type) (*ExplicitCast, error) { } // NewExplicitCast returns a new *ExplicitCast expression. -func NewExplicitCast(expr sql.Expression, toType pgtypes.DoltgresType) *ExplicitCast { +func NewExplicitCast(expr sql.Expression, toType *pgtypes.DoltgresType) *ExplicitCast { toType = checkForDomainType(toType) return &ExplicitCast{ sqlChild: expr, @@ -72,7 +73,7 @@ func (c *ExplicitCast) Eval(ctx *sql.Context, row sql.Row) (any, error) { if err != nil { return nil, err } - fromType, ok := c.sqlChild.Type().(pgtypes.DoltgresType) + fromType, ok := c.sqlChild.Type().(*pgtypes.DoltgresType) if !ok { // We'll leverage GMSCast to handle the conversion from a GMS type to a Doltgres type. // Rather than re-evaluating the expression, we put the result in a literal. @@ -87,9 +88,9 @@ func (c *ExplicitCast) Eval(ctx *sql.Context, row sql.Row) (any, error) { return nil, nil } - castFunction := framework.GetExplicitCast(fromType.BaseID(), c.castToType.BaseID()) + castFunction := framework.GetExplicitCast(fromType, c.castToType) if castFunction == nil { - if fromType.BaseID() == pgtypes.DoltgresTypeBaseID_Unknown { + if fromType.OID == uint32(oid.T_unknown) { castFunction = framework.UnknownLiteralCast } else { return nil, fmt.Errorf("EXPLICIT CAST: cast from `%s` to `%s` does not exist: %s", @@ -101,12 +102,12 @@ func (c *ExplicitCast) Eval(ctx *sql.Context, row sql.Row) (any, error) { // For string types and string array types, we intentionally ignore the error as using a length-restricted cast // is a way to intentionally truncate the data. All string types will always return the truncated result, even // during an error, so it's safe to use. - baseID := c.castToType.BaseID() - if arrayType, ok := c.castToType.BaseID().IsBaseIDArrayType(); ok { - baseID = arrayType.BaseType().BaseID() + castToType := c.castToType + if c.castToType.IsArrayType() { + castToType = c.castToType.ArrayBaseType() } // A nil result will be returned if there's a critical error, which we should never ignore. - if baseID.GetTypeCategory() != pgtypes.TypeCategory_StringTypes || castResult == nil { + if castToType.TypCategory != pgtypes.TypeCategory_StringTypes || castResult == nil { return nil, err } } diff --git a/server/expression/gms_cast.go b/server/expression/gms_cast.go index 26e64b6fc2..fbd164dedf 100644 --- a/server/expression/gms_cast.go +++ b/server/expression/gms_cast.go @@ -52,9 +52,9 @@ func (c *GMSCast) Child() sql.Expression { } // DoltgresType returns the DoltgresType that the cast evaluates to. This is the same value that is returned by Type(). -func (c *GMSCast) DoltgresType() pgtypes.DoltgresType { +func (c *GMSCast) DoltgresType() *pgtypes.DoltgresType { // GMSCast shouldn't receive a DoltgresType, but we shouldn't error if it happens - if t, ok := c.sqlChild.Type().(pgtypes.DoltgresType); ok { + if t, ok := c.sqlChild.Type().(*pgtypes.DoltgresType); ok { return t } @@ -71,7 +71,7 @@ func (c *GMSCast) Eval(ctx *sql.Context, row sql.Row) (any, error) { return nil, nil } // GMSCast shouldn't receive a DoltgresType, but we shouldn't error if it happens - if _, ok := c.sqlChild.Type().(pgtypes.DoltgresType); ok { + if _, ok := c.sqlChild.Type().(*pgtypes.DoltgresType); ok { return val, nil } sqlTyp := c.sqlChild.Type() diff --git a/server/expression/implicit_cast.go b/server/expression/implicit_cast.go index d698cf25c0..905ce6606e 100644 --- a/server/expression/implicit_cast.go +++ b/server/expression/implicit_cast.go @@ -26,14 +26,14 @@ import ( // ImplicitCast handles implicit casts. type ImplicitCast struct { expr sql.Expression - fromType pgtypes.DoltgresType - toType pgtypes.DoltgresType + fromType *pgtypes.DoltgresType + toType *pgtypes.DoltgresType } var _ sql.Expression = (*ImplicitCast)(nil) // NewImplicitCast returns a new *ImplicitCast expression. -func NewImplicitCast(expr sql.Expression, fromType pgtypes.DoltgresType, toType pgtypes.DoltgresType) *ImplicitCast { +func NewImplicitCast(expr sql.Expression, fromType *pgtypes.DoltgresType, toType *pgtypes.DoltgresType) *ImplicitCast { toType = checkForDomainType(toType) fromType = checkForDomainType(fromType) return &ImplicitCast{ @@ -54,7 +54,7 @@ func (ic *ImplicitCast) Eval(ctx *sql.Context, row sql.Row) (any, error) { if err != nil || val == nil { return val, err } - castFunc := framework.GetImplicitCast(ic.fromType.BaseID(), ic.toType.BaseID()) + castFunc := framework.GetImplicitCast(ic.fromType, ic.toType) if castFunc == nil { return nil, fmt.Errorf("target is of type %s but expression is of type %s", ic.toType.String(), ic.fromType.String()) } diff --git a/server/expression/in_subquery.go b/server/expression/in_subquery.go old mode 100755 new mode 100644 index b0735e9aae..1744a24ca1 --- a/server/expression/in_subquery.go +++ b/server/expression/in_subquery.go @@ -22,6 +22,7 @@ import ( "github.com/dolthub/go-mysql-server/sql/plan" "github.com/dolthub/go-mysql-server/sql/types" vitess "github.com/dolthub/vitess/go/vt/sqlparser" + "github.com/lib/pq/oid" "github.com/dolthub/doltgresql/server/functions/framework" pgtypes "github.com/dolthub/doltgresql/server/types" @@ -36,7 +37,7 @@ type InSubquery struct { // These are assigned in WithChildren, so refer there for more information. leftLiteral *Literal rightLiterals []*Literal - compFuncs []*framework.CompiledFunction + compFuncs []framework.Function } var _ vitess.Injectable = (*InSubquery)(nil) @@ -185,7 +186,7 @@ func (in *InSubquery) WithChildren(children ...sql.Expression) (sql.Expression, } // We'll only resolve the comparison functions once we have all Doltgres types. // We may see GMS types during some analyzer steps, so we should wait until those are done. - if leftType, ok := children[0].Type().(pgtypes.DoltgresType); ok { + if leftType, ok := children[0].Type().(*pgtypes.DoltgresType); ok { // Rather than finding and resolving a comparison function every time we call Eval, we resolve them once and // reuse the functions. We also want to avoid re-assigning the parameters of the comparison functions since that // will also cause the functions to resolve again. To do this, we store expressions within our struct that the @@ -199,10 +200,10 @@ func (in *InSubquery) WithChildren(children ...sql.Expression) (sql.Expression, sch := sq.Query.Schema() leftLiteral := &Literal{typ: leftType} rightLiterals := make([]*Literal, len(sch)) - compFuncs := make([]*framework.CompiledFunction, len(sch)) + compFuncs := make([]framework.Function, len(sch)) allValidChildren := true for i, rightCol := range sch { - rightType, ok := rightCol.Type.(pgtypes.DoltgresType) + rightType, ok := rightCol.Type.(*pgtypes.DoltgresType) if !ok { allValidChildren = false break @@ -212,7 +213,7 @@ func (in *InSubquery) WithChildren(children ...sql.Expression) (sql.Expression, if compFuncs[i] == nil { return nil, fmt.Errorf("operator does not exist: %s = %s", leftType.String(), rightType.String()) } - if compFuncs[i].Type().(pgtypes.DoltgresType).BaseID() != pgtypes.DoltgresTypeBaseID_Bool { + if compFuncs[i].Type().(*pgtypes.DoltgresType).OID != uint32(oid.T_bool) { // This should never happen, but this is just to be safe return nil, fmt.Errorf("%T: found equality comparison that does not return a bool", in) } diff --git a/server/expression/in_tuple.go b/server/expression/in_tuple.go index ae1c78084e..ed3977440a 100644 --- a/server/expression/in_tuple.go +++ b/server/expression/in_tuple.go @@ -20,6 +20,7 @@ import ( "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/expression" vitess "github.com/dolthub/vitess/go/vt/sqlparser" + "github.com/lib/pq/oid" "github.com/dolthub/doltgresql/server/functions/framework" pgtypes "github.com/dolthub/doltgresql/server/types" @@ -34,7 +35,7 @@ type InTuple struct { // These are assigned in WithChildren, so refer there for more information. staticLiteral *Literal arrayLiterals []*Literal - compFuncs []*framework.CompiledFunction + compFuncs []framework.Function } var _ vitess.Injectable = (*BinaryOperator)(nil) @@ -57,13 +58,23 @@ func (it *InTuple) Children() []sql.Expression { // Decay returns the expression as a series of OR expressions. The behavior is not the same, however it allows some // paths to simplify their expression handling (such as filters). func (it *InTuple) Decay() sql.Expression { - it.compFuncs[0].Arguments = []sql.Expression{it.leftExpr, it.rightExpr[0]} + switch f := it.compFuncs[0].(type) { + case *framework.CompiledFunction: + f.Arguments = []sql.Expression{it.leftExpr, it.rightExpr[0]} + case *framework.QuickFunction2: + f.Arguments = [2]sql.Expression{it.leftExpr, it.rightExpr[0]} + } var expr sql.Expression = &BinaryOperator{ operator: framework.Operator_BinaryEqual, compiledFunc: it.compFuncs[0], } for i := 1; i < len(it.rightExpr); i++ { - it.compFuncs[i].Arguments = []sql.Expression{it.leftExpr, it.rightExpr[i]} + switch f := it.compFuncs[i].(type) { + case *framework.CompiledFunction: + f.Arguments = []sql.Expression{it.leftExpr, it.rightExpr[i]} + case *framework.QuickFunction2: + f.Arguments = [2]sql.Expression{it.leftExpr, it.rightExpr[i]} + } expr = expression.NewOr(expr, &BinaryOperator{ operator: framework.Operator_BinaryEqual, compiledFunc: it.compFuncs[i], @@ -175,7 +186,7 @@ func (it *InTuple) WithChildren(children ...sql.Expression) (sql.Expression, err } // We'll only resolve the comparison functions once we have all Doltgres types. // We may see GMS types during some analyzer steps, so we should wait until those are done. - if leftType, ok := children[0].Type().(pgtypes.DoltgresType); ok { + if leftType, ok := children[0].Type().(*pgtypes.DoltgresType); ok { // Rather than finding and resolving a comparison function every time we call Eval, we resolve them once and // reuse the functions. We also want to avoid re-assigning the parameters of the comparison functions since that // will also cause the functions to resolve again. To do this, we store expressions within our struct that the @@ -187,10 +198,10 @@ func (it *InTuple) WithChildren(children ...sql.Expression) (sql.Expression, err staticLiteral := &Literal{typ: leftType} arrayLiterals := make([]*Literal, len(rightTuple)) // Each expression may be a different type (which is valid), so we need a comparison function for each expression. - compFuncs := make([]*framework.CompiledFunction, len(rightTuple)) + compFuncs := make([]framework.Function, len(rightTuple)) allValidChildren := true for i, rightExpr := range rightTuple { - rightType, ok := rightExpr.Type().(pgtypes.DoltgresType) + rightType, ok := rightExpr.Type().(*pgtypes.DoltgresType) if !ok { allValidChildren = false break @@ -200,7 +211,7 @@ func (it *InTuple) WithChildren(children ...sql.Expression) (sql.Expression, err if compFuncs[i] == nil { return nil, fmt.Errorf("operator does not exist: %s = %s", leftType.String(), rightType.String()) } - if compFuncs[i].Type().(pgtypes.DoltgresType).BaseID() != pgtypes.DoltgresTypeBaseID_Bool { + if compFuncs[i].Type().(*pgtypes.DoltgresType).OID != uint32(oid.T_bool) { // This should never happen, but this is just to be safe return nil, fmt.Errorf("%T: found equality comparison that does not return a bool", it) } diff --git a/server/expression/literal.go b/server/expression/literal.go index 76418b3ff7..50c41970e0 100644 --- a/server/expression/literal.go +++ b/server/expression/literal.go @@ -17,10 +17,12 @@ package expression import ( "fmt" "strconv" + "strings" "time" "github.com/dolthub/go-mysql-server/sql" vitess "github.com/dolthub/vitess/go/vt/sqlparser" + "github.com/lib/pq/oid" "github.com/shopspring/decimal" "github.com/dolthub/doltgresql/postgres/parser/duration" @@ -32,7 +34,7 @@ import ( // Literal represents a raw literal (number, string, etc.). type Literal struct { value any - typ pgtypes.DoltgresType + typ *pgtypes.DoltgresType } var _ vitess.Injectable = (*Literal)(nil) @@ -214,7 +216,7 @@ func NewRawLiteralUuid(val uuid.UUID) *Literal { // NewUnsafeLiteral returns a new *Literal containing the given value and type. This should almost never be used, as // it does not perform any checking and circumvents type safety, which may lead to hard-to-debug errors. This is // currently only used within the analyzer, and will likely be removed in the future. -func NewUnsafeLiteral(val any, t pgtypes.DoltgresType) *Literal { +func NewUnsafeLiteral(val any, t *pgtypes.DoltgresType) *Literal { return &Literal{ value: val, typ: t, @@ -235,7 +237,7 @@ func (l *Literal) Eval(ctx *sql.Context, row sql.Row) (any, error) { } // GetDoltgresType implements the framework.LiteralInterface interface. -func (l *Literal) GetDoltgresType() pgtypes.DoltgresType { +func (l *Literal) GetDoltgresType() *pgtypes.DoltgresType { return l.typ } @@ -254,33 +256,38 @@ func (l *Literal) String() string { if l.value == nil { return "" } - str, err := l.typ.IoOutput(nil, l.value) + str, err := l.typ.FormatValue(l.value) if err != nil { - panic("got error from IoOutput") + panic(fmt.Sprintf("attempted to get string output for Literal: %s", err.Error())) + } + switch oid.Oid(l.typ.OID) { + case oid.T_char, oid.T_bpchar, oid.T_name, oid.T_text, oid.T_varchar, oid.T_unknown: + return `'` + strings.ReplaceAll(str, `'`, `''`) + `'` + default: + return str } - return pgtypes.QuoteString(l.typ.BaseID(), str) } // ToVitessLiteral returns the literal as a Vitess literal. This is strictly for situations where GMS is hardcoded to // expect a Vitess literal. This should only be used as a temporary measure, as the GMS code needs to be updated, or the // equivalent functionality should be built into Doltgres (recommend the second approach). func (l *Literal) ToVitessLiteral() *vitess.SQLVal { - switch l.typ.BaseID() { - case pgtypes.DoltgresTypeBaseID_Bool: + switch oid.Oid(l.typ.OID) { + case oid.T_bool: if l.value.(bool) { return vitess.NewIntVal([]byte("1")) } else { return vitess.NewIntVal([]byte("0")) } - case pgtypes.DoltgresTypeBaseID_Int32: + case oid.T_int4: return vitess.NewIntVal([]byte(strconv.FormatInt(int64(l.value.(int32)), 10))) - case pgtypes.DoltgresTypeBaseID_Int64: + case oid.T_int8: return vitess.NewIntVal([]byte(strconv.FormatInt(l.value.(int64), 10))) - case pgtypes.DoltgresTypeBaseID_Numeric: + case oid.T_numeric: return vitess.NewFloatVal([]byte(l.value.(decimal.Decimal).String())) - case pgtypes.DoltgresTypeBaseID_Text: + case oid.T_text: return vitess.NewStrVal([]byte(l.value.(string))) - case pgtypes.DoltgresTypeBaseID_Unknown: + case oid.T_unknown: if l.value == nil { return nil } else if str, ok := l.value.(string); ok { diff --git a/server/expression/unary_operator.go b/server/expression/unary_operator.go index 3465014b57..25bbb8f519 100644 --- a/server/expression/unary_operator.go +++ b/server/expression/unary_operator.go @@ -26,7 +26,7 @@ import ( // UnaryOperator represents a VALUE OPERATOR VALUE expression. type UnaryOperator struct { operator framework.Operator - compiledFunc *framework.CompiledFunction + compiledFunc framework.Function } var _ vitess.Injectable = (*UnaryOperator)(nil) @@ -63,7 +63,14 @@ func (b *UnaryOperator) String() string { return fmt.Sprintf("%s?", b.operator.String()) } // We know that we'll always have one parameter here - return fmt.Sprintf("%s%s", b.operator.String(), b.compiledFunc.Arguments[0].String()) + switch f := b.compiledFunc.(type) { + case *framework.CompiledFunction: + return fmt.Sprintf("%s%s", b.operator.String(), f.Arguments[0].String()) + case *framework.QuickFunction1: + return fmt.Sprintf("%s%s", b.operator.String(), f.Argument.String()) + default: + return fmt.Sprintf("unexpected unary operator function type: %T", b.compiledFunc) + } } // Type implements the sql.Expression interface. @@ -82,7 +89,7 @@ func (b *UnaryOperator) WithChildren(children ...sql.Expression) (sql.Expression } return &UnaryOperator{ operator: b.operator, - compiledFunc: compiledFunc.(*framework.CompiledFunction), + compiledFunc: compiledFunc.(framework.Function), }, nil } diff --git a/server/functions/abs.go b/server/functions/abs.go index c75bc71326..b60ff4a153 100644 --- a/server/functions/abs.go +++ b/server/functions/abs.go @@ -36,9 +36,9 @@ func initAbs() { var abs_int16 = framework.Function1{ Name: "abs", Return: pgtypes.Int16, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int16}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return utils.Abs(val1.(int16)), nil }, } @@ -47,9 +47,9 @@ var abs_int16 = framework.Function1{ var abs_int32 = framework.Function1{ Name: "abs", Return: pgtypes.Int32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int32}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return utils.Abs(val1.(int32)), nil }, } @@ -58,9 +58,9 @@ var abs_int32 = framework.Function1{ var abs_int64 = framework.Function1{ Name: "abs", Return: pgtypes.Int64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return utils.Abs(val1.(int64)), nil }, } @@ -69,9 +69,9 @@ var abs_int64 = framework.Function1{ var abs_float64 = framework.Function1{ Name: "abs", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return utils.Abs(val1.(float64)), nil }, } @@ -80,9 +80,9 @@ var abs_float64 = framework.Function1{ var abs_numeric = framework.Function1{ Name: "abs", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return val1.(decimal.Decimal).Abs(), nil }, } diff --git a/server/functions/acos.go b/server/functions/acos.go index 47da54b15f..238b158a3f 100644 --- a/server/functions/acos.go +++ b/server/functions/acos.go @@ -33,9 +33,9 @@ func initAcos() { var acos_float64 = framework.Function1{ Name: "acos", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { r := math.Acos(val1.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/acosd.go b/server/functions/acosd.go index 2bbc2192e5..999670c61c 100644 --- a/server/functions/acosd.go +++ b/server/functions/acosd.go @@ -33,9 +33,9 @@ func initAcosd() { var acosd_float64 = framework.Function1{ Name: "acosd", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { r := math.Acos(val1.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/acosh.go b/server/functions/acosh.go index 6d26615649..d44fda20dc 100644 --- a/server/functions/acosh.go +++ b/server/functions/acosh.go @@ -33,9 +33,9 @@ func initAcosh() { var acosh_float64 = framework.Function1{ Name: "acosh", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { r := math.Acosh(val1.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/age.go b/server/functions/age.go index 21facec090..2735388156 100644 --- a/server/functions/age.go +++ b/server/functions/age.go @@ -34,10 +34,10 @@ func initAge() { var age_timestamp_timestamp = framework.Function2{ Name: "age", Return: pgtypes.Interval, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { t1 := val1.(time.Time) t2 := val2.(time.Time) return diffTimes(t1, t2), nil @@ -48,10 +48,10 @@ var age_timestamp_timestamp = framework.Function2{ var age_timestamp = framework.Function1{ Name: "age", Return: pgtypes.Interval, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Timestamp}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Timestamp}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { t := val.(time.Time) // current_date (at midnight) cur, err := time.Parse("2006-01-02", time.Now().Format("2006-01-02")) diff --git a/server/functions/any.go b/server/functions/any.go new file mode 100644 index 0000000000..51f99e6940 --- /dev/null +++ b/server/functions/any.go @@ -0,0 +1,52 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initAny registers the functions to the catalog. +func initAny() { + framework.RegisterFunction(any_in) + framework.RegisterFunction(any_out) +} + +// any_in represents the PostgreSQL function of any type IO input. +var any_in = framework.Function1{ + Name: "any_in", + Return: pgtypes.Any, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return nil, nil + }, +} + +// any_out represents the PostgreSQL function of any type IO output. +var any_out = framework.Function1{ + Name: "any_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Any}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return "", nil + }, +} diff --git a/server/functions/anyarray.go b/server/functions/anyarray.go new file mode 100644 index 0000000000..b747b7e17d --- /dev/null +++ b/server/functions/anyarray.go @@ -0,0 +1,78 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initAnyArray registers the functions to the catalog. +func initAnyArray() { + framework.RegisterFunction(anyarray_in) + framework.RegisterFunction(anyarray_out) + framework.RegisterFunction(anyarray_recv) + framework.RegisterFunction(anyarray_send) +} + +// anyarray_in represents the PostgreSQL function of anyarray type IO input. +var anyarray_in = framework.Function1{ + Name: "anyarray_in", + Return: pgtypes.AnyArray, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return []any{}, nil + }, +} + +// anyarray_out represents the PostgreSQL function of anyarray type IO output. +var anyarray_out = framework.Function1{ + Name: "anyarray_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.AnyArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return "", nil + }, +} + +// anyarray_recv represents the PostgreSQL function of anyarray type IO receive. +var anyarray_recv = framework.Function1{ + Name: "anyarray_recv", + Return: pgtypes.AnyArray, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return []any{}, nil + }, +} + +// anyarray_send represents the PostgreSQL function of anyarray type IO send. +var anyarray_send = framework.Function1{ + Name: "anyarray_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.AnyArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return []byte{}, nil + }, +} diff --git a/server/functions/anyelement.go b/server/functions/anyelement.go new file mode 100644 index 0000000000..50c836004c --- /dev/null +++ b/server/functions/anyelement.go @@ -0,0 +1,52 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initAnyElement registers the functions to the catalog. +func initAnyElement() { + framework.RegisterFunction(anyelement_in) + framework.RegisterFunction(anyelement_out) +} + +// anyelement_in represents the PostgreSQL function of anyelement type IO input. +var anyelement_in = framework.Function1{ + Name: "anyelement_in", + Return: pgtypes.AnyElement, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return nil, nil + }, +} + +// anyelement_out represents the PostgreSQL function of anyelement type IO output. +var anyelement_out = framework.Function1{ + Name: "anyelement_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.AnyElement}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return "", nil + }, +} diff --git a/server/functions/anynonarray.go b/server/functions/anynonarray.go new file mode 100644 index 0000000000..9f7869d473 --- /dev/null +++ b/server/functions/anynonarray.go @@ -0,0 +1,52 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initAnyNonArray registers the functions to the catalog. +func initAnyNonArray() { + framework.RegisterFunction(anynonarray_in) + framework.RegisterFunction(anynonarray_out) +} + +// anynonarray_in represents the PostgreSQL function of anynonarray type IO input. +var anynonarray_in = framework.Function1{ + Name: "anynonarray_in", + Return: pgtypes.AnyNonArray, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return nil, nil + }, +} + +// anynonarray_out represents the PostgreSQL function of anynonarray type IO output. +var anynonarray_out = framework.Function1{ + Name: "anynonarray_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.AnyNonArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return "", nil + }, +} diff --git a/server/functions/array.go b/server/functions/array.go new file mode 100644 index 0000000000..25e4fec053 --- /dev/null +++ b/server/functions/array.go @@ -0,0 +1,302 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "bytes" + "encoding/binary" + "fmt" + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" + "github.com/dolthub/doltgresql/utils" +) + +// initArray registers the functions to the catalog. +func initArray() { + framework.RegisterFunction(array_in) + framework.RegisterFunction(array_out) + framework.RegisterFunction(array_recv) + framework.RegisterFunction(array_send) + framework.RegisterFunction(btarraycmp) + framework.RegisterFunction(array_subscript_handler) +} + +// array_in represents the PostgreSQL function of array type IO input. +var array_in = framework.Function3{ + Name: "array_in", + Return: pgtypes.AnyArray, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Cstring, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + input := val1.(string) + baseTypeOid := val2.(uint32) + baseType := pgtypes.OidToBuiltInDoltgresType[baseTypeOid] + typmod := val3.(int32) + baseType = baseType.WithAttTypMod(typmod) + if len(input) < 2 || input[0] != '{' || input[len(input)-1] != '}' { + // This error is regarded as a critical error, and thus we immediately return the error alongside a nil + // value. Returning a nil value is a signal to not ignore the error. + return nil, fmt.Errorf(`malformed array literal: "%s"`, input) + } + // We'll remove the surrounding braces since we've already verified that they're there + input = input[1 : len(input)-1] + var values []any + var err error + sb := strings.Builder{} + quoteStartCount := 0 + quoteEndCount := 0 + escaped := false + // Iterate over each rune in the input to collect and process the rune elements + for _, r := range input { + if escaped { + sb.WriteRune(r) + escaped = false + } else if quoteStartCount > quoteEndCount { + switch r { + case '\\': + escaped = true + case '"': + quoteEndCount++ + default: + sb.WriteRune(r) + } + } else { + switch r { + case ' ', '\t', '\n', '\r': + continue + case '\\': + escaped = true + case '"': + quoteStartCount++ + case ',': + if quoteStartCount >= 2 { + // This is a malformed string, thus we treat it as a critical error. + return nil, fmt.Errorf(`malformed array literal: "%s"`, input) + } + str := sb.String() + var innerValue any + if quoteStartCount == 0 && strings.EqualFold(str, "null") { + // An unquoted case-insensitive NULL is treated as an actual null value + innerValue = nil + } else { + var nErr error + innerValue, nErr = baseType.IoInput(ctx, str) + if nErr != nil && err == nil { + // This is a non-critical error, therefore the error may be ignored at a higher layer (such as + // an explicit cast) and the inner type will still return a valid result, so we must allow the + // values to propagate. + err = nErr + } + } + values = append(values, innerValue) + sb.Reset() + quoteStartCount = 0 + quoteEndCount = 0 + default: + sb.WriteRune(r) + } + } + } + // Use anything remaining in the buffer as the last element + if sb.Len() > 0 { + if escaped || quoteStartCount > quoteEndCount || quoteStartCount >= 2 { + // These errors are regarded as critical errors, and thus we immediately return the error alongside a nil + // value. Returning a nil value is a signal to not ignore the error. + return nil, fmt.Errorf(`malformed array literal: "%s"`, input) + } else { + str := sb.String() + var innerValue any + if quoteStartCount == 0 && strings.EqualFold(str, "NULL") { + // An unquoted case-insensitive NULL is treated as an actual null value + innerValue = nil + } else { + var nErr error + innerValue, nErr = baseType.IoInput(ctx, str) + if nErr != nil && err == nil { + // This is a non-critical error, therefore the error may be ignored at a higher layer (such as + // an explicit cast) and the inner type will still return a valid result, so we must allow the + // values to propagate. + err = nErr + } + } + values = append(values, innerValue) + } + } + + return values, err + }, +} + +// array_out represents the PostgreSQL function of array type IO output. +var array_out = framework.Function1{ + Name: "array_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.AnyArray}, + Strict: true, + Callable: func(ctx *sql.Context, t [2]*pgtypes.DoltgresType, val any) (any, error) { + arrType := t[0] + baseType := arrType.ArrayBaseType() + return pgtypes.ArrToString(ctx, val.([]any), baseType, false) + }, +} + +// array_recv represents the PostgreSQL function of array type IO receive. +var array_recv = framework.Function3{ + Name: "array_recv", + Return: pgtypes.AnyArray, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Internal, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + data := val1.([]byte) + baseTypeOid := val2.(uint32) + baseType := pgtypes.OidToBuiltInDoltgresType[baseTypeOid] + typmod := val3.(int32) + baseType = baseType.WithAttTypMod(typmod) + // Check for the nil value, then ensure the minimum length of the slice + if len(data) == 0 { + return nil, nil + } + if len(data) < 4 { + return nil, fmt.Errorf("deserializing non-nil array value has invalid length of %d", len(data)) + } + // Grab the number of elements and construct an output slice of the appropriate size + elementCount := binary.LittleEndian.Uint32(data) + output := make([]any, elementCount) + // Read all elements + for i := uint32(0); i < elementCount; i++ { + // We read from i+1 to account for the element count at the beginning + offset := binary.LittleEndian.Uint32(data[(i+1)*4:]) + // If the value is null, then we can skip it, since the output slice default initializes all values to nil + if data[offset] == 1 { + continue + } + // The element data is everything from the offset to the next offset, excluding the null determinant + nextOffset := binary.LittleEndian.Uint32(data[(i+2)*4:]) + o, err := baseType.DeserializeValue(data[offset+1 : nextOffset]) + if err != nil { + return nil, err + } + output[i] = o + } + // Returns all read elements + return output, nil + }, +} + +// array_send represents the PostgreSQL function of array type IO send. +var array_send = framework.Function1{ + Name: "array_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.AnyArray}, + Strict: true, + Callable: func(ctx *sql.Context, t [2]*pgtypes.DoltgresType, val any) (any, error) { + arrType := t[0] + baseType := arrType.ArrayBaseType() + vals := val.([]any) + + bb := bytes.Buffer{} + // Write the element count to a buffer. We're using an array since it's stack-allocated, so no need for pooling. + var elementCount [4]byte + binary.LittleEndian.PutUint32(elementCount[:], uint32(len(vals))) + bb.Write(elementCount[:]) + // Create an array that contains the offsets for each value. Since we can't update the offset portion of the buffer + // as we determine the offsets, we have to track them outside the buffer. We'll overwrite the buffer later with the + // correct offsets. The last offset represents the end of the slice, which simplifies the logic for reading elements + // using the "current offset to next offset" strategy. We use a byte slice since the buffer only works with byte + // slices. + offsets := make([]byte, (len(vals)+1)*4) + bb.Write(offsets) + // The starting offset for the first element is Count(uint32) + (NumberOfElementOffsets * sizeof(uint32)) + currentOffset := uint32(4 + (len(vals)+1)*4) + for i := range vals { + // Write the current offset + binary.LittleEndian.PutUint32(offsets[i*4:], currentOffset) + // Handle serialization of the value + // TODO: ARRAYs may be multidimensional, such as ARRAY[[4,2],[6,3]], which isn't accounted for here + serializedVal, err := baseType.SerializeValue(vals[i]) + if err != nil { + return nil, err + } + // Handle the nil case and non-nil case + if serializedVal == nil { + bb.WriteByte(1) + currentOffset += 1 + } else { + bb.WriteByte(0) + bb.Write(serializedVal) + currentOffset += 1 + uint32(len(serializedVal)) + } + } + // Write the final offset, which will equal the length of the serialized slice + binary.LittleEndian.PutUint32(offsets[len(offsets)-4:], currentOffset) + // Get the final output, and write the updated offsets to it + outputBytes := bb.Bytes() + copy(outputBytes[4:], offsets) + return outputBytes, nil + }, +} + +// btarraycmp represents the PostgreSQL function of array type byte compare. +var btarraycmp = framework.Function2{ + Name: "btarraycmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.AnyArray, pgtypes.AnyArray}, + Strict: true, + Callable: func(ctx *sql.Context, t [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + at := t[0] + bt := t[1] + if !at.Equals(bt) { + // TODO: currently, types should match. + // Technically, does not have to e.g.: float4 vs float8 + return nil, fmt.Errorf("different type comparison is not supported yet") + } + + ab := val1.([]any) + bb := val2.([]any) + minLength := utils.Min(len(ab), len(bb)) + for i := 0; i < minLength; i++ { + res, err := at.ArrayBaseType().Compare(ab[i], bb[i]) + if err != nil { + return 0, err + } + if res != 0 { + return res, nil + } + } + if len(ab) == len(bb) { + return int32(0), nil + } else if len(ab) < len(bb) { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// array_subscript_handler represents the PostgreSQL function of array type subscript handler. +var array_subscript_handler = framework.Function1{ + Name: "array_subscript_handler", + Return: pgtypes.Internal, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, t [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return []byte{}, nil + }, +} diff --git a/server/functions/array_append.go b/server/functions/array_append.go index d8938e982a..4edf007aa6 100644 --- a/server/functions/array_append.go +++ b/server/functions/array_append.go @@ -30,8 +30,8 @@ func initArrayAppend() { var array_append_anyarray_anyelement = framework.Function2{ Name: "array_append", Return: pgtypes.AnyArray, - Parameters: [2]pgtypes.DoltgresType{pgtypes.AnyArray, pgtypes.AnyElement}, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Parameters: [2]*pgtypes.DoltgresType{pgtypes.AnyArray, pgtypes.AnyElement}, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val1 == nil { return []any{val2}, nil } diff --git a/server/functions/array_to_string.go b/server/functions/array_to_string.go index e11c9c7f45..62858f884d 100644 --- a/server/functions/array_to_string.go +++ b/server/functions/array_to_string.go @@ -34,10 +34,10 @@ func initArrayToString() { var array_to_string_anyarray_text = framework.Function2{ Name: "array_to_string", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.AnyArray, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.AnyArray, pgtypes.Text}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, paramsAndReturn [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, paramsAndReturn [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { arr := val1.([]any) delimiter := val2.(string) return getStringArrFromAnyArray(ctx, paramsAndReturn[0], arr, delimiter, nil) @@ -48,10 +48,10 @@ var array_to_string_anyarray_text = framework.Function2{ var array_to_string_anyarray_text_text = framework.Function3{ Name: "array_to_string", Return: pgtypes.Text, - Parameters: [3]pgtypes.DoltgresType{pgtypes.AnyArray, pgtypes.Text, pgtypes.Text}, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.AnyArray, pgtypes.Text, pgtypes.Text}, IsNonDeterministic: true, Strict: false, - Callable: func(ctx *sql.Context, paramsAndReturn [4]pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + Callable: func(ctx *sql.Context, paramsAndReturn [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { if val1 == nil { return nil, fmt.Errorf("could not determine polymorphic type because input has type unknown") } else if val2 == nil { @@ -65,8 +65,8 @@ var array_to_string_anyarray_text_text = framework.Function3{ // getStringArrFromAnyArray takes inputs of any array, delimiter and null entry replacement. It uses the IoOutput() of the // base type of the AnyArray type to get string representation of array elements. -func getStringArrFromAnyArray(ctx *sql.Context, anyArrayType pgtypes.DoltgresType, arr []any, delimiter string, nullEntry any) (string, error) { - baseType := anyArrayType.ToArrayType().BaseType() +func getStringArrFromAnyArray(ctx *sql.Context, arrType *pgtypes.DoltgresType, arr []any, delimiter string, nullEntry any) (string, error) { + baseType := arrType.ArrayBaseType() strs := make([]string, 0) for _, el := range arr { if el != nil { diff --git a/server/functions/ascii.go b/server/functions/ascii.go index 1fbe0cdc5a..6d6d89b4a0 100644 --- a/server/functions/ascii.go +++ b/server/functions/ascii.go @@ -30,9 +30,9 @@ func initAscii() { var ascii_text = framework.Function1{ Name: "ascii", Return: pgtypes.Int32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1Interface any) (any, error) { val1 := val1Interface.(string) if len(val1) == 0 { return int32(0), nil diff --git a/server/functions/asin.go b/server/functions/asin.go index a75049c9ee..ee3eeee2cc 100644 --- a/server/functions/asin.go +++ b/server/functions/asin.go @@ -33,9 +33,9 @@ func initAsin() { var asin_float64 = framework.Function1{ Name: "asin", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { r := math.Asin(val1.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/asind.go b/server/functions/asind.go index bf290bf912..3db50f0382 100644 --- a/server/functions/asind.go +++ b/server/functions/asind.go @@ -33,9 +33,9 @@ func initAsind() { var asind_float64 = framework.Function1{ Name: "asind", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { r := math.Asin(val1.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/asinh.go b/server/functions/asinh.go index 24076c6456..99354988d8 100644 --- a/server/functions/asinh.go +++ b/server/functions/asinh.go @@ -33,9 +33,9 @@ func initAsinh() { var asinh_float64 = framework.Function1{ Name: "asinh", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { r := math.Asinh(val1.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/atan.go b/server/functions/atan.go index 3018e0d1dd..caf460d176 100644 --- a/server/functions/atan.go +++ b/server/functions/atan.go @@ -33,9 +33,9 @@ func initAtan() { var atan_float64 = framework.Function1{ Name: "atan", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { r := math.Atan(val1.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/atan2.go b/server/functions/atan2.go index bc3b3349df..4aca232b42 100644 --- a/server/functions/atan2.go +++ b/server/functions/atan2.go @@ -33,9 +33,9 @@ func initAtan2() { var atan2_float64 = framework.Function2{ Name: "atan2", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, y any, x any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, y any, x any) (any, error) { r := math.Atan2(y.(float64), x.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/atan2d.go b/server/functions/atan2d.go index a53d07625a..f0645a9a02 100644 --- a/server/functions/atan2d.go +++ b/server/functions/atan2d.go @@ -33,9 +33,9 @@ func initAtan2d() { var atan2d_float64 = framework.Function2{ Name: "atan2d", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, y any, x any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, y any, x any) (any, error) { r := math.Atan2(y.(float64), x.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/atand.go b/server/functions/atand.go index d9d118747c..8410bd068a 100644 --- a/server/functions/atand.go +++ b/server/functions/atand.go @@ -33,9 +33,9 @@ func initAtand() { var atand_float64 = framework.Function1{ Name: "atand", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { r := math.Atan(val1.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/atanh.go b/server/functions/atanh.go index ac5b4d80db..1dddd7f357 100644 --- a/server/functions/atanh.go +++ b/server/functions/atanh.go @@ -33,9 +33,9 @@ func initAtanh() { var atanh_float64 = framework.Function1{ Name: "atanh", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { r := math.Atanh(val1.(float64)) if math.IsNaN(r) { return nil, fmt.Errorf("input is out of range") diff --git a/server/functions/binary/bit_and.go b/server/functions/binary/bit_and.go index 08a80fecc2..346096519c 100644 --- a/server/functions/binary/bit_and.go +++ b/server/functions/binary/bit_and.go @@ -35,9 +35,9 @@ func initBinaryBitAnd() { var int2and = framework.Function2{ Name: "int2and", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int16(val1.(int16) & val2.(int16)), nil }, } @@ -46,9 +46,9 @@ var int2and = framework.Function2{ var int4and = framework.Function2{ Name: "int4and", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int32(val1.(int32) & val2.(int32)), nil }, } @@ -57,9 +57,9 @@ var int4and = framework.Function2{ var int8and = framework.Function2{ Name: "int8and", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int64(val1.(int64) & val2.(int64)), nil }, } diff --git a/server/functions/binary/bit_or.go b/server/functions/binary/bit_or.go index 43088da375..caaaf5c0cd 100644 --- a/server/functions/binary/bit_or.go +++ b/server/functions/binary/bit_or.go @@ -35,9 +35,9 @@ func initBinaryBitOr() { var int2or = framework.Function2{ Name: "int2or", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int16(val1.(int16) | val2.(int16)), nil }, } @@ -46,9 +46,9 @@ var int2or = framework.Function2{ var int4or = framework.Function2{ Name: "int4or", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int32(val1.(int32) | val2.(int32)), nil }, } @@ -57,9 +57,9 @@ var int4or = framework.Function2{ var int8or = framework.Function2{ Name: "int8or", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int64(val1.(int64) | val2.(int64)), nil }, } diff --git a/server/functions/binary/bit_xor.go b/server/functions/binary/bit_xor.go index 6195f9e80f..10de19c109 100644 --- a/server/functions/binary/bit_xor.go +++ b/server/functions/binary/bit_xor.go @@ -35,9 +35,9 @@ func initBinaryBitXor() { var int2xor = framework.Function2{ Name: "int2xor", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int16(val1.(int16) ^ val2.(int16)), nil }, } @@ -46,9 +46,9 @@ var int2xor = framework.Function2{ var int4xor = framework.Function2{ Name: "int4xor", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int32(val1.(int32) ^ val2.(int32)), nil }, } @@ -57,9 +57,9 @@ var int4xor = framework.Function2{ var int8xor = framework.Function2{ Name: "int8xor", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int64(val1.(int64) ^ val2.(int64)), nil }, } diff --git a/server/functions/binary/concatenate.go b/server/functions/binary/concatenate.go index f5be0f2341..b53f8248d5 100644 --- a/server/functions/binary/concatenate.go +++ b/server/functions/binary/concatenate.go @@ -40,9 +40,9 @@ func initBinaryConcatenate() { var anytextcat = framework.Function2{ Name: "anytextcat", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.AnyNonArray, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.AnyNonArray, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, paramsAndReturn [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, paramsAndReturn [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { valType := paramsAndReturn[0] val1String, err := valType.IoOutput(ctx, val1) if err != nil { @@ -56,9 +56,9 @@ var anytextcat = framework.Function2{ var byteacat = framework.Function2{ Name: "byteacat", Return: pgtypes.Bytea, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, Strict: true, - Callable: func(ctx *sql.Context, paramsAndReturn [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, paramsAndReturn [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { v1 := val1.([]byte) v2 := val2.([]byte) copied := make([]byte, len(v1)+len(v2)) @@ -72,9 +72,9 @@ var byteacat = framework.Function2{ var jsonb_concat = framework.Function2{ Name: "jsonb_concat", Return: pgtypes.JsonB, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1Interface any, val2Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1Interface any, val2Interface any) (any, error) { val1 := val1Interface.(pgtypes.JsonDocument).Value val2 := val2Interface.(pgtypes.JsonDocument).Value // First we'll merge objects if they're both objects @@ -126,9 +126,9 @@ var jsonb_concat = framework.Function2{ var textanycat = framework.Function2{ Name: "textanycat", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.AnyNonArray}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.AnyNonArray}, Strict: true, - Callable: func(ctx *sql.Context, paramsAndReturn [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, paramsAndReturn [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { valType := paramsAndReturn[1] val2String, err := valType.IoOutput(ctx, val2) if err != nil { @@ -142,9 +142,9 @@ var textanycat = framework.Function2{ var textcat = framework.Function2{ Name: "textcat", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(string) + val2.(string), nil }, } diff --git a/server/functions/binary/divide.go b/server/functions/binary/divide.go index 224dafa8d4..4658d077ef 100644 --- a/server/functions/binary/divide.go +++ b/server/functions/binary/divide.go @@ -51,9 +51,9 @@ func initBinaryDivide() { var float4div = framework.Function2{ Name: "float4div", Return: pgtypes.Float32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(float32) == 0 { return nil, fmt.Errorf("division by zero") } @@ -65,9 +65,9 @@ var float4div = framework.Function2{ var float48div = framework.Function2{ Name: "float48div", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(float64) == 0 { return nil, fmt.Errorf("division by zero") } @@ -79,9 +79,9 @@ var float48div = framework.Function2{ var float8div = framework.Function2{ Name: "float8div", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(float64) == 0 { return nil, fmt.Errorf("division by zero") } @@ -93,9 +93,9 @@ var float8div = framework.Function2{ var float84div = framework.Function2{ Name: "float84div", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(float32) == 0 { return nil, fmt.Errorf("division by zero") } @@ -107,9 +107,9 @@ var float84div = framework.Function2{ var int2div = framework.Function2{ Name: "int2div", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int16) == 0 { return nil, fmt.Errorf("division by zero") } @@ -121,9 +121,9 @@ var int2div = framework.Function2{ var int24div = framework.Function2{ Name: "int24div", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int32) == 0 { return nil, fmt.Errorf("division by zero") } @@ -135,9 +135,9 @@ var int24div = framework.Function2{ var int28div = framework.Function2{ Name: "int28div", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int64) == 0 { return nil, fmt.Errorf("division by zero") } @@ -149,9 +149,9 @@ var int28div = framework.Function2{ var int4div = framework.Function2{ Name: "int4div", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int32) == 0 { return nil, fmt.Errorf("division by zero") } @@ -163,9 +163,9 @@ var int4div = framework.Function2{ var int42div = framework.Function2{ Name: "int42div", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int16) == 0 { return nil, fmt.Errorf("division by zero") } @@ -177,9 +177,9 @@ var int42div = framework.Function2{ var int48div = framework.Function2{ Name: "int48div", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int64) == 0 { return nil, fmt.Errorf("division by zero") } @@ -191,9 +191,9 @@ var int48div = framework.Function2{ var int8div = framework.Function2{ Name: "int8div", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int64) == 0 { return nil, fmt.Errorf("division by zero") } @@ -205,9 +205,9 @@ var int8div = framework.Function2{ var int82div = framework.Function2{ Name: "int82div", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int16) == 0 { return nil, fmt.Errorf("division by zero") } @@ -219,9 +219,9 @@ var int82div = framework.Function2{ var int84div = framework.Function2{ Name: "int84div", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int32) == 0 { return nil, fmt.Errorf("division by zero") } @@ -233,9 +233,9 @@ var int84div = framework.Function2{ var interval_div = framework.Function2{ Name: "interval_div", Return: pgtypes.Interval, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(float64) == 0 { return nil, fmt.Errorf("division by zero") } @@ -247,9 +247,9 @@ var interval_div = framework.Function2{ var numeric_div = framework.Function2{ Name: "numeric_div", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(decimal.Decimal).Equal(decimal.Zero) { return nil, fmt.Errorf("division by zero") } diff --git a/server/functions/binary/equal.go b/server/functions/binary/equal.go index b9e403fbab..29dd2ca474 100644 --- a/server/functions/binary/equal.go +++ b/server/functions/binary/equal.go @@ -76,9 +76,9 @@ func initBinaryEqual() { var booleq = framework.Function2{ Name: "booleq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bool.Compare(val1.(bool), val2.(bool)) return res == 0, err }, @@ -88,9 +88,9 @@ var booleq = framework.Function2{ var bpchareq = framework.Function2{ Name: "bpchareq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.BpChar.Compare(val1.(string), val2.(string)) return res == 0, err }, @@ -100,9 +100,9 @@ var bpchareq = framework.Function2{ var byteaeq = framework.Function2{ Name: "byteaeq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bytea.Compare(val1.([]byte), val2.([]byte)) return res == 0, err }, @@ -112,9 +112,9 @@ var byteaeq = framework.Function2{ var chareq = framework.Function2{ Name: "chareq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.InternalChar.Compare(val1.(string), val2.(string)) return res == 0, err }, @@ -124,9 +124,9 @@ var chareq = framework.Function2{ var date_eq = framework.Function2{ Name: "date_eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Date.Compare(val1.(time.Time), val2.(time.Time)) return res == 0, err }, @@ -136,9 +136,9 @@ var date_eq = framework.Function2{ var date_eq_timestamp = framework.Function2{ Name: "date_eq_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == 0, nil }, @@ -148,9 +148,9 @@ var date_eq_timestamp = framework.Function2{ var date_eq_timestamptz = framework.Function2{ Name: "date_eq_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == 0, nil }, @@ -160,9 +160,9 @@ var date_eq_timestamptz = framework.Function2{ var float4eq = framework.Function2{ Name: "float4eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float32.Compare(val1.(float32), val2.(float32)) return res == 0, err }, @@ -172,9 +172,9 @@ var float4eq = framework.Function2{ var float48eq = framework.Function2{ Name: "float48eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(float64(val1.(float32)), val2.(float64)) return res == 0, err }, @@ -184,9 +184,9 @@ var float48eq = framework.Function2{ var float84eq = framework.Function2{ Name: "float84eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), float64(val2.(float32))) return res == 0, err }, @@ -196,9 +196,9 @@ var float84eq = framework.Function2{ var float8eq = framework.Function2{ Name: "float8eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), val2.(float64)) return res == 0, err }, @@ -208,9 +208,9 @@ var float8eq = framework.Function2{ var int2eq = framework.Function2{ Name: "int2eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int16.Compare(val1.(int16), val2.(int16)) return res == 0, err }, @@ -220,9 +220,9 @@ var int2eq = framework.Function2{ var int24eq = framework.Function2{ Name: "int24eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(int32(val1.(int16)), val2.(int32)) return res == 0, err }, @@ -232,9 +232,9 @@ var int24eq = framework.Function2{ var int28eq = framework.Function2{ Name: "int28eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int16)), val2.(int64)) return res == 0, err }, @@ -244,9 +244,9 @@ var int28eq = framework.Function2{ var int42eq = framework.Function2{ Name: "int42eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), int32(val2.(int16))) return res == 0, err }, @@ -256,9 +256,9 @@ var int42eq = framework.Function2{ var int4eq = framework.Function2{ Name: "int4eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), val2.(int32)) return res == 0, err }, @@ -268,9 +268,9 @@ var int4eq = framework.Function2{ var int48eq = framework.Function2{ Name: "int48eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int32)), val2.(int64)) return res == 0, err }, @@ -280,9 +280,9 @@ var int48eq = framework.Function2{ var int82eq = framework.Function2{ Name: "int82eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int16))) return res == 0, err }, @@ -292,9 +292,9 @@ var int82eq = framework.Function2{ var int84eq = framework.Function2{ Name: "int84eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int32))) return res == 0, err }, @@ -304,9 +304,9 @@ var int84eq = framework.Function2{ var int8eq = framework.Function2{ Name: "int8eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), val2.(int64)) return res == 0, err }, @@ -316,9 +316,9 @@ var int8eq = framework.Function2{ var interval_eq = framework.Function2{ Name: "interval_eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Interval.Compare(val1.(duration.Duration), val2.(duration.Duration)) return res == 0, err }, @@ -328,9 +328,9 @@ var interval_eq = framework.Function2{ var jsonb_eq = framework.Function2{ Name: "jsonb_eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.JsonB.Compare(val1.(pgtypes.JsonDocument), val2.(pgtypes.JsonDocument)) return res == 0, err }, @@ -340,9 +340,9 @@ var jsonb_eq = framework.Function2{ var nameeq = framework.Function2{ Name: "nameeq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Name.Compare(val1.(string), val2.(string)) return res == 0, err }, @@ -352,9 +352,9 @@ var nameeq = framework.Function2{ var nameeqtext = framework.Function2{ Name: "nameeqtext", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res == 0, err }, @@ -364,9 +364,9 @@ var nameeqtext = framework.Function2{ var numeric_eq = framework.Function2{ Name: "numeric_eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Numeric.Compare(val1.(decimal.Decimal), val2.(decimal.Decimal)) return res == 0, err }, @@ -376,9 +376,9 @@ var numeric_eq = framework.Function2{ var oideq = framework.Function2{ Name: "oideq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Oid.Compare(val1.(uint32), val2.(uint32)) return res == 0, err }, @@ -388,9 +388,9 @@ var oideq = framework.Function2{ var texteqname = framework.Function2{ Name: "texteqname", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res == 0, err }, @@ -400,9 +400,9 @@ var texteqname = framework.Function2{ var text_eq = framework.Function2{ Name: "text_eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res == 0, err }, @@ -412,9 +412,9 @@ var text_eq = framework.Function2{ var time_eq = framework.Function2{ Name: "time_eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Time.Compare(val1.(time.Time), val2.(time.Time)) return res == 0, err }, @@ -424,9 +424,9 @@ var time_eq = framework.Function2{ var timestamp_eq_date = framework.Function2{ Name: "timestamp_eq_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == 0, nil }, @@ -436,9 +436,9 @@ var timestamp_eq_date = framework.Function2{ var timestamp_eq = framework.Function2{ Name: "timestamp_eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Timestamp.Compare(val1.(time.Time), val2.(time.Time)) return res == 0, err }, @@ -448,9 +448,9 @@ var timestamp_eq = framework.Function2{ var timestamp_eq_timestamptz = framework.Function2{ Name: "timestamp_eq_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == 0, err }, @@ -460,9 +460,9 @@ var timestamp_eq_timestamptz = framework.Function2{ var timestamptz_eq_date = framework.Function2{ Name: "timestamptz_eq_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == 0, nil }, @@ -472,9 +472,9 @@ var timestamptz_eq_date = framework.Function2{ var timestamptz_eq_timestamp = framework.Function2{ Name: "timestamptz_eq_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == 0, err }, @@ -484,9 +484,9 @@ var timestamptz_eq_timestamp = framework.Function2{ var timestamptz_eq = framework.Function2{ Name: "timestamptz_eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == 0, err }, @@ -496,9 +496,9 @@ var timestamptz_eq = framework.Function2{ var timetz_eq = framework.Function2{ Name: "timetz_eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimeTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == 0, err }, @@ -508,9 +508,9 @@ var timetz_eq = framework.Function2{ var uuid_eq = framework.Function2{ Name: "uuid_eq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Uuid.Compare(val1.(uuid.UUID), val2.(uuid.UUID)) return res == 0, err }, @@ -520,9 +520,9 @@ var uuid_eq = framework.Function2{ var xideqint4 = framework.Function2{ Name: "xideqint4", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Xid, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Xid, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: investigate the edge cases res, err := pgtypes.Int64.Compare(int64(val1.(uint32)), int64(val2.(int32))) return res == 0, err @@ -533,9 +533,9 @@ var xideqint4 = framework.Function2{ var xideq = framework.Function2{ Name: "xideq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Xid, pgtypes.Xid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Xid, pgtypes.Xid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Xid.Compare(val1.(uint32), val2.(uint32)) return res == 0, err }, diff --git a/server/functions/binary/greater.go b/server/functions/binary/greater.go index 1c87220787..702a9d5648 100644 --- a/server/functions/binary/greater.go +++ b/server/functions/binary/greater.go @@ -74,9 +74,9 @@ func initBinaryGreaterThan() { var boolgt = framework.Function2{ Name: "boolgt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bool.Compare(val1.(bool), val2.(bool)) return res == 1, err }, @@ -86,9 +86,9 @@ var boolgt = framework.Function2{ var bpchargt = framework.Function2{ Name: "bpchargt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.BpChar.Compare(val1.(string), val2.(string)) return res == 1, err }, @@ -98,9 +98,9 @@ var bpchargt = framework.Function2{ var byteagt = framework.Function2{ Name: "byteagt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bytea.Compare(val1.([]byte), val2.([]byte)) return res == 1, err }, @@ -110,9 +110,9 @@ var byteagt = framework.Function2{ var chargt = framework.Function2{ Name: "chargt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.InternalChar.Compare(val1.(string), val2.(string)) return res == 1, err }, @@ -122,9 +122,9 @@ var chargt = framework.Function2{ var date_gt = framework.Function2{ Name: "date_gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Date.Compare(val1.(time.Time), val2.(time.Time)) return res == 1, err }, @@ -134,9 +134,9 @@ var date_gt = framework.Function2{ var date_gt_timestamp = framework.Function2{ Name: "date_gt_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == 1, nil }, @@ -146,9 +146,9 @@ var date_gt_timestamp = framework.Function2{ var date_gt_timestamptz = framework.Function2{ Name: "date_gt_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == 1, nil }, @@ -158,9 +158,9 @@ var date_gt_timestamptz = framework.Function2{ var float4gt = framework.Function2{ Name: "float4gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float32.Compare(val1.(float32), val2.(float32)) return res == 1, err }, @@ -170,9 +170,9 @@ var float4gt = framework.Function2{ var float48gt = framework.Function2{ Name: "float48gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(float64(val1.(float32)), val2.(float64)) return res == 1, err }, @@ -182,9 +182,9 @@ var float48gt = framework.Function2{ var float84gt = framework.Function2{ Name: "float84gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), float64(val2.(float32))) return res == 1, err }, @@ -194,9 +194,9 @@ var float84gt = framework.Function2{ var float8gt = framework.Function2{ Name: "float8gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), val2.(float64)) return res == 1, err }, @@ -206,9 +206,9 @@ var float8gt = framework.Function2{ var int2gt = framework.Function2{ Name: "int2gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int16.Compare(val1.(int16), val2.(int16)) return res == 1, err }, @@ -218,9 +218,9 @@ var int2gt = framework.Function2{ var int24gt = framework.Function2{ Name: "int24gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(int32(val1.(int16)), val2.(int32)) return res == 1, err }, @@ -230,9 +230,9 @@ var int24gt = framework.Function2{ var int28gt = framework.Function2{ Name: "int28gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int16)), val2.(int64)) return res == 1, err }, @@ -242,9 +242,9 @@ var int28gt = framework.Function2{ var int42gt = framework.Function2{ Name: "int42gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), int32(val2.(int16))) return res == 1, err }, @@ -254,9 +254,9 @@ var int42gt = framework.Function2{ var int4gt = framework.Function2{ Name: "int4gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), val2.(int32)) return res == 1, err }, @@ -266,9 +266,9 @@ var int4gt = framework.Function2{ var int48gt = framework.Function2{ Name: "int48gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int32)), val2.(int64)) return res == 1, err }, @@ -278,9 +278,9 @@ var int48gt = framework.Function2{ var int82gt = framework.Function2{ Name: "int82gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int16))) return res == 1, err }, @@ -290,9 +290,9 @@ var int82gt = framework.Function2{ var int84gt = framework.Function2{ Name: "int84gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int32))) return res == 1, err }, @@ -302,9 +302,9 @@ var int84gt = framework.Function2{ var int8gt = framework.Function2{ Name: "int8gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), val2.(int64)) return res == 1, err }, @@ -314,9 +314,9 @@ var int8gt = framework.Function2{ var interval_gt = framework.Function2{ Name: "interval_gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Interval.Compare(val1.(duration.Duration), val2.(duration.Duration)) return res == 1, err }, @@ -326,9 +326,9 @@ var interval_gt = framework.Function2{ var jsonb_gt = framework.Function2{ Name: "jsonb_gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.JsonB.Compare(val1.(pgtypes.JsonDocument), val2.(pgtypes.JsonDocument)) return res == 1, err }, @@ -338,9 +338,9 @@ var jsonb_gt = framework.Function2{ var namegt = framework.Function2{ Name: "namegt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Name.Compare(val1.(string), val2.(string)) return res == 1, err }, @@ -350,9 +350,9 @@ var namegt = framework.Function2{ var namegttext = framework.Function2{ Name: "namegttext", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res == 1, err }, @@ -362,9 +362,9 @@ var namegttext = framework.Function2{ var numeric_gt = framework.Function2{ Name: "numeric_gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Numeric.Compare(val1.(decimal.Decimal), val2.(decimal.Decimal)) return res == 1, err }, @@ -374,9 +374,9 @@ var numeric_gt = framework.Function2{ var oidgt = framework.Function2{ Name: "oidgt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Oid.Compare(val1.(uint32), val2.(uint32)) return res == 1, err }, @@ -386,9 +386,9 @@ var oidgt = framework.Function2{ var textgtname = framework.Function2{ Name: "textgtname", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res == 1, err }, @@ -398,9 +398,9 @@ var textgtname = framework.Function2{ var text_gt = framework.Function2{ Name: "text_gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res == 1, err }, @@ -410,9 +410,9 @@ var text_gt = framework.Function2{ var time_gt = framework.Function2{ Name: "time_gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Time.Compare(val1.(time.Time), val2.(time.Time)) return res == 1, err }, @@ -422,9 +422,9 @@ var time_gt = framework.Function2{ var timestamp_gt_date = framework.Function2{ Name: "timestamp_gt_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == 1, nil }, @@ -434,9 +434,9 @@ var timestamp_gt_date = framework.Function2{ var timestamp_gt = framework.Function2{ Name: "timestamp_gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Timestamp.Compare(val1.(time.Time), val2.(time.Time)) return res == 1, err }, @@ -446,9 +446,9 @@ var timestamp_gt = framework.Function2{ var timestamp_gt_timestamptz = framework.Function2{ Name: "timestamp_gt_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == 1, err }, @@ -458,9 +458,9 @@ var timestamp_gt_timestamptz = framework.Function2{ var timestamptz_gt_date = framework.Function2{ Name: "timestamptz_gt_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == 1, nil }, @@ -470,9 +470,9 @@ var timestamptz_gt_date = framework.Function2{ var timestamptz_gt_timestamp = framework.Function2{ Name: "timestamptz_gt_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == 1, err }, @@ -482,9 +482,9 @@ var timestamptz_gt_timestamp = framework.Function2{ var timestamptz_gt = framework.Function2{ Name: "timestamptz_gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == 1, err }, @@ -494,9 +494,9 @@ var timestamptz_gt = framework.Function2{ var timetz_gt = framework.Function2{ Name: "timetz_gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimeTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == 1, err }, @@ -506,9 +506,9 @@ var timetz_gt = framework.Function2{ var uuid_gt = framework.Function2{ Name: "uuid_gt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Uuid.Compare(val1.(uuid.UUID), val2.(uuid.UUID)) return res == 1, err }, diff --git a/server/functions/binary/greater_equal.go b/server/functions/binary/greater_equal.go index 1179209a7f..32e13507b6 100644 --- a/server/functions/binary/greater_equal.go +++ b/server/functions/binary/greater_equal.go @@ -74,9 +74,9 @@ func initBinaryGreaterOrEqual() { var boolge = framework.Function2{ Name: "boolge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bool.Compare(val1.(bool), val2.(bool)) return res >= 0, err }, @@ -86,9 +86,9 @@ var boolge = framework.Function2{ var bpcharge = framework.Function2{ Name: "bpcharge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.BpChar.Compare(val1.(string), val2.(string)) return res >= 0, err }, @@ -98,9 +98,9 @@ var bpcharge = framework.Function2{ var byteage = framework.Function2{ Name: "byteage", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bytea.Compare(val1.([]byte), val2.([]byte)) return res >= 0, err }, @@ -110,9 +110,9 @@ var byteage = framework.Function2{ var charge = framework.Function2{ Name: "charge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.InternalChar.Compare(val1.(string), val2.(string)) return res >= 0, err }, @@ -122,9 +122,9 @@ var charge = framework.Function2{ var date_ge = framework.Function2{ Name: "date_ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Date.Compare(val1.(time.Time), val2.(time.Time)) return res >= 0, err }, @@ -134,9 +134,9 @@ var date_ge = framework.Function2{ var date_ge_timestamp = framework.Function2{ Name: "date_ge_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res >= 0, nil }, @@ -146,9 +146,9 @@ var date_ge_timestamp = framework.Function2{ var date_ge_timestamptz = framework.Function2{ Name: "date_ge_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res >= 0, nil }, @@ -158,9 +158,9 @@ var date_ge_timestamptz = framework.Function2{ var float4ge = framework.Function2{ Name: "float4ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float32.Compare(val1.(float32), val2.(float32)) return res >= 0, err }, @@ -170,9 +170,9 @@ var float4ge = framework.Function2{ var float48ge = framework.Function2{ Name: "float48ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(float64(val1.(float32)), val2.(float64)) return res >= 0, err }, @@ -182,9 +182,9 @@ var float48ge = framework.Function2{ var float84ge = framework.Function2{ Name: "float84ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), float64(val2.(float32))) return res >= 0, err }, @@ -194,9 +194,9 @@ var float84ge = framework.Function2{ var float8ge = framework.Function2{ Name: "float8ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), val2.(float64)) return res >= 0, err }, @@ -206,9 +206,9 @@ var float8ge = framework.Function2{ var int2ge = framework.Function2{ Name: "int2ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int16.Compare(val1.(int16), val2.(int16)) return res >= 0, err }, @@ -218,9 +218,9 @@ var int2ge = framework.Function2{ var int24ge = framework.Function2{ Name: "int24ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(int32(val1.(int16)), val2.(int32)) return res >= 0, err }, @@ -230,9 +230,9 @@ var int24ge = framework.Function2{ var int28ge = framework.Function2{ Name: "int28ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int16)), val2.(int64)) return res >= 0, err }, @@ -242,9 +242,9 @@ var int28ge = framework.Function2{ var int42ge = framework.Function2{ Name: "int42ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), int32(val2.(int16))) return res >= 0, err }, @@ -254,9 +254,9 @@ var int42ge = framework.Function2{ var int4ge = framework.Function2{ Name: "int4ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), val2.(int32)) return res >= 0, err }, @@ -266,9 +266,9 @@ var int4ge = framework.Function2{ var int48ge = framework.Function2{ Name: "int48ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int32)), val2.(int64)) return res >= 0, err }, @@ -278,9 +278,9 @@ var int48ge = framework.Function2{ var int82ge = framework.Function2{ Name: "int82ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int16))) return res >= 0, err }, @@ -290,9 +290,9 @@ var int82ge = framework.Function2{ var int84ge = framework.Function2{ Name: "int84ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int32))) return res >= 0, err }, @@ -302,9 +302,9 @@ var int84ge = framework.Function2{ var int8ge = framework.Function2{ Name: "int8ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), val2.(int64)) return res >= 0, err }, @@ -314,9 +314,9 @@ var int8ge = framework.Function2{ var interval_ge = framework.Function2{ Name: "interval_ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Interval.Compare(val1.(duration.Duration), val2.(duration.Duration)) return res >= 0, err }, @@ -326,9 +326,9 @@ var interval_ge = framework.Function2{ var jsonb_ge = framework.Function2{ Name: "jsonb_ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.JsonB.Compare(val1.(pgtypes.JsonDocument), val2.(pgtypes.JsonDocument)) return res >= 0, err }, @@ -338,9 +338,9 @@ var jsonb_ge = framework.Function2{ var namege = framework.Function2{ Name: "namege", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Name.Compare(val1.(string), val2.(string)) return res >= 0, err }, @@ -350,9 +350,9 @@ var namege = framework.Function2{ var namegetext = framework.Function2{ Name: "namegetext", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res >= 0, err }, @@ -362,9 +362,9 @@ var namegetext = framework.Function2{ var numeric_ge = framework.Function2{ Name: "numeric_ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Numeric.Compare(val1.(decimal.Decimal), val2.(decimal.Decimal)) return res >= 0, err }, @@ -374,9 +374,9 @@ var numeric_ge = framework.Function2{ var oidge = framework.Function2{ Name: "oidge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Oid.Compare(val1.(uint32), val2.(uint32)) return res >= 0, err }, @@ -386,9 +386,9 @@ var oidge = framework.Function2{ var textgename = framework.Function2{ Name: "textgename", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res >= 0, err }, @@ -398,9 +398,9 @@ var textgename = framework.Function2{ var text_ge = framework.Function2{ Name: "text_ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res >= 0, err }, @@ -410,9 +410,9 @@ var text_ge = framework.Function2{ var time_ge = framework.Function2{ Name: "time_ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Time.Compare(val1.(time.Time), val2.(time.Time)) return res >= 0, err }, @@ -422,9 +422,9 @@ var time_ge = framework.Function2{ var timestamp_ge_date = framework.Function2{ Name: "timestamp_ge_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res >= 0, nil }, @@ -434,9 +434,9 @@ var timestamp_ge_date = framework.Function2{ var timestamp_ge = framework.Function2{ Name: "timestamp_ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Timestamp.Compare(val1.(time.Time), val2.(time.Time)) return res >= 0, err }, @@ -446,9 +446,9 @@ var timestamp_ge = framework.Function2{ var timestamp_ge_timestamptz = framework.Function2{ Name: "timestamp_ge_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res >= 0, err }, @@ -458,9 +458,9 @@ var timestamp_ge_timestamptz = framework.Function2{ var timestamptz_ge_date = framework.Function2{ Name: "timestamptz_ge_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res >= 0, nil }, @@ -470,9 +470,9 @@ var timestamptz_ge_date = framework.Function2{ var timestamptz_ge_timestamp = framework.Function2{ Name: "timestamptz_ge_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res >= 0, err }, @@ -482,9 +482,9 @@ var timestamptz_ge_timestamp = framework.Function2{ var timestamptz_ge = framework.Function2{ Name: "timestamptz_ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res >= 0, err }, @@ -494,9 +494,9 @@ var timestamptz_ge = framework.Function2{ var timetz_ge = framework.Function2{ Name: "timetz_ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimeTZ.Compare(val1.(time.Time), val2.(time.Time)) return res >= 0, err }, @@ -506,9 +506,9 @@ var timetz_ge = framework.Function2{ var uuid_ge = framework.Function2{ Name: "uuid_ge", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Uuid.Compare(val1.(uuid.UUID), val2.(uuid.UUID)) return res >= 0, err }, diff --git a/server/functions/binary/json.go b/server/functions/binary/json.go index 2a51c7a8ac..6124525289 100644 --- a/server/functions/binary/json.go +++ b/server/functions/binary/json.go @@ -56,15 +56,15 @@ func initJSON() { var json_array_element = framework.Function2{ Name: "json_array_element", Return: pgtypes.Json, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Json, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Json, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: make a bespoke implementation that preserves whitespace newVal, err := pgtypes.JsonB.IoInput(ctx, val1.(string)) if err != nil { return nil, err } - var unusedTypes [3]pgtypes.DoltgresType + var unusedTypes [3]*pgtypes.DoltgresType retVal, err := jsonb_array_element.Callable(ctx, unusedTypes, newVal, val2) if err != nil { return nil, err @@ -80,9 +80,9 @@ var json_array_element = framework.Function2{ var jsonb_array_element = framework.Function2{ Name: "jsonb_array_element", Return: pgtypes.JsonB, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { array, ok := val1.(pgtypes.JsonDocument).Value.(pgtypes.JsonValueArray) if !ok { return nil, nil @@ -102,15 +102,15 @@ var jsonb_array_element = framework.Function2{ var json_object_field = framework.Function2{ Name: "json_object_field", Return: pgtypes.Json, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Json, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Json, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: make a bespoke implementation that preserves whitespace newVal, err := pgtypes.JsonB.IoInput(ctx, val1.(string)) if err != nil { return nil, err } - var unusedTypes [3]pgtypes.DoltgresType + var unusedTypes [3]*pgtypes.DoltgresType retVal, err := jsonb_object_field.Callable(ctx, unusedTypes, newVal, val2) if err != nil { return nil, err @@ -126,9 +126,9 @@ var json_object_field = framework.Function2{ var jsonb_object_field = framework.Function2{ Name: "jsonb_object_field", Return: pgtypes.JsonB, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { object, ok := val1.(pgtypes.JsonDocument).Value.(pgtypes.JsonValueObject) if !ok { return nil, nil @@ -145,15 +145,15 @@ var jsonb_object_field = framework.Function2{ var json_array_element_text = framework.Function2{ Name: "json_array_element_text", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Json, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Json, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: make a bespoke implementation that preserves whitespace newVal, err := pgtypes.JsonB.IoInput(ctx, val1.(string)) if err != nil { return nil, err } - var unusedTypes [3]pgtypes.DoltgresType + var unusedTypes [3]*pgtypes.DoltgresType return jsonb_array_element_text.Callable(ctx, unusedTypes, newVal, val2) }, } @@ -162,9 +162,9 @@ var json_array_element_text = framework.Function2{ var jsonb_array_element_text = framework.Function2{ Name: "jsonb_array_element_text", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, dt [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, dt [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { doc, err := jsonb_array_element.Callable(ctx, dt, val1, val2) if err != nil || doc == nil { return nil, err @@ -182,15 +182,15 @@ var jsonb_array_element_text = framework.Function2{ var json_object_field_text = framework.Function2{ Name: "json_object_field_text", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Json, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Json, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: make a bespoke implementation that preserves whitespace newVal, err := pgtypes.JsonB.IoInput(ctx, val1.(string)) if err != nil { return nil, err } - var unusedTypes [3]pgtypes.DoltgresType + var unusedTypes [3]*pgtypes.DoltgresType return jsonb_object_field_text.Callable(ctx, unusedTypes, newVal, val2) }, } @@ -199,9 +199,9 @@ var json_object_field_text = framework.Function2{ var jsonb_object_field_text = framework.Function2{ Name: "jsonb_object_field_text", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, dt [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, dt [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { doc, err := jsonb_object_field.Callable(ctx, dt, val1, val2) if err != nil || doc == nil { return nil, err @@ -219,15 +219,15 @@ var jsonb_object_field_text = framework.Function2{ var json_extract_path = framework.Function2{ Name: "json_extract_path", Return: pgtypes.Json, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Json, pgtypes.TextArray}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Json, pgtypes.TextArray}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: make a bespoke implementation that preserves whitespace newVal, err := pgtypes.JsonB.IoInput(ctx, val1.(string)) if err != nil { return nil, err } - var unusedTypes [3]pgtypes.DoltgresType + var unusedTypes [3]*pgtypes.DoltgresType retVal, err := jsonb_extract_path.Callable(ctx, unusedTypes, newVal, val2) if err != nil { return nil, err @@ -243,9 +243,9 @@ var json_extract_path = framework.Function2{ var jsonb_extract_path = framework.Function2{ Name: "jsonb_extract_path", Return: pgtypes.JsonB, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.TextArray}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.TextArray}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { value := val1.(pgtypes.JsonDocument).Value paths := val2.([]interface{}) for _, path := range paths { @@ -279,15 +279,15 @@ var jsonb_extract_path = framework.Function2{ var json_extract_path_text = framework.Function2{ Name: "json_extract_path_text", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Json, pgtypes.TextArray}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Json, pgtypes.TextArray}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: make a bespoke implementation that preserves whitespace newVal, err := pgtypes.JsonB.IoInput(ctx, val1.(string)) if err != nil { return nil, err } - var unusedTypes [3]pgtypes.DoltgresType + var unusedTypes [3]*pgtypes.DoltgresType return jsonb_extract_path_text.Callable(ctx, unusedTypes, newVal, val2) }, } @@ -296,9 +296,9 @@ var json_extract_path_text = framework.Function2{ var jsonb_extract_path_text = framework.Function2{ Name: "jsonb_extract_path_text", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.TextArray}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.TextArray}, Strict: true, - Callable: func(ctx *sql.Context, dt [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, dt [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { doc, err := jsonb_extract_path.Callable(ctx, dt, val1, val2) if err != nil || doc == nil { return nil, err @@ -316,9 +316,9 @@ var jsonb_extract_path_text = framework.Function2{ var jsonb_contains = framework.Function2{ Name: "jsonb_contains", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return nil, fmt.Errorf("JSON contains is not yet supported") }, } @@ -327,9 +327,9 @@ var jsonb_contains = framework.Function2{ var jsonb_contained = framework.Function2{ Name: "jsonb_contained", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, Strict: true, - Callable: func(ctx *sql.Context, dt [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, dt [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return jsonb_contains.Callable(ctx, dt, val2, val1) }, } @@ -338,9 +338,9 @@ var jsonb_contained = framework.Function2{ var jsonb_exists = framework.Function2{ Name: "jsonb_exists", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { switch value := val1.(pgtypes.JsonDocument).Value.(type) { case pgtypes.JsonValueObject: _, ok := value.Index[val2.(string)] @@ -366,9 +366,9 @@ var jsonb_exists = framework.Function2{ var jsonb_exists_any = framework.Function2{ Name: "jsonb_exists_any", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.TextArray}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.TextArray}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { keys := val2.([]interface{}) switch value := val1.(pgtypes.JsonDocument).Value.(type) { case pgtypes.JsonValueObject: @@ -406,9 +406,9 @@ var jsonb_exists_any = framework.Function2{ var jsonb_exists_all = framework.Function2{ Name: "jsonb_exists_all", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.TextArray}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.TextArray}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { keys := val2.([]interface{}) switch value := val1.(pgtypes.JsonDocument).Value.(type) { case pgtypes.JsonValueObject: @@ -451,9 +451,9 @@ var jsonb_exists_all = framework.Function2{ var jsonb_delete_text = framework.Function2{ Name: "jsonb_delete", Return: pgtypes.JsonB, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return nil, fmt.Errorf("JSON deletions are not yet supported") }, } @@ -462,9 +462,9 @@ var jsonb_delete_text = framework.Function2{ var jsonb_delete_text_array = framework.Function2{ Name: "jsonb_delete", Return: pgtypes.JsonB, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.TextArray}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.TextArray}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return nil, fmt.Errorf("JSON deletions are not yet supported") }, } @@ -473,9 +473,9 @@ var jsonb_delete_text_array = framework.Function2{ var jsonb_delete_int32 = framework.Function2{ Name: "jsonb_delete", Return: pgtypes.JsonB, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return nil, fmt.Errorf("JSON deletions are not yet supported") }, } diff --git a/server/functions/binary/less.go b/server/functions/binary/less.go index 8cd212eaf8..629108dc94 100644 --- a/server/functions/binary/less.go +++ b/server/functions/binary/less.go @@ -74,9 +74,9 @@ func initBinaryLessThan() { var boollt = framework.Function2{ Name: "boollt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bool.Compare(val1.(bool), val2.(bool)) return res == -1, err }, @@ -86,9 +86,9 @@ var boollt = framework.Function2{ var bpcharlt = framework.Function2{ Name: "bpcharlt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.BpChar.Compare(val1.(string), val2.(string)) return res == -1, err }, @@ -98,9 +98,9 @@ var bpcharlt = framework.Function2{ var bytealt = framework.Function2{ Name: "bytealt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bytea.Compare(val1.([]byte), val2.([]byte)) return res == -1, err }, @@ -110,9 +110,9 @@ var bytealt = framework.Function2{ var charlt = framework.Function2{ Name: "charlt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.InternalChar.Compare(val1.(string), val2.(string)) return res == -1, err }, @@ -122,9 +122,9 @@ var charlt = framework.Function2{ var date_lt = framework.Function2{ Name: "date_lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Date.Compare(val1.(time.Time), val2.(time.Time)) return res == -1, err }, @@ -134,9 +134,9 @@ var date_lt = framework.Function2{ var date_lt_timestamp = framework.Function2{ Name: "date_lt_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == -1, nil }, @@ -146,9 +146,9 @@ var date_lt_timestamp = framework.Function2{ var date_lt_timestamptz = framework.Function2{ Name: "date_lt_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == -1, nil }, @@ -158,9 +158,9 @@ var date_lt_timestamptz = framework.Function2{ var float4lt = framework.Function2{ Name: "float4lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float32.Compare(val1.(float32), val2.(float32)) return res == -1, err }, @@ -170,9 +170,9 @@ var float4lt = framework.Function2{ var float48lt = framework.Function2{ Name: "float48lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(float64(val1.(float32)), val2.(float64)) return res == -1, err }, @@ -182,9 +182,9 @@ var float48lt = framework.Function2{ var float84lt = framework.Function2{ Name: "float84lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), float64(val2.(float32))) return res == -1, err }, @@ -194,9 +194,9 @@ var float84lt = framework.Function2{ var float8lt = framework.Function2{ Name: "float8lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), val2.(float64)) return res == -1, err }, @@ -206,9 +206,9 @@ var float8lt = framework.Function2{ var int2lt = framework.Function2{ Name: "int2lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int16.Compare(val1.(int16), val2.(int16)) return res == -1, err }, @@ -218,9 +218,9 @@ var int2lt = framework.Function2{ var int24lt = framework.Function2{ Name: "int24lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(int32(val1.(int16)), val2.(int32)) return res == -1, err }, @@ -230,9 +230,9 @@ var int24lt = framework.Function2{ var int28lt = framework.Function2{ Name: "int28lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int16)), val2.(int64)) return res == -1, err }, @@ -242,9 +242,9 @@ var int28lt = framework.Function2{ var int42lt = framework.Function2{ Name: "int42lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), int32(val2.(int16))) return res == -1, err }, @@ -254,9 +254,9 @@ var int42lt = framework.Function2{ var int4lt = framework.Function2{ Name: "int4lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), val2.(int32)) return res == -1, err }, @@ -266,9 +266,9 @@ var int4lt = framework.Function2{ var int48lt = framework.Function2{ Name: "int48lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int32)), val2.(int64)) return res == -1, err }, @@ -278,9 +278,9 @@ var int48lt = framework.Function2{ var int82lt = framework.Function2{ Name: "int82lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int16))) return res == -1, err }, @@ -290,9 +290,9 @@ var int82lt = framework.Function2{ var int84lt = framework.Function2{ Name: "int84lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int32))) return res == -1, err }, @@ -302,9 +302,9 @@ var int84lt = framework.Function2{ var int8lt = framework.Function2{ Name: "int8lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), val2.(int64)) return res == -1, err }, @@ -314,9 +314,9 @@ var int8lt = framework.Function2{ var interval_lt = framework.Function2{ Name: "interval_lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Interval.Compare(val1.(duration.Duration), val2.(duration.Duration)) return res == -1, err }, @@ -326,9 +326,9 @@ var interval_lt = framework.Function2{ var jsonb_lt = framework.Function2{ Name: "jsonb_lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.JsonB.Compare(val1.(pgtypes.JsonDocument), val2.(pgtypes.JsonDocument)) return res == -1, err }, @@ -338,9 +338,9 @@ var jsonb_lt = framework.Function2{ var namelt = framework.Function2{ Name: "namelt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Name.Compare(val1.(string), val2.(string)) return res == -1, err }, @@ -350,9 +350,9 @@ var namelt = framework.Function2{ var namelttext = framework.Function2{ Name: "namelttext", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res == -1, err }, @@ -362,9 +362,9 @@ var namelttext = framework.Function2{ var numeric_lt = framework.Function2{ Name: "numeric_lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Numeric.Compare(val1.(decimal.Decimal), val2.(decimal.Decimal)) return res == -1, err }, @@ -374,9 +374,9 @@ var numeric_lt = framework.Function2{ var oidlt = framework.Function2{ Name: "oidlt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Oid.Compare(val1.(uint32), val2.(uint32)) return res == -1, err }, @@ -386,9 +386,9 @@ var oidlt = framework.Function2{ var textltname = framework.Function2{ Name: "textltname", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res == -1, err }, @@ -398,9 +398,9 @@ var textltname = framework.Function2{ var text_lt = framework.Function2{ Name: "text_lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res == -1, err }, @@ -410,9 +410,9 @@ var text_lt = framework.Function2{ var time_lt = framework.Function2{ Name: "time_lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Time.Compare(val1.(time.Time), val2.(time.Time)) return res == -1, err }, @@ -422,9 +422,9 @@ var time_lt = framework.Function2{ var timestamp_lt_date = framework.Function2{ Name: "timestamp_lt_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == -1, nil }, @@ -434,9 +434,9 @@ var timestamp_lt_date = framework.Function2{ var timestamp_lt = framework.Function2{ Name: "timestamp_lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Timestamp.Compare(val1.(time.Time), val2.(time.Time)) return res == -1, err }, @@ -446,9 +446,9 @@ var timestamp_lt = framework.Function2{ var timestamp_lt_timestamptz = framework.Function2{ Name: "timestamp_lt_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == -1, err }, @@ -458,9 +458,9 @@ var timestamp_lt_timestamptz = framework.Function2{ var timestamptz_lt_date = framework.Function2{ Name: "timestamptz_lt_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res == -1, nil }, @@ -470,9 +470,9 @@ var timestamptz_lt_date = framework.Function2{ var timestamptz_lt_timestamp = framework.Function2{ Name: "timestamptz_lt_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == -1, err }, @@ -482,9 +482,9 @@ var timestamptz_lt_timestamp = framework.Function2{ var timestamptz_lt = framework.Function2{ Name: "timestamptz_lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == -1, err }, @@ -494,9 +494,9 @@ var timestamptz_lt = framework.Function2{ var timetz_lt = framework.Function2{ Name: "timetz_lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimeTZ.Compare(val1.(time.Time), val2.(time.Time)) return res == -1, err }, @@ -506,9 +506,9 @@ var timetz_lt = framework.Function2{ var uuid_lt = framework.Function2{ Name: "uuid_lt", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Uuid.Compare(val1.(uuid.UUID), val2.(uuid.UUID)) return res == -1, err }, diff --git a/server/functions/binary/less_equal.go b/server/functions/binary/less_equal.go index 4ec1caaee7..95b3ee1f05 100644 --- a/server/functions/binary/less_equal.go +++ b/server/functions/binary/less_equal.go @@ -74,9 +74,9 @@ func initBinaryLessOrEqual() { var boolle = framework.Function2{ Name: "boolle", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bool.Compare(val1.(bool), val2.(bool)) return res <= 0, err }, @@ -86,9 +86,9 @@ var boolle = framework.Function2{ var bpcharle = framework.Function2{ Name: "bpcharle", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.BpChar.Compare(val1.(string), val2.(string)) return res <= 0, err }, @@ -98,9 +98,9 @@ var bpcharle = framework.Function2{ var byteale = framework.Function2{ Name: "byteale", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bytea.Compare(val1.([]byte), val2.([]byte)) return res <= 0, err }, @@ -110,9 +110,9 @@ var byteale = framework.Function2{ var charle = framework.Function2{ Name: "charle", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.InternalChar.Compare(val1.(string), val2.(string)) return res <= 0, err }, @@ -122,9 +122,9 @@ var charle = framework.Function2{ var date_le = framework.Function2{ Name: "date_le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Date.Compare(val1.(time.Time), val2.(time.Time)) return res <= 0, err }, @@ -134,9 +134,9 @@ var date_le = framework.Function2{ var date_le_timestamp = framework.Function2{ Name: "date_le_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res <= 0, nil }, @@ -146,9 +146,9 @@ var date_le_timestamp = framework.Function2{ var date_le_timestamptz = framework.Function2{ Name: "date_le_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res <= 0, nil }, @@ -158,9 +158,9 @@ var date_le_timestamptz = framework.Function2{ var float4le = framework.Function2{ Name: "float4le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float32.Compare(val1.(float32), val2.(float32)) return res <= 0, err }, @@ -170,9 +170,9 @@ var float4le = framework.Function2{ var float48le = framework.Function2{ Name: "float48le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(float64(val1.(float32)), val2.(float64)) return res <= 0, err }, @@ -182,9 +182,9 @@ var float48le = framework.Function2{ var float84le = framework.Function2{ Name: "float84le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), float64(val2.(float32))) return res <= 0, err }, @@ -194,9 +194,9 @@ var float84le = framework.Function2{ var float8le = framework.Function2{ Name: "float8le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), val2.(float64)) return res <= 0, err }, @@ -206,9 +206,9 @@ var float8le = framework.Function2{ var int2le = framework.Function2{ Name: "int2le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int16.Compare(val1.(int16), val2.(int16)) return res <= 0, err }, @@ -218,9 +218,9 @@ var int2le = framework.Function2{ var int24le = framework.Function2{ Name: "int24le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(int32(val1.(int16)), val2.(int32)) return res <= 0, err }, @@ -230,9 +230,9 @@ var int24le = framework.Function2{ var int28le = framework.Function2{ Name: "int28le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int16)), val2.(int64)) return res <= 0, err }, @@ -242,9 +242,9 @@ var int28le = framework.Function2{ var int42le = framework.Function2{ Name: "int42le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), int32(val2.(int16))) return res <= 0, err }, @@ -254,9 +254,9 @@ var int42le = framework.Function2{ var int4le = framework.Function2{ Name: "int4le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), val2.(int32)) return res <= 0, err }, @@ -266,9 +266,9 @@ var int4le = framework.Function2{ var int48le = framework.Function2{ Name: "int48le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int32)), val2.(int64)) return res <= 0, err }, @@ -278,9 +278,9 @@ var int48le = framework.Function2{ var int82le = framework.Function2{ Name: "int82le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int16))) return res <= 0, err }, @@ -290,9 +290,9 @@ var int82le = framework.Function2{ var int84le = framework.Function2{ Name: "int84le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int32))) return res <= 0, err }, @@ -302,9 +302,9 @@ var int84le = framework.Function2{ var int8le = framework.Function2{ Name: "int8le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), val2.(int64)) return res <= 0, err }, @@ -314,9 +314,9 @@ var int8le = framework.Function2{ var interval_le = framework.Function2{ Name: "interval_le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Interval.Compare(val1.(duration.Duration), val2.(duration.Duration)) return res <= 0, err }, @@ -326,9 +326,9 @@ var interval_le = framework.Function2{ var jsonb_le = framework.Function2{ Name: "jsonb_le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.JsonB.Compare(val1.(pgtypes.JsonDocument), val2.(pgtypes.JsonDocument)) return res <= 0, err }, @@ -338,9 +338,9 @@ var jsonb_le = framework.Function2{ var namele = framework.Function2{ Name: "namele", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Name.Compare(val1.(string), val2.(string)) return res <= 0, err }, @@ -350,9 +350,9 @@ var namele = framework.Function2{ var nameletext = framework.Function2{ Name: "nameletext", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res <= 0, err }, @@ -362,9 +362,9 @@ var nameletext = framework.Function2{ var numeric_le = framework.Function2{ Name: "numeric_le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Numeric.Compare(val1.(decimal.Decimal), val2.(decimal.Decimal)) return res <= 0, err }, @@ -374,9 +374,9 @@ var numeric_le = framework.Function2{ var oidle = framework.Function2{ Name: "oidle", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Oid.Compare(val1.(uint32), val2.(uint32)) return res <= 0, err }, @@ -386,9 +386,9 @@ var oidle = framework.Function2{ var textlename = framework.Function2{ Name: "textlename", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res <= 0, err }, @@ -398,9 +398,9 @@ var textlename = framework.Function2{ var text_le = framework.Function2{ Name: "text_le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res <= 0, err }, @@ -410,9 +410,9 @@ var text_le = framework.Function2{ var time_le = framework.Function2{ Name: "time_le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Time.Compare(val1.(time.Time), val2.(time.Time)) return res <= 0, err }, @@ -422,9 +422,9 @@ var time_le = framework.Function2{ var timestamp_le_date = framework.Function2{ Name: "timestamp_le_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res <= 0, nil }, @@ -434,9 +434,9 @@ var timestamp_le_date = framework.Function2{ var timestamp_le = framework.Function2{ Name: "timestamp_le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Timestamp.Compare(val1.(time.Time), val2.(time.Time)) return res <= 0, err }, @@ -446,9 +446,9 @@ var timestamp_le = framework.Function2{ var timestamp_le_timestamptz = framework.Function2{ Name: "timestamp_le_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res <= 0, err }, @@ -458,9 +458,9 @@ var timestamp_le_timestamptz = framework.Function2{ var timestamptz_le_date = framework.Function2{ Name: "timestamptz_le_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res <= 0, nil }, @@ -470,9 +470,9 @@ var timestamptz_le_date = framework.Function2{ var timestamptz_le_timestamp = framework.Function2{ Name: "timestamptz_le_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res <= 0, err }, @@ -482,9 +482,9 @@ var timestamptz_le_timestamp = framework.Function2{ var timestamptz_le = framework.Function2{ Name: "timestamptz_le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res <= 0, err }, @@ -494,9 +494,9 @@ var timestamptz_le = framework.Function2{ var timetz_le = framework.Function2{ Name: "timetz_le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimeTZ.Compare(val1.(time.Time), val2.(time.Time)) return res <= 0, err }, @@ -506,9 +506,9 @@ var timetz_le = framework.Function2{ var uuid_le = framework.Function2{ Name: "uuid_le", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Uuid.Compare(val1.(uuid.UUID), val2.(uuid.UUID)) return res <= 0, err }, diff --git a/server/functions/binary/minus.go b/server/functions/binary/minus.go index 7fa9f3b217..501df88157 100644 --- a/server/functions/binary/minus.go +++ b/server/functions/binary/minus.go @@ -52,9 +52,9 @@ func initBinaryMinus() { var float4mi = framework.Function2{ Name: "float4mi", Return: pgtypes.Float32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(float32) - val2.(float32), nil }, } @@ -63,9 +63,9 @@ var float4mi = framework.Function2{ var float48mi = framework.Function2{ Name: "float48mi", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return float64(val1.(float32)) - val2.(float64), nil }, } @@ -74,9 +74,9 @@ var float48mi = framework.Function2{ var float8mi = framework.Function2{ Name: "float8mi", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(float64) - val2.(float64), nil }, } @@ -85,9 +85,9 @@ var float8mi = framework.Function2{ var float84mi = framework.Function2{ Name: "float84mi", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(float64) - float64(val2.(float32)), nil }, } @@ -96,9 +96,9 @@ var float84mi = framework.Function2{ var int2mi = framework.Function2{ Name: "int2mi", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int16)) - int64(val2.(int16)) if result > math.MaxInt16 || result < math.MinInt16 { return nil, fmt.Errorf("smallint out of range") @@ -111,9 +111,9 @@ var int2mi = framework.Function2{ var int24mi = framework.Function2{ Name: "int24mi", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int16)) - int64(val2.(int32)) if result > math.MaxInt16 || result < math.MinInt16 { return nil, fmt.Errorf("integer out of range") @@ -126,9 +126,9 @@ var int24mi = framework.Function2{ var int28mi = framework.Function2{ Name: "int28mi", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return minusOverflow(int64(val1.(int16)), val2.(int64)) }, } @@ -137,9 +137,9 @@ var int28mi = framework.Function2{ var int4mi = framework.Function2{ Name: "int4mi", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int32)) - int64(val2.(int32)) if result > math.MaxInt32 || result < math.MinInt32 { return nil, fmt.Errorf("integer out of range") @@ -152,9 +152,9 @@ var int4mi = framework.Function2{ var int42mi = framework.Function2{ Name: "int42mi", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int32)) - int64(val2.(int16)) if result > math.MaxInt32 || result < math.MinInt32 { return nil, fmt.Errorf("integer out of range") @@ -167,9 +167,9 @@ var int42mi = framework.Function2{ var int48mi = framework.Function2{ Name: "int48mi", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return minusOverflow(int64(val1.(int32)), val2.(int64)) }, } @@ -178,9 +178,9 @@ var int48mi = framework.Function2{ var int8mi = framework.Function2{ Name: "int8mi", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return minusOverflow(val1.(int64), val2.(int64)) }, } @@ -189,9 +189,9 @@ var int8mi = framework.Function2{ var int82mi = framework.Function2{ Name: "int82mi", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return minusOverflow(val1.(int64), int64(val2.(int16))) }, } @@ -200,9 +200,9 @@ var int82mi = framework.Function2{ var int84mi = framework.Function2{ Name: "int84mi", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return minusOverflow(val1.(int64), int64(val2.(int32))) }, } @@ -211,9 +211,9 @@ var int84mi = framework.Function2{ var interval_mi = framework.Function2{ Name: "interval_mi", Return: pgtypes.Interval, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { dur1 := val1.(duration.Duration) dur2 := val2.(duration.Duration) return dur1.Sub(dur2), nil @@ -224,9 +224,9 @@ var interval_mi = framework.Function2{ var numeric_sub = framework.Function2{ Name: "numeric_sub", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(decimal.Decimal).Sub(val2.(decimal.Decimal)), nil }, } diff --git a/server/functions/binary/mod.go b/server/functions/binary/mod.go index 55e9a75a52..da61f5b09f 100644 --- a/server/functions/binary/mod.go +++ b/server/functions/binary/mod.go @@ -39,9 +39,9 @@ func initBinaryMod() { var int2mod = framework.Function2{ Name: "int2mod", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int16) == 0 { return nil, fmt.Errorf("division by zero") } @@ -53,9 +53,9 @@ var int2mod = framework.Function2{ var int4mod = framework.Function2{ Name: "int4mod", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int32) == 0 { return nil, fmt.Errorf("division by zero") } @@ -67,9 +67,9 @@ var int4mod = framework.Function2{ var int8mod = framework.Function2{ Name: "int8mod", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int64) == 0 { return nil, fmt.Errorf("division by zero") } @@ -81,9 +81,9 @@ var int8mod = framework.Function2{ var numeric_mod = framework.Function2{ Name: "numeric_mod", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(decimal.Decimal).Equal(decimal.Zero) { return nil, fmt.Errorf("division by zero") } diff --git a/server/functions/binary/multiply.go b/server/functions/binary/multiply.go index 8f5381eca9..e9b21d0864 100644 --- a/server/functions/binary/multiply.go +++ b/server/functions/binary/multiply.go @@ -52,9 +52,9 @@ func initBinaryMultiply() { var float4mul = framework.Function2{ Name: "float4mul", Return: pgtypes.Float32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(float32) * val2.(float32), nil }, } @@ -63,9 +63,9 @@ var float4mul = framework.Function2{ var float48mul = framework.Function2{ Name: "float48mul", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return float64(val1.(float32)) * val2.(float64), nil }, } @@ -74,9 +74,9 @@ var float48mul = framework.Function2{ var float8mul = framework.Function2{ Name: "float8mul", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(float64) * val2.(float64), nil }, } @@ -85,9 +85,9 @@ var float8mul = framework.Function2{ var float84mul = framework.Function2{ Name: "float84mul", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(float64) * float64(val2.(float32)), nil }, } @@ -96,9 +96,9 @@ var float84mul = framework.Function2{ var int2mul = framework.Function2{ Name: "int2mul", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int16)) * int64(val2.(int16)) if result > math.MaxInt16 || result < math.MinInt16 { return nil, fmt.Errorf("smallint out of range") @@ -111,9 +111,9 @@ var int2mul = framework.Function2{ var int24mul = framework.Function2{ Name: "int24mul", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int16)) * int64(val2.(int32)) if result > math.MaxInt16 || result < math.MinInt16 { return nil, fmt.Errorf("integer out of range") @@ -126,9 +126,9 @@ var int24mul = framework.Function2{ var int28mul = framework.Function2{ Name: "int28mul", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return multiplyOverflow(int64(val1.(int16)), val2.(int64)) }, } @@ -137,9 +137,9 @@ var int28mul = framework.Function2{ var int4mul = framework.Function2{ Name: "int4mul", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int32)) * int64(val2.(int32)) if result > math.MaxInt32 || result < math.MinInt32 { return nil, fmt.Errorf("integer out of range") @@ -152,9 +152,9 @@ var int4mul = framework.Function2{ var int42mul = framework.Function2{ Name: "int42mul", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int32)) * int64(val2.(int16)) if result > math.MaxInt32 || result < math.MinInt32 { return nil, fmt.Errorf("integer out of range") @@ -167,9 +167,9 @@ var int42mul = framework.Function2{ var int48mul = framework.Function2{ Name: "int48mul", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return multiplyOverflow(int64(val1.(int32)), val2.(int64)) }, } @@ -178,9 +178,9 @@ var int48mul = framework.Function2{ var int8mul = framework.Function2{ Name: "int8mul", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return multiplyOverflow(val1.(int64), val2.(int64)) }, } @@ -189,9 +189,9 @@ var int8mul = framework.Function2{ var int82mul = framework.Function2{ Name: "int82mul", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return multiplyOverflow(val1.(int64), int64(val2.(int16))) }, } @@ -200,9 +200,9 @@ var int82mul = framework.Function2{ var int84mul = framework.Function2{ Name: "int84mul", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return multiplyOverflow(val1.(int64), int64(val2.(int32))) }, } @@ -211,9 +211,9 @@ var int84mul = framework.Function2{ var interval_mul = framework.Function2{ Name: "interval_mul", Return: pgtypes.Interval, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(duration.Duration).MulFloat(val2.(float64)), nil }, } @@ -222,9 +222,9 @@ var interval_mul = framework.Function2{ var numeric_mul = framework.Function2{ Name: "numeric_mul", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(decimal.Decimal).Mul(val2.(decimal.Decimal)), nil }, } diff --git a/server/functions/binary/not_equal.go b/server/functions/binary/not_equal.go index 9f54b82b40..9befee597c 100644 --- a/server/functions/binary/not_equal.go +++ b/server/functions/binary/not_equal.go @@ -76,9 +76,9 @@ func initBinaryNotEqual() { var boolne = framework.Function2{ Name: "boolne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bool.Compare(val1.(bool), val2.(bool)) return res != 0, err }, @@ -88,9 +88,9 @@ var boolne = framework.Function2{ var bpcharne = framework.Function2{ Name: "bpcharne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.BpChar.Compare(val1.(string), val2.(string)) return res != 0, err }, @@ -100,9 +100,9 @@ var bpcharne = framework.Function2{ var byteane = framework.Function2{ Name: "byteane", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Bytea.Compare(val1.([]byte), val2.([]byte)) return res != 0, err }, @@ -112,9 +112,9 @@ var byteane = framework.Function2{ var charne = framework.Function2{ Name: "charne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.InternalChar.Compare(val1.(string), val2.(string)) return res != 0, err }, @@ -124,9 +124,9 @@ var charne = framework.Function2{ var date_ne = framework.Function2{ Name: "date_ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Date.Compare(val1.(time.Time), val2.(time.Time)) return res != 0, err }, @@ -136,9 +136,9 @@ var date_ne = framework.Function2{ var date_ne_timestamp = framework.Function2{ Name: "date_ne_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res != 0, nil }, @@ -148,9 +148,9 @@ var date_ne_timestamp = framework.Function2{ var date_ne_timestamptz = framework.Function2{ Name: "date_ne_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res != 0, nil }, @@ -160,9 +160,9 @@ var date_ne_timestamptz = framework.Function2{ var float4ne = framework.Function2{ Name: "float4ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float32.Compare(val1.(float32), val2.(float32)) return res != 0, err }, @@ -172,9 +172,9 @@ var float4ne = framework.Function2{ var float48ne = framework.Function2{ Name: "float48ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(float64(val1.(float32)), val2.(float64)) return res != 0, err }, @@ -184,9 +184,9 @@ var float48ne = framework.Function2{ var float84ne = framework.Function2{ Name: "float84ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), float64(val2.(float32))) return res != 0, err }, @@ -196,9 +196,9 @@ var float84ne = framework.Function2{ var float8ne = framework.Function2{ Name: "float8ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Float64.Compare(val1.(float64), val2.(float64)) return res != 0, err }, @@ -208,9 +208,9 @@ var float8ne = framework.Function2{ var int2ne = framework.Function2{ Name: "int2ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int16.Compare(val1.(int16), val2.(int16)) return res != 0, err }, @@ -220,9 +220,9 @@ var int2ne = framework.Function2{ var int24ne = framework.Function2{ Name: "int24ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(int32(val1.(int16)), val2.(int32)) return res != 0, err }, @@ -232,9 +232,9 @@ var int24ne = framework.Function2{ var int28ne = framework.Function2{ Name: "int28ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int16)), val2.(int64)) return res != 0, err }, @@ -244,9 +244,9 @@ var int28ne = framework.Function2{ var int42ne = framework.Function2{ Name: "int42ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), int32(val2.(int16))) return res != 0, err }, @@ -256,9 +256,9 @@ var int42ne = framework.Function2{ var int4ne = framework.Function2{ Name: "int4ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int32.Compare(val1.(int32), val2.(int32)) return res != 0, err }, @@ -268,9 +268,9 @@ var int4ne = framework.Function2{ var int48ne = framework.Function2{ Name: "int48ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(int64(val1.(int32)), val2.(int64)) return res != 0, err }, @@ -280,9 +280,9 @@ var int48ne = framework.Function2{ var int82ne = framework.Function2{ Name: "int82ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int16))) return res != 0, err }, @@ -292,9 +292,9 @@ var int82ne = framework.Function2{ var int84ne = framework.Function2{ Name: "int84ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), int64(val2.(int32))) return res != 0, err }, @@ -304,9 +304,9 @@ var int84ne = framework.Function2{ var int8ne = framework.Function2{ Name: "int8ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Int64.Compare(val1.(int64), val2.(int64)) return res != 0, err }, @@ -316,9 +316,9 @@ var int8ne = framework.Function2{ var interval_ne = framework.Function2{ Name: "interval_ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Interval.Compare(val1.(duration.Duration), val2.(duration.Duration)) return res != 0, err }, @@ -328,9 +328,9 @@ var interval_ne = framework.Function2{ var jsonb_ne = framework.Function2{ Name: "jsonb_ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.JsonB.Compare(val1.(pgtypes.JsonDocument), val2.(pgtypes.JsonDocument)) return res != 0, err }, @@ -340,9 +340,9 @@ var jsonb_ne = framework.Function2{ var namene = framework.Function2{ Name: "namene", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Name.Compare(val1.(string), val2.(string)) return res != 0, err }, @@ -352,9 +352,9 @@ var namene = framework.Function2{ var namenetext = framework.Function2{ Name: "namenetext", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res != 0, err }, @@ -364,9 +364,9 @@ var namenetext = framework.Function2{ var numeric_ne = framework.Function2{ Name: "numeric_ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Numeric.Compare(val1.(decimal.Decimal), val2.(decimal.Decimal)) return res != 0, err }, @@ -376,9 +376,9 @@ var numeric_ne = framework.Function2{ var oidne = framework.Function2{ Name: "oidne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Oid.Compare(val1.(uint32), val2.(uint32)) return res != 0, err }, @@ -388,9 +388,9 @@ var oidne = framework.Function2{ var textnename = framework.Function2{ Name: "textnename", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Name}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res != 0, err }, @@ -400,9 +400,9 @@ var textnename = framework.Function2{ var text_ne = framework.Function2{ Name: "text_ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Text.Compare(val1.(string), val2.(string)) return res != 0, err }, @@ -412,9 +412,9 @@ var text_ne = framework.Function2{ var time_ne = framework.Function2{ Name: "time_ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Time.Compare(val1.(time.Time), val2.(time.Time)) return res != 0, err }, @@ -424,9 +424,9 @@ var time_ne = framework.Function2{ var timestamp_ne_date = framework.Function2{ Name: "timestamp_ne_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res != 0, nil }, @@ -436,9 +436,9 @@ var timestamp_ne_date = framework.Function2{ var timestamp_ne = framework.Function2{ Name: "timestamp_ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Timestamp.Compare(val1.(time.Time), val2.(time.Time)) return res != 0, err }, @@ -448,9 +448,9 @@ var timestamp_ne = framework.Function2{ var timestamp_ne_timestamptz = framework.Function2{ Name: "timestamp_ne_timestamptz", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res != 0, err }, @@ -460,9 +460,9 @@ var timestamp_ne_timestamptz = framework.Function2{ var timestamptz_ne_date = framework.Function2{ Name: "timestamptz_ne_date", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res := val1.(time.Time).Compare(val2.(time.Time)) return res != 0, nil }, @@ -472,9 +472,9 @@ var timestamptz_ne_date = framework.Function2{ var timestamptz_ne_timestamp = framework.Function2{ Name: "timestamptz_ne_timestamp", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res != 0, err }, @@ -484,9 +484,9 @@ var timestamptz_ne_timestamp = framework.Function2{ var timestamptz_ne = framework.Function2{ Name: "timestamptz_ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimestampTZ.Compare(val1.(time.Time), val2.(time.Time)) return res != 0, err }, @@ -496,9 +496,9 @@ var timestamptz_ne = framework.Function2{ var timetz_ne = framework.Function2{ Name: "timetz_ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.TimeTZ.Compare(val1.(time.Time), val2.(time.Time)) return res != 0, err }, @@ -508,9 +508,9 @@ var timetz_ne = framework.Function2{ var uuid_ne = framework.Function2{ Name: "uuid_ne", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Uuid.Compare(val1.(uuid.UUID), val2.(uuid.UUID)) return res != 0, err }, @@ -520,9 +520,9 @@ var uuid_ne = framework.Function2{ var xidneqint4 = framework.Function2{ Name: "xidneqint4", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Xid, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Xid, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: investigate the edge cases res, err := pgtypes.Int64.Compare(int64(val1.(uint32)), int64(val2.(int32))) return res != 0, err @@ -533,9 +533,9 @@ var xidneqint4 = framework.Function2{ var xidneq = framework.Function2{ Name: "xidneq", Return: pgtypes.Bool, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Xid, pgtypes.Xid}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Xid, pgtypes.Xid}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { res, err := pgtypes.Xid.Compare(val1.(uint32), val2.(uint32)) return res != 0, err }, diff --git a/server/functions/binary/plus.go b/server/functions/binary/plus.go index c354539f45..b2b8a58ca1 100644 --- a/server/functions/binary/plus.go +++ b/server/functions/binary/plus.go @@ -59,9 +59,9 @@ func initBinaryPlus() { var float4pl = framework.Function2{ Name: "float4pl", Return: pgtypes.Float32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(float32) + val2.(float32), nil }, } @@ -70,9 +70,9 @@ var float4pl = framework.Function2{ var float48pl = framework.Function2{ Name: "float48pl", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return float64(val1.(float32)) + val2.(float64), nil }, } @@ -81,9 +81,9 @@ var float48pl = framework.Function2{ var float8pl = framework.Function2{ Name: "float8pl", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(float64) + val2.(float64), nil }, } @@ -92,9 +92,9 @@ var float8pl = framework.Function2{ var float84pl = framework.Function2{ Name: "float84pl", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(float64) + float64(val2.(float32)), nil }, } @@ -103,9 +103,9 @@ var float84pl = framework.Function2{ var int2pl = framework.Function2{ Name: "int2pl", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int16)) + int64(val2.(int16)) if result > math.MaxInt16 || result < math.MinInt16 { return nil, fmt.Errorf("smallint out of range") @@ -118,9 +118,9 @@ var int2pl = framework.Function2{ var int24pl = framework.Function2{ Name: "int24pl", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int16)) + int64(val2.(int32)) if result > math.MaxInt16 || result < math.MinInt16 { return nil, fmt.Errorf("integer out of range") @@ -133,9 +133,9 @@ var int24pl = framework.Function2{ var int28pl = framework.Function2{ Name: "int28pl", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return plusOverflow(int64(val1.(int16)), val2.(int64)) }, } @@ -144,9 +144,9 @@ var int28pl = framework.Function2{ var int4pl = framework.Function2{ Name: "int4pl", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int32)) + int64(val2.(int32)) if result > math.MaxInt32 || result < math.MinInt32 { return nil, fmt.Errorf("integer out of range") @@ -159,9 +159,9 @@ var int4pl = framework.Function2{ var int42pl = framework.Function2{ Name: "int42pl", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { result := int64(val1.(int32)) + int64(val2.(int16)) if result > math.MaxInt32 || result < math.MinInt32 { return nil, fmt.Errorf("integer out of range") @@ -174,9 +174,9 @@ var int42pl = framework.Function2{ var int48pl = framework.Function2{ Name: "int48pl", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return plusOverflow(int64(val1.(int32)), val2.(int64)) }, } @@ -185,9 +185,9 @@ var int48pl = framework.Function2{ var int8pl = framework.Function2{ Name: "int8pl", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return plusOverflow(val1.(int64), val2.(int64)) }, } @@ -196,9 +196,9 @@ var int8pl = framework.Function2{ var int82pl = framework.Function2{ Name: "int82pl", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return plusOverflow(val1.(int64), int64(val2.(int16))) }, } @@ -207,9 +207,9 @@ var int82pl = framework.Function2{ var int84pl = framework.Function2{ Name: "int84pl", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return plusOverflow(val1.(int64), int64(val2.(int32))) }, } @@ -218,9 +218,9 @@ var int84pl = framework.Function2{ var interval_pl = framework.Function2{ Name: "interval_pl", Return: pgtypes.Interval, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { dur1 := val1.(duration.Duration) dur2 := val2.(duration.Duration) return dur1.Add(dur2), nil @@ -231,9 +231,9 @@ var interval_pl = framework.Function2{ var interval_pl_time = framework.Function2{ Name: "interval_pl_time", Return: pgtypes.Time, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Time}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Time}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val2.(time.Time).Add(time.Duration(val1.(duration.Duration).Nanos())), nil }, } @@ -242,9 +242,9 @@ var interval_pl_time = framework.Function2{ var interval_pl_date = framework.Function2{ Name: "interval_pl_date", Return: pgtypes.Timestamp, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Date}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return intervalPlusNonInterval(val1.(duration.Duration), val2.(time.Time)) }, } @@ -253,9 +253,9 @@ var interval_pl_date = framework.Function2{ var interval_pl_timetz = framework.Function2{ Name: "interval_pl_timetz", Return: pgtypes.TimeTZ, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.TimeTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.TimeTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { ttz := val2.(time.Time) return ttz.Add(time.Duration(val1.(duration.Duration).Nanos())), nil }, @@ -265,9 +265,9 @@ var interval_pl_timetz = framework.Function2{ var interval_pl_timestamp = framework.Function2{ Name: "interval_pl_timestamp", Return: pgtypes.Timestamp, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Timestamp}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return intervalPlusNonInterval(val1.(duration.Duration), val2.(time.Time)) }, } @@ -276,9 +276,9 @@ var interval_pl_timestamp = framework.Function2{ var interval_pl_timestamptz = framework.Function2{ Name: "interval_pl_timestamptz", Return: pgtypes.TimestampTZ, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.TimestampTZ}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return intervalPlusNonInterval(val1.(duration.Duration), val2.(time.Time)) }, } @@ -287,9 +287,9 @@ var interval_pl_timestamptz = framework.Function2{ var numeric_add = framework.Function2{ Name: "numeric_add", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(decimal.Decimal).Add(val2.(decimal.Decimal)), nil }, } diff --git a/server/functions/binary/shift_left.go b/server/functions/binary/shift_left.go index 5f17a573b1..02299d8847 100644 --- a/server/functions/binary/shift_left.go +++ b/server/functions/binary/shift_left.go @@ -35,9 +35,9 @@ func initBinaryShiftLeft() { var int2shl = framework.Function2{ Name: "int2shl", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int16(int32(val1.(int16)) << val2.(int32)), nil }, } @@ -46,9 +46,9 @@ var int2shl = framework.Function2{ var int4shl = framework.Function2{ Name: "int4shl", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int32(val1.(int32) << val2.(int32)), nil }, } @@ -57,9 +57,9 @@ var int4shl = framework.Function2{ var int8shl = framework.Function2{ Name: "int8shl", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int64(val1.(int64) << int64(val2.(int32))), nil }, } diff --git a/server/functions/binary/shift_right.go b/server/functions/binary/shift_right.go index dbc4a055bc..6bafae7e19 100644 --- a/server/functions/binary/shift_right.go +++ b/server/functions/binary/shift_right.go @@ -35,9 +35,9 @@ func initBinaryShiftRight() { var int2shr = framework.Function2{ Name: "int2shr", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int16(int32(val1.(int16)) >> val2.(int32)), nil }, } @@ -46,9 +46,9 @@ var int2shr = framework.Function2{ var int4shr = framework.Function2{ Name: "int4shr", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int32(val1.(int32) >> val2.(int32)), nil }, } @@ -57,9 +57,9 @@ var int4shr = framework.Function2{ var int8shr = framework.Function2{ Name: "int8shr", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return int64(val1.(int64) >> int64(val2.(int32))), nil }, } diff --git a/server/functions/bit_length.go b/server/functions/bit_length.go index b86dfdb9c6..e34e44b8cf 100644 --- a/server/functions/bit_length.go +++ b/server/functions/bit_length.go @@ -30,9 +30,9 @@ func initBitLength() { var bit_length_text = framework.Function1{ Name: "bit_length", Return: pgtypes.Int32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, t [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, t [2]*pgtypes.DoltgresType, val1 any) (any, error) { result, err := octet_length_text.Callable(ctx, t, val1) if err != nil { return nil, err diff --git a/server/functions/bool.go b/server/functions/bool.go new file mode 100644 index 0000000000..8572498493 --- /dev/null +++ b/server/functions/bool.go @@ -0,0 +1,115 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initBool registers the functions to the catalog. +func initBool() { + framework.RegisterFunction(boolin) + framework.RegisterFunction(boolout) + framework.RegisterFunction(boolrecv) + framework.RegisterFunction(boolsend) + framework.RegisterFunction(btboolcmp) +} + +// boolin represents the PostgreSQL function of boolean type IO input. +var boolin = framework.Function1{ + Name: "boolin", + Return: pgtypes.Bool, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + val = strings.TrimSpace(strings.ToLower(val.(string))) + if val == "true" || val == "t" || val == "yes" || val == "on" || val == "1" { + return true, nil + } else if val == "false" || val == "f" || val == "no" || val == "off" || val == "0" { + return false, nil + } else { + return nil, pgtypes.ErrInvalidSyntaxForType.New("boolean", val) + } + }, +} + +// boolout represents the PostgreSQL function of boolean type IO output. +var boolout = framework.Function1{ + Name: "boolout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Bool}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + if val.(bool) { + return "true", nil + } else { + return "false", nil + } + }, +} + +// boolrecv represents the PostgreSQL function of boolean type IO receive. +var boolrecv = framework.Function1{ + Name: "boolrecv", + Return: pgtypes.Bool, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return data[0] != 0, nil + }, +} + +// boolsend represents the PostgreSQL function of boolean type IO send. +var boolsend = framework.Function1{ + Name: "boolsend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Bool}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + if val.(bool) { + return []byte{1}, nil + } else { + return []byte{0}, nil + } + }, +} + +// btboolcmp represents the PostgreSQL function of boolean type byte compare. +var btboolcmp = framework.Function2{ + Name: "btboolcmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bool, pgtypes.Bool}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(bool) + bb := val2.(bool) + if ab == bb { + return int32(0), nil + } else if !ab { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} diff --git a/server/functions/bpchar.go b/server/functions/bpchar.go new file mode 100644 index 0000000000..2a3952b225 --- /dev/null +++ b/server/functions/bpchar.go @@ -0,0 +1,191 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "bytes" + "fmt" + "strconv" + "strings" + "unicode/utf8" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" + "github.com/dolthub/doltgresql/utils" +) + +// initBpChar registers the functions to the catalog. +func initBpChar() { + framework.RegisterFunction(bpcharin) + framework.RegisterFunction(bpcharout) + framework.RegisterFunction(bpcharrecv) + framework.RegisterFunction(bpcharsend) + framework.RegisterFunction(bpchartypmodin) + framework.RegisterFunction(bpchartypmodout) + framework.RegisterFunction(bpcharcmp) +} + +// bpcharin represents the PostgreSQL function of bpchar type IO input. +var bpcharin = framework.Function3{ + Name: "bpcharin", + Return: pgtypes.BpChar, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Cstring, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + input := val1.(string) + typmod := val3.(int32) + maxChars := int32(pgtypes.StringMaxLength) + if typmod != -1 { + maxChars = pgtypes.GetCharLengthFromTypmod(typmod) + if maxChars < pgtypes.StringUnbounded { + maxChars = pgtypes.StringMaxLength + } + } + input, runeLength := truncateString(input, maxChars) + if runeLength > maxChars { + return input, fmt.Errorf("value too long for type varying(%v)", maxChars) + } else if runeLength < maxChars { + return input + strings.Repeat(" ", int(maxChars-runeLength)), nil + } else { + return input, nil + } + }, +} + +// bpcharout represents the PostgreSQL function of bpchar type IO output. +var bpcharout = framework.Function1{ + Name: "bpcharout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.BpChar}, + Strict: true, + Callable: func(ctx *sql.Context, t [2]*pgtypes.DoltgresType, val any) (any, error) { + typ := t[0] + tm := typ.GetAttTypMod() + if tm == -1 { + return val.(string), nil + } + maxChars := pgtypes.GetCharLengthFromTypmod(tm) + if maxChars < 1 { + return val.(string), nil + } else { + str, runeCount := truncateString(val.(string), maxChars) + if runeCount < maxChars { + return str + strings.Repeat(" ", int(maxChars-runeCount)), nil + } + return str, nil + } + }, +} + +// bpcharrecv represents the PostgreSQL function of bpchar type IO receive. +var bpcharrecv = framework.Function3{ + Name: "bpcharrecv", + Return: pgtypes.BpChar, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Internal, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + data := val1.([]byte) + if len(data) == 0 { + return nil, nil + } + // TODO: use typmod? + reader := utils.NewReader(data) + return reader.String(), nil + }, +} + +// bpcharsend represents the PostgreSQL function of bpchar type IO send. +var bpcharsend = framework.Function1{ + Name: "bpcharsend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.BpChar}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + str := val.(string) + writer := utils.NewWriter(uint64(len(str) + 4)) + writer.String(str) + return writer.Data(), nil + }, +} + +// bpchartypmodin represents the PostgreSQL function of bpchar type IO typmod input. +var bpchartypmodin = framework.Function1{ + Name: "bpchartypmodin", + Return: pgtypes.Int32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.CstringArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return getTypModFromStringArr("char", val.([]any)) + }, +} + +// bpchartypmodout represents the PostgreSQL function of bpchar type IO typmod output. +var bpchartypmodout = framework.Function1{ + Name: "bpchartypmodout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + typmod := val.(int32) + if typmod < 5 { + return "", nil + } + maxChars := pgtypes.GetCharLengthFromTypmod(typmod) + return fmt.Sprintf("(%v)", maxChars), nil + }, +} + +// bpcharcmp represents the PostgreSQL function of bpchar type compare. +var bpcharcmp = framework.Function2{ + Name: "bpcharcmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.BpChar, pgtypes.BpChar}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + return int32(bytes.Compare([]byte(val1.(string)), []byte(val2.(string)))), nil + }, +} + +// truncateString returns a string that has been truncated to the given length. Uses the rune count rather than the +// byte count. Returns the input string if it's smaller than the length. Also returns the rune count of the string. +func truncateString(val string, runeLimit int32) (string, int32) { + runeLength := int32(utf8.RuneCountInString(val)) + if runeLength > runeLimit { + // TODO: figure out if there's a faster way to truncate based on rune count + startString := val + for i := int32(0); i < runeLimit; i++ { + _, size := utf8.DecodeRuneInString(val) + val = val[size:] + } + return startString[:len(startString)-len(val)], runeLength + } + return val, runeLength +} + +func getTypModFromStringArr(typName string, inputArr []any) (int32, error) { + if len(inputArr) == 0 { + return 0, pgtypes.ErrTypmodArrayMustBe1D.New() + } else if len(inputArr) > 1 { + return 0, fmt.Errorf("invalid type modifier") + } + + l, err := strconv.ParseInt(inputArr[0].(string), 10, 32) + if err != nil { + return 0, err + } + return pgtypes.GetTypModFromCharLength(typName, int32(l)) +} diff --git a/server/functions/btrim.go b/server/functions/btrim.go index 15711c49cf..468913760a 100644 --- a/server/functions/btrim.go +++ b/server/functions/btrim.go @@ -31,9 +31,9 @@ func initBtrim() { var btrim_text = framework.Function1{ Name: "btrim", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, t [2]pgtypes.DoltgresType, str any) (any, error) { + Callable: func(ctx *sql.Context, t [2]*pgtypes.DoltgresType, str any) (any, error) { result, err := ltrim_text.Callable(ctx, t, str) if err != nil { return nil, err @@ -46,9 +46,9 @@ var btrim_text = framework.Function1{ var btrim_text_text = framework.Function2{ Name: "btrim", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, t [3]pgtypes.DoltgresType, str any, characters any) (any, error) { + Callable: func(ctx *sql.Context, t [3]*pgtypes.DoltgresType, str any, characters any) (any, error) { result, err := ltrim_text_text.Callable(ctx, t, str, characters) if err != nil { return nil, err diff --git a/server/functions/bytea.go b/server/functions/bytea.go new file mode 100644 index 0000000000..ec4919bf24 --- /dev/null +++ b/server/functions/bytea.go @@ -0,0 +1,104 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "bytes" + "encoding/hex" + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" + "github.com/dolthub/doltgresql/utils" +) + +// initBytea registers the functions to the catalog. +func initBytea() { + framework.RegisterFunction(byteain) + framework.RegisterFunction(byteaout) + framework.RegisterFunction(bytearecv) + framework.RegisterFunction(byteasend) + framework.RegisterFunction(byteacmp) +} + +// byteain represents the PostgreSQL function of bytea type IO input. +var byteain = framework.Function1{ + Name: "byteain", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + if strings.HasPrefix(input, `\x`) { + return hex.DecodeString(input[2:]) + } else { + return []byte(input), nil + } + }, +} + +// byteaout represents the PostgreSQL function of bytea type IO output. +var byteaout = framework.Function1{ + Name: "byteaout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Bytea}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return `\x` + hex.EncodeToString(val.([]byte)), nil + }, +} + +// bytearecv represents the PostgreSQL function of bytea type IO receive. +var bytearecv = framework.Function1{ + Name: "bytearecv", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + reader := utils.NewReader(data) + return reader.ByteSlice(), nil + }, +} + +// byteasend represents the PostgreSQL function of bytea type IO send. +var byteasend = framework.Function1{ + Name: "byteasend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Bytea}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + str := val.([]byte) + writer := utils.NewWriter(uint64(len(str) + 4)) + writer.ByteSlice(str) + return writer.Data(), nil + }, +} + +// byteacmp represents the PostgreSQL function of bytea type compare. +var byteacmp = framework.Function2{ + Name: "byteacmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Bytea, pgtypes.Bytea}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + return int32(bytes.Compare(val1.([]byte), val2.([]byte))), nil + }, +} diff --git a/server/functions/cbrt.go b/server/functions/cbrt.go index f3435805f0..889793eaae 100644 --- a/server/functions/cbrt.go +++ b/server/functions/cbrt.go @@ -32,9 +32,9 @@ func initCbrt() { var cbrt_float64 = framework.Function1{ Name: "cbrt", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Cbrt(val1.(float64)), nil }, } diff --git a/server/functions/ceil.go b/server/functions/ceil.go index 5ade00472e..ff2cfa6aa4 100644 --- a/server/functions/ceil.go +++ b/server/functions/ceil.go @@ -41,9 +41,9 @@ func initCeil() { var ceil_float64 = framework.Function1{ Name: "ceil", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Ceil(val1.(float64)), nil }, } @@ -52,9 +52,9 @@ var ceil_float64 = framework.Function1{ var ceil_numeric = framework.Function1{ Name: "ceil", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return val1.(decimal.Decimal).Ceil(), nil }, } diff --git a/server/functions/char.go b/server/functions/char.go new file mode 100644 index 0000000000..705c2b724c --- /dev/null +++ b/server/functions/char.go @@ -0,0 +1,114 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" + "github.com/dolthub/doltgresql/utils" +) + +// initChar registers the functions to the catalog. +func initChar() { + framework.RegisterFunction(charin) + framework.RegisterFunction(charout) + framework.RegisterFunction(charrecv) + framework.RegisterFunction(charsend) + framework.RegisterFunction(btcharcmp) +} + +// charin represents the PostgreSQL function of "char" type IO input. +var charin = framework.Function1{ + Name: "charin", + Return: pgtypes.InternalChar, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + c := []byte(input) + if uint32(len(c)) > pgtypes.InternalCharLength { + return input[:pgtypes.InternalCharLength], nil + } + return input, nil + }, +} + +// charout represents the PostgreSQL function of "char" type IO output. +var charout = framework.Function1{ + Name: "charout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.InternalChar}, + Strict: true, + Callable: func(ctx *sql.Context, t [2]*pgtypes.DoltgresType, val any) (any, error) { + str := val.(string) + if uint32(len(str)) > pgtypes.InternalCharLength { + return str[:pgtypes.InternalCharLength], nil + } + return str, nil + }, +} + +// charrecv represents the PostgreSQL function of "char" type IO receive. +var charrecv = framework.Function1{ + Name: "charrecv", + Return: pgtypes.InternalChar, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + reader := utils.NewReader(data) + return reader.String(), nil + }, +} + +// charsend represents the PostgreSQL function of "char" type IO send. +var charsend = framework.Function1{ + Name: "charsend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.InternalChar}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + str := val.(string) + writer := utils.NewWriter(uint64(len(str) + 4)) + writer.String(str) + return writer.Data(), nil + }, +} + +// btcharcmp represents the PostgreSQL function of "char" type compare. +var btcharcmp = framework.Function2{ + Name: "btcharcmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.InternalChar, pgtypes.InternalChar}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := strings.TrimRight(val1.(string), " ") + bb := strings.TrimRight(val2.(string), " ") + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} diff --git a/server/functions/char_length.go b/server/functions/char_length.go index 254bd0bc71..99ecccaf47 100644 --- a/server/functions/char_length.go +++ b/server/functions/char_length.go @@ -34,9 +34,9 @@ func initCharLength() { var char_length_text = framework.Function1{ Name: "char_length", Return: pgtypes.Int32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return int32(len([]rune(val1.(string)))), nil }, } diff --git a/server/functions/chr.go b/server/functions/chr.go index 811e47d197..cd040a1af7 100644 --- a/server/functions/chr.go +++ b/server/functions/chr.go @@ -33,9 +33,9 @@ func initChr() { var chr_int32 = framework.Function1{ Name: "chr", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int32}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1Interface any) (any, error) { val1 := val1Interface.(int32) if val1 == 0 { return nil, fmt.Errorf("null character not permitted") diff --git a/server/functions/col_description.go b/server/functions/col_description.go index 57cfd55a15..f671c6dda0 100644 --- a/server/functions/col_description.go +++ b/server/functions/col_description.go @@ -30,10 +30,10 @@ func initColDescription() { var col_description = framework.Function2{ Name: "col_description", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Int32}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: When we support comments this should return the comment for a table // column, which is specified by the OID of its table and its column number return "", nil diff --git a/server/functions/cos.go b/server/functions/cos.go index 16af4298e8..ea7019ea40 100644 --- a/server/functions/cos.go +++ b/server/functions/cos.go @@ -32,9 +32,9 @@ func initCos() { var cos_float64 = framework.Function1{ Name: "cos", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Cos(val1.(float64)), nil }, } diff --git a/server/functions/cosd.go b/server/functions/cosd.go index 88a388f529..642f0378a4 100644 --- a/server/functions/cosd.go +++ b/server/functions/cosd.go @@ -32,9 +32,9 @@ func initCosd() { var cosd_float64 = framework.Function1{ Name: "cosd", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Cos(toRadians(val1.(float64))), nil }, } diff --git a/server/functions/cosh.go b/server/functions/cosh.go index 4ef2350a49..49a31e7a53 100644 --- a/server/functions/cosh.go +++ b/server/functions/cosh.go @@ -32,9 +32,9 @@ func initCosh() { var cosh_float64 = framework.Function1{ Name: "cosh", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Cosh(val1.(float64)), nil }, } diff --git a/server/functions/cot.go b/server/functions/cot.go index ba34a066e2..affd47dda0 100644 --- a/server/functions/cot.go +++ b/server/functions/cot.go @@ -32,9 +32,9 @@ func initCot() { var cot_float64 = framework.Function1{ Name: "cot", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1Interface any) (any, error) { val1 := val1Interface.(float64) if val1 == 0 { return math.Inf(1), nil diff --git a/server/functions/cotd.go b/server/functions/cotd.go index 5572bbc538..5f2aea52b7 100644 --- a/server/functions/cotd.go +++ b/server/functions/cotd.go @@ -32,9 +32,9 @@ func initCotd() { var cotd_float64 = framework.Function1{ Name: "cotd", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1Interface any) (any, error) { val1 := toRadians(val1Interface.(float64)) if val1 == 0 { return math.Inf(1), nil diff --git a/server/functions/current_schemas.go b/server/functions/current_schemas.go index 5371c74ef7..486bc3b0d5 100644 --- a/server/functions/current_schemas.go +++ b/server/functions/current_schemas.go @@ -32,10 +32,10 @@ func initCurrentSchemas() { var current_schemas = framework.Function1{ Name: "current_schemas", Return: pgtypes.NameArray, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Bool}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Bool}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { schemas := make([]any, 0) if val.(bool) { schemas = append(schemas, sessiondata.PgCatalogName) diff --git a/server/functions/current_setting.go b/server/functions/current_setting.go index 1eaa68dca4..aff2412346 100755 --- a/server/functions/current_setting.go +++ b/server/functions/current_setting.go @@ -32,9 +32,9 @@ func initCurrentSetting() { var current_setting = framework.Function1{ Name: "current_setting", Return: pgtypes.Text, // TODO: it would be nice to support non-text values as well, but this is all postgres supports - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { s := val1.(string) _, variable, err := ctx.GetUserVariable(ctx, s) if err != nil { diff --git a/server/functions/date.go b/server/functions/date.go new file mode 100644 index 0000000000..bcd0ea6327 --- /dev/null +++ b/server/functions/date.go @@ -0,0 +1,108 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "time" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/postgres/parser/pgdate" + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initDate registers the functions to the catalog. +func initDate() { + framework.RegisterFunction(date_in) + framework.RegisterFunction(date_out) + framework.RegisterFunction(date_recv) + framework.RegisterFunction(date_send) + framework.RegisterFunction(date_cmp) +} + +// date_in represents the PostgreSQL function of date type IO input. +var date_in = framework.Function1{ + Name: "date_in", + Return: pgtypes.Date, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + if date, _, err := pgdate.ParseDate(time.Now(), pgdate.ParseModeYMD, input); err == nil { + return date.ToTime() + } else if date, _, err = pgdate.ParseDate(time.Now(), pgdate.ParseModeDMY, input); err == nil { + return date.ToTime() + } else if date, _, err = pgdate.ParseDate(time.Now(), pgdate.ParseModeMDY, input); err == nil { + return date.ToTime() + } else { + return nil, err + } + }, +} + +// date_out represents the PostgreSQL function of date type IO output. +var date_out = framework.Function1{ + Name: "date_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Date}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(time.Time).Format("2006-01-02"), nil + }, +} + +// date_recv represents the PostgreSQL function of date type IO receive. +var date_recv = framework.Function1{ + Name: "date_recv", + Return: pgtypes.Date, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + t := time.Time{} + if err := t.UnmarshalBinary(data); err != nil { + return nil, err + } + return t, nil + }, +} + +// date_send represents the PostgreSQL function of date type IO send. +var date_send = framework.Function1{ + Name: "date_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Date}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(time.Time).MarshalBinary() + }, +} + +// date_cmp represents the PostgreSQL function of date type compare. +var date_cmp = framework.Function2{ + Name: "date_cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Date, pgtypes.Date}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(time.Time) + bb := val2.(time.Time) + return int32(ab.Compare(bb)), nil + }, +} diff --git a/server/functions/degrees.go b/server/functions/degrees.go index 612fe02524..b0507b93c2 100644 --- a/server/functions/degrees.go +++ b/server/functions/degrees.go @@ -32,9 +32,9 @@ func initDegrees() { var degrees_float64 = framework.Function1{ Name: "degrees", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return toDegrees(val1.(float64)), nil }, } diff --git a/server/functions/div.go b/server/functions/div.go index e94894b1e8..36b2eeaa43 100644 --- a/server/functions/div.go +++ b/server/functions/div.go @@ -33,9 +33,9 @@ func initDiv() { var div_numeric = framework.Function2{ Name: "div", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1Interface any, val2Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1Interface any, val2Interface any) (any, error) { val1 := val1Interface.(decimal.Decimal) val2 := val2Interface.(decimal.Decimal) if val2.Cmp(decimal.Zero) == 0 { diff --git a/server/functions/dolt_procedures.go b/server/functions/dolt_procedures.go index b36bb0d18a..7069ab3de9 100755 --- a/server/functions/dolt_procedures.go +++ b/server/functions/dolt_procedures.go @@ -44,17 +44,17 @@ func initDoltProcedures() { framework.RegisterFunction(framework.Function1{ Name: procDef.Name, Return: pgtypes.TextArray, - Parameters: [1]pgtypes.DoltgresType{pgtypes.TextArray}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.TextArray}, Variadic: true, Callable: callable, }) } } -func callableForDoltProcedure(p *plan.ExternalProcedure, funcVal reflect.Value) func(ctx *sql.Context, paramsAndReturn [2]pgtypes.DoltgresType, val1 any) (any, error) { +func callableForDoltProcedure(p *plan.ExternalProcedure, funcVal reflect.Value) func(ctx *sql.Context, paramsAndReturn [2]*pgtypes.DoltgresType, val1 any) (any, error) { funcType := funcVal.Type() - return func(ctx *sql.Context, paramsAndReturn [2]pgtypes.DoltgresType, val1 any) (any, error) { + return func(ctx *sql.Context, paramsAndReturn [2]*pgtypes.DoltgresType, val1 any) (any, error) { values, ok := val1.([]any) if !ok { return nil, sql.ErrExternalProcedureInvalidParamType.New(reflect.TypeOf(val1).String()) @@ -119,7 +119,7 @@ func drainRowIter(ctx *sql.Context, rowIter sql.RowIter) (any, error) { return nil, err } - castFn := framework.GetExplicitCast(fromType, pgtypes.Text.BaseID()) + castFn := framework.GetExplicitCast(fromType, pgtypes.Text) textVal, err := castFn(ctx, row[i], pgtypes.Text) if err != nil { return nil, err @@ -130,18 +130,18 @@ func drainRowIter(ctx *sql.Context, rowIter sql.RowIter) (any, error) { return rowSlice, nil } -func typeForElement(v any) (pgtypes.DoltgresTypeBaseID, error) { +func typeForElement(v any) (*pgtypes.DoltgresType, error) { switch x := v.(type) { case int64: - return pgtypes.Int64.BaseID(), nil + return pgtypes.Int64, nil case int32: - return pgtypes.Int32.BaseID(), nil + return pgtypes.Int32, nil case int16, int8: - return pgtypes.Int16.BaseID(), nil + return pgtypes.Int16, nil case string: - return pgtypes.Text.BaseID(), nil + return pgtypes.Text, nil default: - return 0, fmt.Errorf("dolt_procedures: unsupported type %T", x) + return nil, fmt.Errorf("dolt_procedures: unsupported type %T", x) } } diff --git a/server/functions/domain.go b/server/functions/domain.go new file mode 100644 index 0000000000..ff679671ff --- /dev/null +++ b/server/functions/domain.go @@ -0,0 +1,58 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initDomain registers the functions to the catalog. +func initDomain() { + framework.RegisterFunction(domain_in) + framework.RegisterFunction(domain_recv) +} + +// domain_in represents the PostgreSQL function of domain type IO input. +var domain_in = framework.Function3{ + Name: "domain_in", + Return: pgtypes.Any, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Cstring, pgtypes.Oid, pgtypes.Int32}, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + str := val1.(string) + baseTypeOid := val2.(uint32) + t := pgtypes.OidToBuiltInDoltgresType[baseTypeOid] + typmod := val3.(int32) + t = t.WithAttTypMod(typmod) + return t.IoInput(ctx, str) + }, +} + +// domain_recv represents the PostgreSQL function of domain type IO receive. +var domain_recv = framework.Function3{ + Name: "domain_recv", + Return: pgtypes.Any, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Internal, pgtypes.Oid, pgtypes.Int32}, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + data := val1.([]byte) + baseTypeOid := val2.(uint32) + t := pgtypes.OidToBuiltInDoltgresType[baseTypeOid] + typmod := val3.(int32) + t = t.WithAttTypMod(typmod) + return t.DeserializeValue(data) + }, +} diff --git a/server/functions/exp.go b/server/functions/exp.go index 6d6fff1ddd..1987844334 100644 --- a/server/functions/exp.go +++ b/server/functions/exp.go @@ -34,9 +34,9 @@ func initExp() { var exp_float64 = framework.Function1{ Name: "exp", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Exp(val1.(float64)), nil }, } @@ -45,9 +45,9 @@ var exp_float64 = framework.Function1{ var exp_numeric = framework.Function1{ Name: "exp", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1 == nil { return nil, nil } diff --git a/server/functions/extract.go b/server/functions/extract.go index f4a2182519..cc4648b004 100644 --- a/server/functions/extract.go +++ b/server/functions/extract.go @@ -45,10 +45,10 @@ var ErrUnitNotSupported = errors.NewKind("unit \"%s\" not supported for type %s" var extract_text_date = framework.Function2{ Name: "extract", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Date}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Date}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { field := val1.(string) dateVal := val2.(time.Time) switch strings.ToLower(field) { @@ -67,10 +67,10 @@ var extract_text_date = framework.Function2{ var extract_text_time = framework.Function2{ Name: "extract", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Time}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Time}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { field := val1.(string) timeVal := val2.(time.Time) switch strings.ToLower(field) { @@ -88,10 +88,10 @@ var extract_text_time = framework.Function2{ var extract_text_timetz = framework.Function2{ Name: "extract", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.TimeTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.TimeTZ}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { field := val1.(string) timetzVal := val2.(time.Time) _, currentOffset := timetzVal.Zone() @@ -116,10 +116,10 @@ var extract_text_timetz = framework.Function2{ var extract_text_timestamp = framework.Function2{ Name: "extract", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Timestamp}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { field := val1.(string) tsVal := val2.(time.Time) switch strings.ToLower(field) { @@ -135,12 +135,12 @@ var extract_text_timestamp = framework.Function2{ var extract_text_timestamptz = framework.Function2{ Name: "extract", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.TimestampTZ}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { field := val1.(string) - loc, err := pgtypes.GetServerLocation(ctx) + loc, err := GetServerLocation(ctx) if err != nil { return nil, err } @@ -171,10 +171,10 @@ const ( var extract_text_interval = framework.Function2{ Name: "extract", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Interval}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Interval}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { field := val1.(string) dur := val2.(duration.Duration) switch strings.ToLower(field) { diff --git a/server/functions/factorial.go b/server/functions/factorial.go index 4cf26deb3d..6279b7f0b4 100644 --- a/server/functions/factorial.go +++ b/server/functions/factorial.go @@ -33,9 +33,9 @@ func initFactorial() { var factorial_int64 = framework.Function1{ Name: "factorial", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1Interface any) (any, error) { val1 := val1Interface.(int64) if val1 < 0 { return nil, fmt.Errorf("factorial of a negative number is undefined") diff --git a/server/functions/float4.go b/server/functions/float4.go new file mode 100644 index 0000000000..ad5584c6ea --- /dev/null +++ b/server/functions/float4.go @@ -0,0 +1,141 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "encoding/binary" + "math" + "strconv" + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initFloat4 registers the functions to the catalog. +func initFloat4() { + framework.RegisterFunction(float4in) + framework.RegisterFunction(float4out) + framework.RegisterFunction(float4recv) + framework.RegisterFunction(float4send) + framework.RegisterFunction(btfloat4cmp) + framework.RegisterFunction(btfloat48cmp) +} + +// float4in represents the PostgreSQL function of float4 type IO input. +var float4in = framework.Function1{ + Name: "float4in", + Return: pgtypes.Float32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + fVal, err := strconv.ParseFloat(strings.TrimSpace(input), 32) + if err != nil { + return nil, pgtypes.ErrInvalidSyntaxForType.New("float4", input) + } + return float32(fVal), nil + }, +} + +// float4out represents the PostgreSQL function of float4 type IO output. +var float4out = framework.Function1{ + Name: "float4out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return strconv.FormatFloat(float64(val.(float32)), 'f', -1, 32), nil + }, +} + +// float4recv represents the PostgreSQL function of float4 type IO receive. +var float4recv = framework.Function1{ + Name: "float4recv", + Return: pgtypes.Float32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + unsignedBits := binary.BigEndian.Uint32(data) + if unsignedBits&(1<<31) != 0 { + unsignedBits ^= 1 << 31 + } else { + unsignedBits = ^unsignedBits + } + return math.Float32frombits(unsignedBits), nil + }, +} + +// float4send represents the PostgreSQL function of float4 type IO send. +var float4send = framework.Function1{ + Name: "float4send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + f32 := val.(float32) + retVal := make([]byte, 4) + // Make the serialized form trivially comparable using bytes.Compare: https://stackoverflow.com/a/54557561 + unsignedBits := math.Float32bits(f32) + if f32 >= 0 { + unsignedBits ^= 1 << 31 + } else { + unsignedBits = ^unsignedBits + } + binary.BigEndian.PutUint32(retVal, unsignedBits) + return retVal, nil + }, +} + +// btfloat4cmp represents the PostgreSQL function of float4 type compare. +var btfloat4cmp = framework.Function2{ + Name: "btfloat4cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(float32) + bb := val2.(float32) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// btfloat48cmp represents the PostgreSQL function of float4 type compare with float8. +var btfloat48cmp = framework.Function2{ + Name: "btfloat48cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float32, pgtypes.Float64}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := float64(val1.(float32)) + bb := val2.(float64) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} diff --git a/server/functions/float8.go b/server/functions/float8.go new file mode 100644 index 0000000000..1f3e7b08ca --- /dev/null +++ b/server/functions/float8.go @@ -0,0 +1,144 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "encoding/binary" + "math" + "strconv" + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initFloat8 registers the functions to the catalog. +func initFloat8() { + framework.RegisterFunction(float8in) + framework.RegisterFunction(float8out) + framework.RegisterFunction(float8recv) + framework.RegisterFunction(float8send) + framework.RegisterFunction(btfloat8cmp) + framework.RegisterFunction(btfloat84cmp) +} + +// float8in represents the PostgreSQL function of float8 type IO input. +var float8in = framework.Function1{ + Name: "float8in", + Return: pgtypes.Float64, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + fVal, err := strconv.ParseFloat(strings.TrimSpace(input), 64) + if err != nil { + return nil, pgtypes.ErrInvalidSyntaxForType.New("float8", input) + } + return fVal, nil + }, +} + +// float8out represents the PostgreSQL function of float8 type IO output. +var float8out = framework.Function1{ + Name: "float8out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return strconv.FormatFloat(val.(float64), 'f', -1, 64), nil + }, +} + +// float8recv represents the PostgreSQL function of float8 type IO receive. +var float8recv = framework.Function1{ + Name: "float8recv", + Return: pgtypes.Float64, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + unsignedBits := binary.BigEndian.Uint64(data) + if unsignedBits&(1<<63) != 0 { + unsignedBits ^= 1 << 63 + } else { + unsignedBits = ^unsignedBits + } + return math.Float64frombits(unsignedBits), nil + }, +} + +// float8send represents the PostgreSQL function of float8 type IO send. +var float8send = framework.Function1{ + Name: "float8send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + f64 := val.(float64) + retVal := make([]byte, 8) + // Make the serialized form trivially comparable using bytes.Compare: https://stackoverflow.com/a/54557561 + unsignedBits := math.Float64bits(f64) + if f64 >= 0 { + unsignedBits ^= 1 << 63 + } else { + unsignedBits = ^unsignedBits + } + binary.BigEndian.PutUint64(retVal, unsignedBits) + return retVal, nil + }, +} + +// btfloat8cmp represents the PostgreSQL function of float8 type compare. +var btfloat8cmp = framework.Function2{ + Name: "btfloat8cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(float64) + bb := val2.(float64) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// btfloat84cmp represents the PostgreSQL function of float8 type compare with float4. +var btfloat84cmp = framework.Function2{ + Name: "btfloat84cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(float64) + bb := float64(val2.(float32)) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} diff --git a/server/functions/floor.go b/server/functions/floor.go index 90af542c03..3191ada5d8 100644 --- a/server/functions/floor.go +++ b/server/functions/floor.go @@ -34,9 +34,9 @@ func initFloor() { var floor_float64 = framework.Function1{ Name: "floor", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Floor(val1.(float64)), nil }, } @@ -45,9 +45,9 @@ var floor_float64 = framework.Function1{ var floor_numeric = framework.Function1{ Name: "floor", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1 == nil { return nil, nil } diff --git a/server/functions/format_type.go b/server/functions/format_type.go index 881682fe17..c6ff3770b0 100644 --- a/server/functions/format_type.go +++ b/server/functions/format_type.go @@ -32,8 +32,8 @@ func initFormatType() { var format_type = framework.Function2{ Name: "format_type", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Int32}, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Int32}, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { if val1 == nil { return nil, nil } diff --git a/server/functions/framework/cast.go b/server/functions/framework/cast.go index 25c3b70d6b..684cc51ab1 100644 --- a/server/functions/framework/cast.go +++ b/server/functions/framework/cast.go @@ -27,16 +27,16 @@ import ( // TypeCastFunction is a function that takes a value of a particular kind of type, and returns it as another kind of type. // The targetType given should match the "To" type used to obtain the cast. -type TypeCastFunction func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) +type TypeCastFunction func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) // getCastFunction is used to recursively call the cast function for when the inner logic sees that it has two array // types. This sidesteps providing -type getCastFunction func(fromType pgtypes.DoltgresTypeBaseID, toType pgtypes.DoltgresTypeBaseID) TypeCastFunction +type getCastFunction func(fromType *pgtypes.DoltgresType, toType *pgtypes.DoltgresType) TypeCastFunction // TypeCast is used to cast from one type to another. type TypeCast struct { - FromType pgtypes.DoltgresType - ToType pgtypes.DoltgresType + FromType *pgtypes.DoltgresType + ToType *pgtypes.DoltgresType Function TypeCastFunction } @@ -44,28 +44,28 @@ type TypeCast struct { var explicitTypeCastMutex = &sync.RWMutex{} // explicitTypeCastsMap is a map that maps: from -> to -> function. -var explicitTypeCastsMap = map[pgtypes.DoltgresTypeBaseID]map[pgtypes.DoltgresTypeBaseID]TypeCastFunction{} +var explicitTypeCastsMap = map[uint32]map[uint32]TypeCastFunction{} // explicitTypeCastsArray is a slice that holds all registered explicit casts from the given type. -var explicitTypeCastsArray = map[pgtypes.DoltgresTypeBaseID][]pgtypes.DoltgresType{} +var explicitTypeCastsArray = map[uint32][]*pgtypes.DoltgresType{} // assignmentTypeCastMutex is used to lock the assignment type cast map and array when writing. var assignmentTypeCastMutex = &sync.RWMutex{} // assignmentTypeCastsMap is a map that maps: from -> to -> function. -var assignmentTypeCastsMap = map[pgtypes.DoltgresTypeBaseID]map[pgtypes.DoltgresTypeBaseID]TypeCastFunction{} +var assignmentTypeCastsMap = map[uint32]map[uint32]TypeCastFunction{} // assignmentTypeCastsArray is a slice that holds all registered assignment casts from the given type. -var assignmentTypeCastsArray = map[pgtypes.DoltgresTypeBaseID][]pgtypes.DoltgresType{} +var assignmentTypeCastsArray = map[uint32][]*pgtypes.DoltgresType{} // implicitTypeCastMutex is used to lock the implicit type cast map and array when writing. var implicitTypeCastMutex = &sync.RWMutex{} // implicitTypeCastsMap is a map that maps: from -> to -> function. -var implicitTypeCastsMap = map[pgtypes.DoltgresTypeBaseID]map[pgtypes.DoltgresTypeBaseID]TypeCastFunction{} +var implicitTypeCastsMap = map[uint32]map[uint32]TypeCastFunction{} // implicitTypeCastsArray is a slice that holds all registered implicit casts from the given type. -var implicitTypeCastsArray = map[pgtypes.DoltgresTypeBaseID][]pgtypes.DoltgresType{} +var implicitTypeCastsArray = map[uint32][]*pgtypes.DoltgresType{} // AddExplicitTypeCast registers the given explicit type cast. func AddExplicitTypeCast(cast TypeCast) error { @@ -104,28 +104,28 @@ func MustAddImplicitTypeCast(cast TypeCast) { } // GetPotentialExplicitCasts returns all registered explicit type casts from the given type. -func GetPotentialExplicitCasts(fromType pgtypes.DoltgresTypeBaseID) []pgtypes.DoltgresType { +func GetPotentialExplicitCasts(fromType uint32) []*pgtypes.DoltgresType { return getPotentialCasts(explicitTypeCastMutex, explicitTypeCastsArray, fromType) } // GetPotentialAssignmentCasts returns all registered assignment and implicit type casts from the given type. -func GetPotentialAssignmentCasts(fromType pgtypes.DoltgresTypeBaseID) []pgtypes.DoltgresType { +func GetPotentialAssignmentCasts(fromType uint32) []*pgtypes.DoltgresType { assignment := getPotentialCasts(assignmentTypeCastMutex, assignmentTypeCastsArray, fromType) implicit := GetPotentialImplicitCasts(fromType) - both := make([]pgtypes.DoltgresType, len(assignment)+len(implicit)) + both := make([]*pgtypes.DoltgresType, len(assignment)+len(implicit)) copy(both, assignment) copy(both[len(assignment):], implicit) return both } // GetPotentialImplicitCasts returns all registered implicit type casts from the given type. -func GetPotentialImplicitCasts(fromType pgtypes.DoltgresTypeBaseID) []pgtypes.DoltgresType { +func GetPotentialImplicitCasts(fromType uint32) []*pgtypes.DoltgresType { return getPotentialCasts(implicitTypeCastMutex, implicitTypeCastsArray, fromType) } // GetExplicitCast returns the explicit type cast function that will cast the "from" type to the "to" type. Returns nil // if such a cast is not valid. -func GetExplicitCast(fromType pgtypes.DoltgresTypeBaseID, toType pgtypes.DoltgresTypeBaseID) TypeCastFunction { +func GetExplicitCast(fromType *pgtypes.DoltgresType, toType *pgtypes.DoltgresType) TypeCastFunction { if tcf := getCast(explicitTypeCastMutex, explicitTypeCastsMap, fromType, toType, GetExplicitCast); tcf != nil { return tcf } else if tcf = getCast(assignmentTypeCastMutex, assignmentTypeCastsMap, fromType, toType, GetExplicitCast); tcf != nil { @@ -136,28 +136,28 @@ func GetExplicitCast(fromType pgtypes.DoltgresTypeBaseID, toType pgtypes.Doltgre // We check for the identity after checking the maps, as the identity may be overridden (such as for types that have // parameters). If one of the types are a string type, then we do not use the identity, and use the I/O conversions // below. - if fromType == toType && toType.GetTypeCategory() != pgtypes.TypeCategory_StringTypes && fromType.GetTypeCategory() != pgtypes.TypeCategory_StringTypes { + if fromType.OID == toType.OID && toType.TypCategory != pgtypes.TypeCategory_StringTypes && fromType.TypCategory != pgtypes.TypeCategory_StringTypes { return identityCast } // All types have a built-in explicit cast from string types: https://www.postgresql.org/docs/15/sql-createcast.html - if fromType.GetTypeCategory() == pgtypes.TypeCategory_StringTypes { - return func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + if fromType.TypCategory == pgtypes.TypeCategory_StringTypes { + return func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val == nil { return nil, nil } - str, err := fromType.GetRepresentativeType().IoOutput(ctx, val) + str, err := fromType.IoOutput(ctx, val) if err != nil { return nil, err } return targetType.IoInput(ctx, str) } - } else if toType.GetTypeCategory() == pgtypes.TypeCategory_StringTypes { + } else if toType.TypCategory == pgtypes.TypeCategory_StringTypes { // All types have a built-in assignment cast to string types, which we can reference in an explicit cast - return func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + return func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val == nil { return nil, nil } - str, err := fromType.GetRepresentativeType().IoOutput(ctx, val) + str, err := fromType.IoOutput(ctx, val) if err != nil { return nil, err } @@ -169,7 +169,7 @@ func GetExplicitCast(fromType pgtypes.DoltgresTypeBaseID, toType pgtypes.Doltgre // GetAssignmentCast returns the assignment type cast function that will cast the "from" type to the "to" type. Returns // nil if such a cast is not valid. -func GetAssignmentCast(fromType pgtypes.DoltgresTypeBaseID, toType pgtypes.DoltgresTypeBaseID) TypeCastFunction { +func GetAssignmentCast(fromType *pgtypes.DoltgresType, toType *pgtypes.DoltgresType) TypeCastFunction { if tcf := getCast(assignmentTypeCastMutex, assignmentTypeCastsMap, fromType, toType, GetAssignmentCast); tcf != nil { return tcf } else if tcf = getCast(implicitTypeCastMutex, implicitTypeCastsMap, fromType, toType, GetAssignmentCast); tcf != nil { @@ -177,16 +177,16 @@ func GetAssignmentCast(fromType pgtypes.DoltgresTypeBaseID, toType pgtypes.Doltg } // We check for the identity after checking the maps, as the identity may be overridden (such as for types that have // parameters). If the "to" type is a string type, then we do not use the identity, and use the I/O conversion below. - if fromType == toType && fromType.GetTypeCategory() != pgtypes.TypeCategory_StringTypes { + if fromType.OID == toType.OID && fromType.TypCategory != pgtypes.TypeCategory_StringTypes { return identityCast } // All types have a built-in assignment cast to string types: https://www.postgresql.org/docs/15/sql-createcast.html - if toType.GetTypeCategory() == pgtypes.TypeCategory_StringTypes { - return func(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { + if toType.TypCategory == pgtypes.TypeCategory_StringTypes { + return func(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val == nil { return nil, nil } - str, err := fromType.GetRepresentativeType().IoOutput(ctx, val) + str, err := fromType.IoOutput(ctx, val) if err != nil { return nil, err } @@ -198,13 +198,13 @@ func GetAssignmentCast(fromType pgtypes.DoltgresTypeBaseID, toType pgtypes.Doltg // GetImplicitCast returns the implicit type cast function that will cast the "from" type to the "to" type. Returns nil // if such a cast is not valid. -func GetImplicitCast(fromType pgtypes.DoltgresTypeBaseID, toType pgtypes.DoltgresTypeBaseID) TypeCastFunction { +func GetImplicitCast(fromType *pgtypes.DoltgresType, toType *pgtypes.DoltgresType) TypeCastFunction { if tcf := getCast(implicitTypeCastMutex, implicitTypeCastsMap, fromType, toType, GetImplicitCast); tcf != nil { return tcf } // We check for the identity after checking the maps, as the identity may be overridden (such as for types that have // parameters). - if fromType == toType { + if fromType.OID == toType.OID { return identityCast } return nil @@ -212,28 +212,28 @@ func GetImplicitCast(fromType pgtypes.DoltgresTypeBaseID, toType pgtypes.Doltgre // addTypeCast registers the given type cast. func addTypeCast(mutex *sync.RWMutex, - castMap map[pgtypes.DoltgresTypeBaseID]map[pgtypes.DoltgresTypeBaseID]TypeCastFunction, - castArray map[pgtypes.DoltgresTypeBaseID][]pgtypes.DoltgresType, cast TypeCast) error { + castMap map[uint32]map[uint32]TypeCastFunction, + castArray map[uint32][]*pgtypes.DoltgresType, cast TypeCast) error { mutex.Lock() defer mutex.Unlock() - toMap, ok := castMap[cast.FromType.BaseID()] + toMap, ok := castMap[cast.FromType.OID] if !ok { - toMap = map[pgtypes.DoltgresTypeBaseID]TypeCastFunction{} - castMap[cast.FromType.BaseID()] = toMap - castArray[cast.FromType.BaseID()] = nil + toMap = map[uint32]TypeCastFunction{} + castMap[cast.FromType.OID] = toMap + castArray[cast.FromType.OID] = nil } - if _, ok := toMap[cast.ToType.BaseID()]; ok { + if _, ok := toMap[cast.ToType.OID]; ok { // TODO: return the actual Postgres error return fmt.Errorf("cast from `%s` to `%s` already exists", cast.FromType.String(), cast.ToType.String()) } - toMap[cast.ToType.BaseID()] = cast.Function - castArray[cast.FromType.BaseID()] = append(castArray[cast.FromType.BaseID()], cast.ToType) + toMap[cast.ToType.OID] = cast.Function + castArray[cast.FromType.OID] = append(castArray[cast.FromType.OID], cast.ToType) return nil } // getPotentialCasts returns all registered type casts from the given type. -func getPotentialCasts(mutex *sync.RWMutex, castArray map[pgtypes.DoltgresTypeBaseID][]pgtypes.DoltgresType, fromType pgtypes.DoltgresTypeBaseID) []pgtypes.DoltgresType { +func getPotentialCasts(mutex *sync.RWMutex, castArray map[uint32][]*pgtypes.DoltgresType, fromType uint32) []*pgtypes.DoltgresType { mutex.RLock() defer mutex.RUnlock() @@ -243,55 +243,56 @@ func getPotentialCasts(mutex *sync.RWMutex, castArray map[pgtypes.DoltgresTypeBa // getCast returns the type cast function that will cast the "from" type to the "to" type. Returns nil if such a cast is // not valid. func getCast(mutex *sync.RWMutex, - castMap map[pgtypes.DoltgresTypeBaseID]map[pgtypes.DoltgresTypeBaseID]TypeCastFunction, - fromType pgtypes.DoltgresTypeBaseID, toType pgtypes.DoltgresTypeBaseID, outerFunc getCastFunction) TypeCastFunction { + castMap map[uint32]map[uint32]TypeCastFunction, + fromType *pgtypes.DoltgresType, toType *pgtypes.DoltgresType, outerFunc getCastFunction) TypeCastFunction { mutex.RLock() defer mutex.RUnlock() - if toMap, ok := castMap[fromType]; ok { - if f, ok := toMap[toType]; ok { + if toMap, ok := castMap[fromType.OID]; ok { + if f, ok := toMap[toType.OID]; ok { return f } } // If there isn't a direct mapping, then we need to check if the types are array variants. // As long as the base types are convertable, the array variants are also convertable. - // TODO: currently, unknown type is considered an array type, need to look into it. - if fromArrayType, ok := fromType.IsBaseIDArrayType(); ok && fromType != pgtypes.DoltgresTypeBaseID_Unknown { - if toArrayType, ok := toType.IsBaseIDArrayType(); ok { - if baseCast := outerFunc(fromArrayType.BaseType().BaseID(), toArrayType.BaseType().BaseID()); baseCast != nil { - // We use a closure that can unwrap the slice, since conversion functions expect a singular non-nil value - return func(ctx *sql.Context, vals any, targetType pgtypes.DoltgresType) (any, error) { - var err error - oldVals := vals.([]any) - newVals := make([]any, len(oldVals)) - for i, oldVal := range oldVals { - if oldVal == nil { - continue - } - // Some errors are optional depending on the context, so we'll still process all values even - // after an error is received. - var nErr error - newVals[i], nErr = baseCast(ctx, oldVal, targetType.(pgtypes.DoltgresArrayType).BaseType()) - if nErr != nil && err == nil { - err = nErr - } + if fromType.IsArrayType() && toType.IsArrayType() { + fromBaseType := fromType.ArrayBaseType() + toBaseType := toType.ArrayBaseType() + if baseCast := outerFunc(fromBaseType, toBaseType); baseCast != nil { + // We use a closure that can unwrap the slice, since conversion functions expect a singular non-nil value + return func(ctx *sql.Context, vals any, targetType *pgtypes.DoltgresType) (any, error) { + var err error + oldVals := vals.([]any) + newVals := make([]any, len(oldVals)) + for i, oldVal := range oldVals { + if oldVal == nil { + continue + } + // Some errors are optional depending on the context, so we'll still process all values even + // after an error is received. + var nErr error + targetBaseType := targetType.ArrayBaseType() + newVals[i], nErr = baseCast(ctx, oldVal, targetBaseType) + if nErr != nil && err == nil { + err = nErr } - return newVals, err } + return newVals, err } } + } return nil } // identityCast returns the input value. -func identityCast(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { +func identityCast(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { return val, nil } // UnknownLiteralCast is used when casting from an unknown literal to any type, as unknown literals are treated special in // some contexts. -func UnknownLiteralCast(ctx *sql.Context, val any, targetType pgtypes.DoltgresType) (any, error) { +func UnknownLiteralCast(ctx *sql.Context, val any, targetType *pgtypes.DoltgresType) (any, error) { if val == nil { return nil, nil } diff --git a/server/functions/framework/catalog.go b/server/functions/framework/catalog.go index d1f53b78cf..03adcadd38 100644 --- a/server/functions/framework/catalog.go +++ b/server/functions/framework/catalog.go @@ -18,6 +18,8 @@ import ( "fmt" "strings" + pgtypes "github.com/dolthub/doltgresql/server/types" + "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/expression/function" ) @@ -65,6 +67,7 @@ func Initialize() { } initializedFunctions = true + pgtypes.LoadFunctionFromCatalog = getQuickFunctionForTypes replaceGmsBuiltIns() validateFunctions() compileFunctions() diff --git a/server/functions/framework/common_type.go b/server/functions/framework/common_type.go index a4e509f560..94789ec9c8 100644 --- a/server/functions/framework/common_type.go +++ b/server/functions/framework/common_type.go @@ -17,54 +17,56 @@ package framework import ( "fmt" + "github.com/lib/pq/oid" + pgtypes "github.com/dolthub/doltgresql/server/types" ) // FindCommonType returns the common type that given types can convert to. // https://www.postgresql.org/docs/15/typeconv-union-case.html -func FindCommonType(types []pgtypes.DoltgresTypeBaseID) (pgtypes.DoltgresTypeBaseID, error) { - var candidateType = pgtypes.DoltgresTypeBaseID_Unknown +func FindCommonType(types []*pgtypes.DoltgresType) (*pgtypes.DoltgresType, error) { + var candidateType = pgtypes.Unknown var fail = false - for _, typBaseID := range types { - if typBaseID == candidateType { + for _, typ := range types { + if typ.OID == candidateType.OID { continue - } else if candidateType == pgtypes.DoltgresTypeBaseID_Unknown { - candidateType = typBaseID + } else if candidateType.OID == uint32(oid.T_unknown) { + candidateType = typ } else { - candidateType = pgtypes.DoltgresTypeBaseID_Unknown + candidateType = pgtypes.Unknown fail = true } } if !fail { - if candidateType == pgtypes.DoltgresTypeBaseID_Unknown { - return pgtypes.DoltgresTypeBaseID_Text, nil + if candidateType.OID == uint32(oid.T_unknown) { + return pgtypes.Text, nil } return candidateType, nil } - for _, typBaseID := range types { - if candidateType == pgtypes.DoltgresTypeBaseID_Unknown { - candidateType = typBaseID + for _, typ := range types { + if candidateType.OID == uint32(oid.T_unknown) { + candidateType = typ } - if typBaseID != pgtypes.DoltgresTypeBaseID_Unknown && candidateType.GetTypeCategory() != typBaseID.GetTypeCategory() { - return 0, fmt.Errorf("types %s and %s cannot be matched", candidateType.GetRepresentativeType().String(), typBaseID.GetRepresentativeType().String()) + if typ.OID != uint32(oid.T_unknown) && candidateType.TypCategory != typ.TypCategory { + return nil, fmt.Errorf("types %s and %s cannot be matched", candidateType.String(), typ.String()) } } var preferredTypeFound = false - for _, typBaseID := range types { - if typBaseID == pgtypes.DoltgresTypeBaseID_Unknown { + for _, typ := range types { + if typ.OID == uint32(oid.T_unknown) { continue - } else if GetImplicitCast(typBaseID, candidateType) != nil { + } else if GetImplicitCast(typ, candidateType) != nil { continue - } else if GetImplicitCast(candidateType, typBaseID) == nil { - return 0, fmt.Errorf("cannot find implicit cast function from %s to %s", candidateType.String(), typBaseID.String()) + } else if GetImplicitCast(candidateType, typ) == nil { + return nil, fmt.Errorf("cannot find implicit cast function from %s to %s", candidateType.String(), typ.String()) } else if !preferredTypeFound { - if candidateType.GetRepresentativeType().IsPreferredType() { - candidateType = typBaseID + if candidateType.IsPreferred { + candidateType = typ preferredTypeFound = true } } else { - return 0, fmt.Errorf("found another preferred candidate type") + return nil, fmt.Errorf("found another preferred candidate type") } } return candidateType, nil diff --git a/server/functions/framework/compiled_catalog.go b/server/functions/framework/compiled_catalog.go index 620d111117..7706d7d096 100644 --- a/server/functions/framework/compiled_catalog.go +++ b/server/functions/framework/compiled_catalog.go @@ -14,9 +14,13 @@ package framework -import "github.com/dolthub/go-mysql-server/sql" +import ( + "github.com/dolthub/go-mysql-server/sql" -// compiledCatalog contains all of the PostgreSQL functions in their compiled forms. + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// compiledCatalog contains all of PostgreSQL functions in their compiled forms. var compiledCatalog = map[string]sql.CreateFuncNArgs{} // GetFunction returns the compiled function with the given name and parameters. Returns false if the function could not @@ -31,3 +35,37 @@ func GetFunction(functionName string, params ...sql.Expression) (*CompiledFuncti } return nil, false, nil } + +// dummyExpression is a simple expression that exists solely to capture type information for a parameter. This is used +// exclusively by the getQuickFunctionForTypes function. +type dummyExpression struct { + t *pgtypes.DoltgresType +} + +var _ sql.Expression = dummyExpression{} + +func (d dummyExpression) Resolved() bool { return true } +func (d dummyExpression) String() string { return d.t.String() } +func (d dummyExpression) Type() sql.Type { return d.t } +func (d dummyExpression) IsNullable() bool { return false } +func (d dummyExpression) Eval(ctx *sql.Context, row sql.Row) (interface{}, error) { + panic("cannot Eval dummyExpression") +} +func (d dummyExpression) Children() []sql.Expression { return nil } +func (d dummyExpression) WithChildren(children ...sql.Expression) (sql.Expression, error) { + return d, nil +} + +// getQuickFunctionForTypes is used by the types package to load quick functions. This is declared here to work around +// import cycles. Returns nil if a QuickFunction could not be constructed. +func getQuickFunctionForTypes(functionName string, params []*pgtypes.DoltgresType) any { + exprs := make([]sql.Expression, len(params)) + for i := range params { + exprs[i] = dummyExpression{t: params[i]} + } + cf, ok, err := GetFunction(functionName, exprs...) + if err != nil || !ok { + return nil + } + return cf.GetQuickFunction() +} diff --git a/server/functions/framework/compiled_function.go b/server/functions/framework/compiled_function.go index d7561958cf..091034d8f7 100644 --- a/server/functions/framework/compiled_function.go +++ b/server/functions/framework/compiled_function.go @@ -20,11 +20,22 @@ import ( "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/expression" - "github.com/dolthub/vitess/go/vt/proto/query" + "github.com/lib/pq/oid" + "gopkg.in/src-d/go-errors.v1" pgtypes "github.com/dolthub/doltgresql/server/types" ) +// ErrFunctionDoesNotExist is returned when the function in use cannot be found. +var ErrFunctionDoesNotExist = errors.NewKind(`function %s does not exist`) + +// Function is an expression that represents either a CompiledFunction or a QuickFunction. +type Function interface { + sql.FunctionExpression + sql.NonDeterministicExpression + specificFuncImpl() +} + // CompiledFunction is an expression that represents a fully-analyzed PostgreSQL function. type CompiledFunction struct { Name string @@ -33,8 +44,8 @@ type CompiledFunction struct { overloads *Overloads fnOverloads []Overload overload overloadMatch - originalTypes []pgtypes.DoltgresType - callResolved []pgtypes.DoltgresType + originalTypes []*pgtypes.DoltgresType + callResolved []*pgtypes.DoltgresType stashedErr error } @@ -76,7 +87,7 @@ func newCompiledFunctionInternal( } // If we do not receive an overload, then the parameters given did not result in a valid match if !overload.Valid() { - c.stashedErr = fmt.Errorf("function %s does not exist", c.OverloadString(originalTypes)) + c.stashedErr = ErrFunctionDoesNotExist.New(c.OverloadString(originalTypes)) return c } @@ -85,25 +96,31 @@ func newCompiledFunctionInternal( // Then we'll handle the polymorphic types // https://www.postgresql.org/docs/15/extend-type-system.html#EXTEND-TYPES-POLYMORPHIC functionParameterTypes := fn.GetParameters() - c.callResolved = make([]pgtypes.DoltgresType, len(functionParameterTypes)+1) + c.callResolved = make([]*pgtypes.DoltgresType, len(functionParameterTypes)+1) hasPolymorphicParam := false for i, param := range functionParameterTypes { - if _, ok := param.(pgtypes.DoltgresPolymorphicType); ok { + if param.IsPolymorphicType() { // resolve will ensure that the parameter types are valid, so we can just assign them here hasPolymorphicParam = true c.callResolved[i] = originalTypes[i] } else { + if d, ok := args[i].Type().(*pgtypes.DoltgresType); ok { + // `param` is a default type which does not have type modifier set + param = param.WithAttTypMod(d.GetAttTypMod()) + } c.callResolved[i] = param } } returnType := fn.GetReturn() c.callResolved[len(c.callResolved)-1] = returnType - if _, ok := returnType.(pgtypes.DoltgresPolymorphicType); ok { + if returnType.IsPolymorphicType() { if hasPolymorphicParam { c.callResolved[len(c.callResolved)-1] = c.resolvePolymorphicReturnType(functionParameterTypes, originalTypes, returnType) + } else if c.Name == "array_in" || c.Name == "array_recv" { + // TODO: `array_in` and `array_recv` functions don't follow this rule + // The return type should resolve to the type of OID value passed in as second argument. } else { - c.stashedErr = fmt.Errorf("A result of type %s requires at least one input of type "+ - "anyelement, anyarray, anynonarray, anyenum, anyrange, or anymultirange.", returnType.String()) + c.stashedErr = fmt.Errorf("A result of type %s requires at least one input of type anyelement, anyarray, anynonarray, anyenum, anyrange, or anymultirange.", returnType.String()) return c } } @@ -162,7 +179,7 @@ func (c *CompiledFunction) String() string { } // OverloadString returns the name of the function represented by the given overload. -func (c *CompiledFunction) OverloadString(types []pgtypes.DoltgresType) string { +func (c *CompiledFunction) OverloadString(types []*pgtypes.DoltgresType) string { sb := strings.Builder{} sb.WriteString(c.Name + "(") for i, t := range types { @@ -199,6 +216,15 @@ func (c *CompiledFunction) IsNonDeterministic() bool { return true } +// IsVariadic returns whether this function has any variadic parameters. +func (c *CompiledFunction) IsVariadic() bool { + if c.overload.Valid() { + return c.overload.params.variadic != -1 + } + // Compilation must have errored, so we'll just return true + return true +} + // Eval implements the interface sql.Expression. func (c *CompiledFunction) Eval(ctx *sql.Context, row sql.Row) (interface{}, error) { // If we have a stashed error, then we should return that now. Errors are stashed when they're supposed to be @@ -207,35 +233,41 @@ func (c *CompiledFunction) Eval(ctx *sql.Context, row sql.Row) (interface{}, err return nil, c.stashedErr } - // Evaluate all of the arguments. - args, err := c.evalArgs(ctx, row) - if err != nil { - return nil, err - } - - if c.overload.Function().IsStrict() { - for i := range args { - if args[i] == nil { - return nil, nil + // Evaluate all arguments, returning immediately if we encounter a null argument and the function is marked STRICT + var err error + isStrict := c.overload.Function().IsStrict() + args := make([]any, len(c.Arguments)) + for i, arg := range c.Arguments { + args[i], err = arg.Eval(ctx, row) + if err != nil { + return nil, err + } + // TODO: once we remove GMS types from all of our expressions, we can remove this step which ensures the correct type + if _, ok := arg.Type().(*pgtypes.DoltgresType); !ok { + dt, err := pgtypes.FromGmsTypeToDoltgresType(arg.Type()) + if err != nil { + return nil, err } + args[i], _, _ = dt.Convert(args[i]) + } + if args[i] == nil && isStrict { + return nil, nil } } - targetParamTypes := c.overload.Function().GetParameters() - if len(c.overload.casts) > 0 { + targetParamTypes := c.overload.Function().GetParameters() for i, arg := range args { // For variadic params, we need to identify the corresponding target type - var targetType pgtypes.DoltgresType + var targetType *pgtypes.DoltgresType isVariadicArg := c.overload.params.variadic >= 0 && i >= len(c.overload.params.paramTypes)-1 if isVariadicArg { targetType = targetParamTypes[c.overload.params.variadic] - targetArrayType, ok := targetType.(pgtypes.DoltgresArrayType) - if !ok { + if !targetType.IsArrayType() { // should be impossible, we check this at function compile time return nil, fmt.Errorf("variadic arguments must be array types, was %T", targetType) } - targetType = targetArrayType.BaseType() + targetType = targetType.ArrayBaseType() } else { targetType = targetParamTypes[i] } @@ -258,13 +290,13 @@ func (c *CompiledFunction) Eval(ctx *sql.Context, row sql.Row) (interface{}, err case Function0: return f.Callable(ctx) case Function1: - return f.Callable(ctx, ([2]pgtypes.DoltgresType)(c.callResolved), args[0]) + return f.Callable(ctx, ([2]*pgtypes.DoltgresType)(c.callResolved), args[0]) case Function2: - return f.Callable(ctx, ([3]pgtypes.DoltgresType)(c.callResolved), args[0], args[1]) + return f.Callable(ctx, ([3]*pgtypes.DoltgresType)(c.callResolved), args[0], args[1]) case Function3: - return f.Callable(ctx, ([4]pgtypes.DoltgresType)(c.callResolved), args[0], args[1], args[2]) + return f.Callable(ctx, ([4]*pgtypes.DoltgresType)(c.callResolved), args[0], args[1], args[2]) case Function4: - return f.Callable(ctx, ([5]pgtypes.DoltgresType)(c.callResolved), args[0], args[1], args[2], args[3]) + return f.Callable(ctx, ([5]*pgtypes.DoltgresType)(c.callResolved), args[0], args[1], args[2], args[3]) default: return nil, fmt.Errorf("unknown function type in CompiledFunction::Eval") } @@ -285,28 +317,59 @@ func (c *CompiledFunction) WithChildren(children ...sql.Expression) (sql.Express return newCompiledFunctionInternal(c.Name, children, c.overloads, c.fnOverloads, c.IsOperator), nil } +// GetQuickFunction returns the QuickFunction form of this function, if it exists. If one does not exist, then this +// return nil. +func (c *CompiledFunction) GetQuickFunction() QuickFunction { + if c.stashedErr != nil || !c.Resolved() || !c.overload.Valid() || c.overload.params.variadic != -1 || + len(c.overload.casts) > 0 { + return nil + } + switch f := c.overload.Function().(type) { + case Function1: + return &QuickFunction1{ + Name: c.Name, + Argument: c.Arguments[0], + IsStrict: c.overload.Function().IsStrict(), + callResolved: ([2]*pgtypes.DoltgresType)(c.callResolved), + function: f, + } + case Function2: + return &QuickFunction2{ + Name: c.Name, + Arguments: ([2]sql.Expression)(c.Arguments), + IsStrict: c.overload.Function().IsStrict(), + callResolved: ([3]*pgtypes.DoltgresType)(c.callResolved), + function: f, + } + case Function3: + return &QuickFunction3{ + Name: c.Name, + Arguments: ([3]sql.Expression)(c.Arguments), + IsStrict: c.overload.Function().IsStrict(), + callResolved: ([4]*pgtypes.DoltgresType)(c.callResolved), + function: f, + } + default: + return nil + } +} + // resolve returns an overloadMatch that either matches the given parameters exactly, or is a viable match after casting. // Returns an invalid overloadMatch if a viable match is not found. -func (c *CompiledFunction) resolve( - overloads *Overloads, - fnOverloads []Overload, - argTypes []pgtypes.DoltgresType, -) (overloadMatch, error) { - +func (c *CompiledFunction) resolve(overloads *Overloads, fnOverloads []Overload, argTypes []*pgtypes.DoltgresType) (overloadMatch, error) { // First check for an exact match - exactMatch, found := overloads.ExactMatchForTypes(argTypes) + exactMatch, found := overloads.ExactMatchForTypes(argTypes...) if found { - baseTypes := overloads.baseIdsForTypes(argTypes) return overloadMatch{ params: Overload{ function: exactMatch, - paramTypes: baseTypes, - argTypes: baseTypes, + paramTypes: argTypes, + argTypes: argTypes, variadic: -1, }, }, nil } - // There are no exact matches, so now we'll look through all of the overloads to determine the best match. This is + // There are no exact matches, so now we'll look through all overloads to determine the best match. This is // much more work, but there's a performance penalty for runtime overload resolution in Postgres as well. if c.IsOperator { return c.resolveOperator(argTypes, overloads, fnOverloads) @@ -317,28 +380,28 @@ func (c *CompiledFunction) resolve( // resolveOperator resolves an operator according to the rules defined by Postgres. // https://www.postgresql.org/docs/15/typeconv-oper.html -func (c *CompiledFunction) resolveOperator(argTypes []pgtypes.DoltgresType, overloads *Overloads, fnOverloads []Overload) (overloadMatch, error) { +func (c *CompiledFunction) resolveOperator(argTypes []*pgtypes.DoltgresType, overloads *Overloads, fnOverloads []Overload) (overloadMatch, error) { // Binary operators treat unknown literals as the other type, so we'll account for that here to see if we can find // an "exact" match. if len(argTypes) == 2 { - leftUnknownType := argTypes[0].BaseID() == pgtypes.DoltgresTypeBaseID_Unknown - rightUnknownType := argTypes[1].BaseID() == pgtypes.DoltgresTypeBaseID_Unknown + leftUnknownType := argTypes[0].OID == uint32(oid.T_unknown) + rightUnknownType := argTypes[1].OID == uint32(oid.T_unknown) if (leftUnknownType && !rightUnknownType) || (!leftUnknownType && rightUnknownType) { - var baseID pgtypes.DoltgresTypeBaseID + var typ *pgtypes.DoltgresType casts := []TypeCastFunction{identityCast, identityCast} if leftUnknownType { casts[0] = UnknownLiteralCast - baseID = argTypes[1].BaseID() + typ = argTypes[1] } else { casts[1] = UnknownLiteralCast - baseID = argTypes[0].BaseID() + typ = argTypes[0] } - if exactMatch, ok := overloads.ExactMatchForBaseIds(baseID, baseID); ok { + if exactMatch, ok := overloads.ExactMatchForTypes(typ, typ); ok { return overloadMatch{ params: Overload{ function: exactMatch, - paramTypes: []pgtypes.DoltgresTypeBaseID{baseID, baseID}, - argTypes: []pgtypes.DoltgresTypeBaseID{baseID, baseID}, + paramTypes: []*pgtypes.DoltgresType{typ, typ}, + argTypes: []*pgtypes.DoltgresType{typ, typ}, variadic: -1, }, casts: casts, @@ -352,7 +415,7 @@ func (c *CompiledFunction) resolveOperator(argTypes []pgtypes.DoltgresType, over // resolveFunction resolves a function according to the rules defined by Postgres. // https://www.postgresql.org/docs/15/typeconv-func.html -func (c *CompiledFunction) resolveFunction(argTypes []pgtypes.DoltgresType, overloads []Overload) (overloadMatch, error) { +func (c *CompiledFunction) resolveFunction(argTypes []*pgtypes.DoltgresType, overloads []Overload) (overloadMatch, error) { // First we'll discard all overloads that do not have implicitly-convertible param types compatibleOverloads := c.typeCompatibleOverloads(overloads, argTypes) @@ -402,24 +465,23 @@ func (c *CompiledFunction) resolveFunction(argTypes []pgtypes.DoltgresType, over // typeCompatibleOverloads returns all overloads that have a matching number of params whose types can be // implicitly converted to the ones provided. This is the set of all possible overloads that could be used with the // param types provided. -func (c *CompiledFunction) typeCompatibleOverloads(fnOverloads []Overload, argTypes []pgtypes.DoltgresType) []overloadMatch { +func (c *CompiledFunction) typeCompatibleOverloads(fnOverloads []Overload, argTypes []*pgtypes.DoltgresType) []overloadMatch { var compatible []overloadMatch for _, overload := range fnOverloads { isConvertible := true overloadCasts := make([]TypeCastFunction, len(argTypes)) // Polymorphic parameters must be gathered so that we can later verify that they all have matching base types - var polymorphicParameters []pgtypes.DoltgresType - var polymorphicTargets []pgtypes.DoltgresType + var polymorphicParameters []*pgtypes.DoltgresType + var polymorphicTargets []*pgtypes.DoltgresType for i := range argTypes { paramType := overload.argTypes[i] - - if polymorphicType, ok := paramType.GetRepresentativeType().(pgtypes.DoltgresPolymorphicType); ok && polymorphicType.IsValid(argTypes[i]) { + if paramType.IsValidForPolymorphicType(argTypes[i]) { overloadCasts[i] = identityCast - polymorphicParameters = append(polymorphicParameters, polymorphicType) + polymorphicParameters = append(polymorphicParameters, paramType) polymorphicTargets = append(polymorphicTargets, argTypes[i]) } else { - if overloadCasts[i] = GetImplicitCast(argTypes[i].BaseID(), paramType); overloadCasts[i] == nil { - if argTypes[i].BaseID() == pgtypes.DoltgresTypeBaseID_Unknown { + if overloadCasts[i] = GetImplicitCast(argTypes[i], paramType); overloadCasts[i] == nil { + if argTypes[i].OID == uint32(oid.T_unknown) { overloadCasts[i] = UnknownLiteralCast } else { isConvertible = false @@ -438,16 +500,14 @@ func (c *CompiledFunction) typeCompatibleOverloads(fnOverloads []Overload, argTy // closestTypeMatches returns the set of overload candidates that have the most exact type matches for the arg types // provided. -func (*CompiledFunction) closestTypeMatches(argTypes []pgtypes.DoltgresType, candidates []overloadMatch) []overloadMatch { +func (*CompiledFunction) closestTypeMatches(argTypes []*pgtypes.DoltgresType, candidates []overloadMatch) []overloadMatch { matchCount := 0 var matches []overloadMatch for _, cand := range candidates { currentMatchCount := 0 for argIdx := range argTypes { argType := cand.params.argTypes[argIdx] - - argBaseId := argTypes[argIdx].BaseID() - if argBaseId == argType || argBaseId == pgtypes.DoltgresTypeBaseID_Unknown { + if argTypes[argIdx].OID == argType.OID || argTypes[argIdx].OID == uint32(oid.T_unknown) { currentMatchCount++ } } @@ -462,15 +522,14 @@ func (*CompiledFunction) closestTypeMatches(argTypes []pgtypes.DoltgresType, can } // preferredTypeMatches returns the overload candidates that have the most preferred types for args that require casts. -func (*CompiledFunction) preferredTypeMatches(argTypes []pgtypes.DoltgresType, candidates []overloadMatch) []overloadMatch { +func (*CompiledFunction) preferredTypeMatches(argTypes []*pgtypes.DoltgresType, candidates []overloadMatch) []overloadMatch { preferredCount := 0 var preferredOverloads []overloadMatch for _, cand := range candidates { currentPreferredCount := 0 for argIdx := range argTypes { argType := cand.params.argTypes[argIdx] - - if argTypes[argIdx].BaseID() != argType && argType.GetTypeCategory().IsPreferredType(argType) { + if argTypes[argIdx].OID != argType.OID && argType.IsPreferred { currentPreferredCount++ } } @@ -487,18 +546,18 @@ func (*CompiledFunction) preferredTypeMatches(argTypes []pgtypes.DoltgresType, c // unknownTypeCategoryMatches checks the type categories of `unknown` types. These types have an inherent bias toward // the string category since an `unknown` literal resembles a string. Returns false if the resolution should fail. -func (c *CompiledFunction) unknownTypeCategoryMatches(argTypes []pgtypes.DoltgresType, candidates []overloadMatch) ([]overloadMatch, bool) { +func (c *CompiledFunction) unknownTypeCategoryMatches(argTypes []*pgtypes.DoltgresType, candidates []overloadMatch) ([]overloadMatch, bool) { matches := make([]overloadMatch, len(candidates)) copy(matches, candidates) // For our first loop, we'll filter matches based on whether they accept the string category for argIdx := range argTypes { // We're only concerned with `unknown` types - if argTypes[argIdx].BaseID() != pgtypes.DoltgresTypeBaseID_Unknown { + if argTypes[argIdx].OID != uint32(oid.T_unknown) { continue } var newMatches []overloadMatch for _, match := range matches { - if match.params.argTypes[argIdx].GetTypeCategory() == pgtypes.TypeCategory_StringTypes { + if match.params.argTypes[argIdx].TypCategory == pgtypes.TypeCategory_StringTypes { newMatches = append(newMatches, match) } } @@ -514,7 +573,7 @@ func (c *CompiledFunction) unknownTypeCategoryMatches(argTypes []pgtypes.Doltgre // TODO: implement the remainder of step 4.e. from the documentation (following code assumes it has been implemented) // ... - // If we've discarded every function, then we'll actually return all of the original candidates + // If we've discarded every function, then we'll actually return all original candidates if len(matches) == 0 { return candidates, true } @@ -523,7 +582,7 @@ func (c *CompiledFunction) unknownTypeCategoryMatches(argTypes []pgtypes.Doltgre } // polymorphicTypesCompatible returns whether any polymorphic types given are compatible with the expression types given -func (*CompiledFunction) polymorphicTypesCompatible(paramTypes []pgtypes.DoltgresType, exprTypes []pgtypes.DoltgresType) bool { +func (*CompiledFunction) polymorphicTypesCompatible(paramTypes []*pgtypes.DoltgresType, exprTypes []*pgtypes.DoltgresType) bool { if len(paramTypes) != len(exprTypes) { return false } @@ -534,12 +593,12 @@ func (*CompiledFunction) polymorphicTypesCompatible(paramTypes []pgtypes.Doltgre // If one of the types is anyarray, then anyelement behaves as anynonarray, so we can convert them to anynonarray for _, paramType := range paramTypes { - if polymorphicParamType, ok := paramType.(pgtypes.DoltgresPolymorphicType); ok && polymorphicParamType.BaseID() == pgtypes.DoltgresTypeBaseID_AnyArray { + if paramType.OID == uint32(oid.T_anyarray) { // At least one parameter is anyarray, so copy all parameters to a new slice and replace anyelement with anynonarray - newParamTypes := make([]pgtypes.DoltgresType, len(paramTypes)) + newParamTypes := make([]*pgtypes.DoltgresType, len(paramTypes)) copy(newParamTypes, paramTypes) for i := range newParamTypes { - if paramTypes[i].BaseID() == pgtypes.DoltgresTypeBaseID_AnyElement { + if paramTypes[i].OID == uint32(oid.T_anyelement) { newParamTypes[i] = pgtypes.AnyNonArray } } @@ -549,24 +608,24 @@ func (*CompiledFunction) polymorphicTypesCompatible(paramTypes []pgtypes.Doltgre } // The base type is the type that must match between all polymorphic types. - var baseType pgtypes.DoltgresType + var baseType *pgtypes.DoltgresType for i, paramType := range paramTypes { - if polymorphicParamType, ok := paramType.(pgtypes.DoltgresPolymorphicType); ok && exprTypes[i].BaseID() != pgtypes.DoltgresTypeBaseID_Unknown { + if paramType.IsPolymorphicType() && exprTypes[i].OID != uint32(oid.T_unknown) { // Although we do this check before we ever reach this function, we do it again as we may convert anyelement // to anynonarray, which changes type validity - if !polymorphicParamType.IsValid(exprTypes[i]) { + if !paramType.IsValidForPolymorphicType(exprTypes[i]) { return false } // Get the base expression type that we'll compare against baseExprType := exprTypes[i] - if arrayBaseExprType, ok := baseExprType.(pgtypes.DoltgresArrayType); ok { - baseExprType = arrayBaseExprType.BaseType() + if baseExprType.IsArrayType() { + baseExprType = baseExprType.ArrayBaseType() } // TODO: handle range types // Check that the base expression type matches the previously-found base type - if baseType == nil { + if baseType.IsEmptyType() { baseType = baseExprType - } else if baseType.BaseID() != baseExprType.BaseID() { + } else if baseType.OID != baseExprType.OID { return false } } @@ -578,135 +637,72 @@ func (*CompiledFunction) polymorphicTypesCompatible(paramTypes []pgtypes.Doltgre // polymorphic type, then the return type is directly returned. However, if the return type is a polymorphic type, then // the type is determined using the expression types and parameter types. This makes the assumption that everything has // already been validated. -func (c *CompiledFunction) resolvePolymorphicReturnType(functionInterfaceTypes []pgtypes.DoltgresType, originalTypes []pgtypes.DoltgresType, returnType pgtypes.DoltgresType) pgtypes.DoltgresType { - polymorphicReturnType, ok := returnType.(pgtypes.DoltgresPolymorphicType) - if !ok { +func (c *CompiledFunction) resolvePolymorphicReturnType(functionInterfaceTypes []*pgtypes.DoltgresType, originalTypes []*pgtypes.DoltgresType, returnType *pgtypes.DoltgresType) *pgtypes.DoltgresType { + if !returnType.IsPolymorphicType() { return returnType } // We can use the first polymorphic non-unknown type that we find, since we can morph it into any type that we need. // We've verified that all polymorphic types are compatible in a previous step, so this is safe to do. - var firstPolymorphicType pgtypes.DoltgresType + var firstPolymorphicType *pgtypes.DoltgresType for i, functionInterfaceType := range functionInterfaceTypes { - if _, ok = functionInterfaceType.(pgtypes.DoltgresPolymorphicType); ok && originalTypes[i].BaseID() != pgtypes.DoltgresTypeBaseID_Unknown { + if functionInterfaceType.IsPolymorphicType() && originalTypes[i].OID != uint32(oid.T_unknown) { firstPolymorphicType = originalTypes[i] break } } // if all types are `unknown`, use `text` type - if firstPolymorphicType == nil { + if firstPolymorphicType.IsEmptyType() { firstPolymorphicType = pgtypes.Text } - switch polymorphicReturnType.BaseID() { - case pgtypes.DoltgresTypeBaseID_AnyElement, pgtypes.DoltgresTypeBaseID_AnyNonArray: + switch oid.Oid(returnType.OID) { + case oid.T_anyelement, oid.T_anynonarray: // For return types, anyelement behaves the same as anynonarray. // This isn't explicitly in the documentation, however it does note that: // "...anynonarray and anyenum do not represent separate type variables; they are the same type as anyelement..." // The implication of this being that anyelement will always return the base type even for array types, // just like anynonarray would. - if minimalArrayType, ok := firstPolymorphicType.(pgtypes.DoltgresArrayType); ok { - return minimalArrayType.BaseType() + if firstPolymorphicType.IsArrayType() { + return firstPolymorphicType.ArrayBaseType() } else { return firstPolymorphicType } - case pgtypes.DoltgresTypeBaseID_AnyArray: + case oid.T_anyarray: // Array types will return themselves, so this is safe - return firstPolymorphicType.ToArrayType() - default: - panic(fmt.Errorf("`%s` is not yet handled during function compilation", polymorphicReturnType.String())) - } -} - -// evalArgs evaluates the function args within an Eval call. -func (c *CompiledFunction) evalArgs(ctx *sql.Context, row sql.Row) ([]any, error) { - args := make([]any, len(c.Arguments)) - for i, arg := range c.Arguments { - var err error - args[i], err = arg.Eval(ctx, row) - if err != nil { - return nil, err - } - // TODO: once we remove GMS types from all of our expressions, we can remove this step which ensures the correct type - if _, ok := arg.Type().(pgtypes.DoltgresType); !ok { - switch arg.Type().Type() { - case query.Type_INT8, query.Type_INT16: - args[i], _, _ = pgtypes.Int16.Convert(args[i]) - case query.Type_INT24, query.Type_INT32: - args[i], _, _ = pgtypes.Int32.Convert(args[i]) - case query.Type_INT64: - args[i], _, _ = pgtypes.Int64.Convert(args[i]) - case query.Type_UINT8, query.Type_UINT16, query.Type_UINT24, query.Type_UINT32, query.Type_UINT64: - args[i], _, _ = pgtypes.Int64.Convert(args[i]) - case query.Type_YEAR: - args[i], _, _ = pgtypes.Int16.Convert(args[i]) - case query.Type_FLOAT32: - args[i], _, _ = pgtypes.Float32.Convert(args[i]) - case query.Type_FLOAT64: - args[i], _, _ = pgtypes.Float64.Convert(args[i]) - case query.Type_DECIMAL: - args[i], _, _ = pgtypes.Numeric.Convert(args[i]) - case query.Type_DATE: - args[i], _, _ = pgtypes.Date.Convert(args[i]) - case query.Type_DATETIME, query.Type_TIMESTAMP: - args[i], _, _ = pgtypes.Timestamp.Convert(args[i]) - case query.Type_CHAR, query.Type_VARCHAR, query.Type_TEXT: - args[i], _, _ = pgtypes.Text.Convert(args[i]) - case query.Type_ENUM: - args[i], _, _ = pgtypes.Int16.Convert(args[i]) - case query.Type_SET: - args[i], _, _ = pgtypes.Int64.Convert(args[i]) - default: - return nil, fmt.Errorf("encountered a GMS type that cannot be handled") - } + if firstPolymorphicType.IsArrayType() { + return firstPolymorphicType + } else if firstPolymorphicType.OID == uint32(oid.T_internal) { + return pgtypes.OidToBuiltInDoltgresType[firstPolymorphicType.BaseTypeForInternal] + } else { + return firstPolymorphicType.ToArrayType() } + default: + panic(fmt.Errorf("`%s` is not yet handled during function compilation", returnType.String())) } - return args, nil } // analyzeParameters analyzes the parameters within an Eval call. -func (c *CompiledFunction) analyzeParameters() (originalTypes []pgtypes.DoltgresType, err error) { - originalTypes = make([]pgtypes.DoltgresType, len(c.Arguments)) +func (c *CompiledFunction) analyzeParameters() (originalTypes []*pgtypes.DoltgresType, err error) { + originalTypes = make([]*pgtypes.DoltgresType, len(c.Arguments)) for i, param := range c.Arguments { returnType := param.Type() - if extendedType, ok := returnType.(pgtypes.DoltgresType); ok { - if domainType, ok := extendedType.(pgtypes.DomainType); ok { - extendedType = domainType.UnderlyingBaseType() + if extendedType, ok := returnType.(*pgtypes.DoltgresType); ok && !extendedType.IsEmptyType() { + if extendedType.TypType == pgtypes.TypeType_Domain { + extendedType = extendedType.DomainUnderlyingBaseType() } originalTypes[i] = extendedType } else { // TODO: we need to remove GMS types from all of our expressions so that we can remove this - switch param.Type().Type() { - case query.Type_INT8, query.Type_INT16: - originalTypes[i] = pgtypes.Int16 - case query.Type_INT24, query.Type_INT32: - originalTypes[i] = pgtypes.Int32 - case query.Type_INT64: - originalTypes[i] = pgtypes.Int64 - case query.Type_UINT8, query.Type_UINT16, query.Type_UINT24, query.Type_UINT32, query.Type_UINT64: - originalTypes[i] = pgtypes.Int64 - case query.Type_YEAR: - originalTypes[i] = pgtypes.Int16 - case query.Type_FLOAT32: - originalTypes[i] = pgtypes.Float32 - case query.Type_FLOAT64: - originalTypes[i] = pgtypes.Float64 - case query.Type_DECIMAL: - originalTypes[i] = pgtypes.Numeric - case query.Type_DATE, query.Type_DATETIME, query.Type_TIMESTAMP: - originalTypes[i] = pgtypes.Timestamp - case query.Type_CHAR, query.Type_VARCHAR, query.Type_TEXT: - originalTypes[i] = pgtypes.Text - case query.Type_ENUM: - originalTypes[i] = pgtypes.Int16 - case query.Type_SET: - originalTypes[i] = pgtypes.Int64 - case query.Type_NULL_TYPE: - originalTypes[i] = pgtypes.Unknown - default: - return nil, fmt.Errorf("encountered a type that does not conform to the DoltgresType interface: %T", param.Type()) + dt, err := pgtypes.FromGmsTypeToDoltgresType(param.Type()) + if err != nil { + return nil, err } + originalTypes[i] = dt } } return originalTypes, nil } + +// specificFuncImpl implements the interface sql.Expression. +func (*CompiledFunction) specificFuncImpl() {} diff --git a/server/functions/framework/functions.go b/server/functions/framework/functions.go index eed58dda28..d57551d53e 100644 --- a/server/functions/framework/functions.go +++ b/server/functions/framework/functions.go @@ -25,9 +25,9 @@ type FunctionInterface interface { // GetName returns the name of the function. The name is case-insensitive, so the casing does not matter. GetName() string // GetReturn returns the return type. - GetReturn() pgtypes.DoltgresType + GetReturn() *pgtypes.DoltgresType // GetParameters returns the parameter types for the function. - GetParameters() []pgtypes.DoltgresType + GetParameters() []*pgtypes.DoltgresType // VariadicIndex returns the index of the variadic parameter, if it exists, or -1 otherwise VariadicIndex() int // GetExpectedParameterCount returns the number of parameters that are valid for this function. @@ -44,7 +44,7 @@ type FunctionInterface interface { // Function0 is a function that does not take any parameters. type Function0 struct { Name string - Return pgtypes.DoltgresType + Return *pgtypes.DoltgresType IsNonDeterministic bool Strict bool Callable func(ctx *sql.Context) (any, error) @@ -54,12 +54,12 @@ type Function0 struct { // when the parameter (and possibly return type) is a polymorphic type. The return type is the last type in the array. type Function1 struct { Name string - Return pgtypes.DoltgresType - Parameters [1]pgtypes.DoltgresType + Return *pgtypes.DoltgresType + Parameters [1]*pgtypes.DoltgresType Variadic bool IsNonDeterministic bool Strict bool - Callable func(ctx *sql.Context, paramsAndReturn [2]pgtypes.DoltgresType, val1 any) (any, error) + Callable func(ctx *sql.Context, paramsAndReturn [2]*pgtypes.DoltgresType, val1 any) (any, error) } // Function2 is a function that takes two parameters. The parameter and return types are passed into the Callable @@ -67,12 +67,12 @@ type Function1 struct { // last type in the array. type Function2 struct { Name string - Return pgtypes.DoltgresType - Parameters [2]pgtypes.DoltgresType + Return *pgtypes.DoltgresType + Parameters [2]*pgtypes.DoltgresType Variadic bool IsNonDeterministic bool Strict bool - Callable func(ctx *sql.Context, paramsAndReturn [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) + Callable func(ctx *sql.Context, paramsAndReturn [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) } // Function3 is a function that takes three parameters. The parameter and return types are passed into the Callable @@ -80,12 +80,12 @@ type Function2 struct { // last type in the array. type Function3 struct { Name string - Return pgtypes.DoltgresType - Parameters [3]pgtypes.DoltgresType + Return *pgtypes.DoltgresType + Parameters [3]*pgtypes.DoltgresType Variadic bool IsNonDeterministic bool Strict bool - Callable func(ctx *sql.Context, paramsAndReturn [4]pgtypes.DoltgresType, val1 any, val2 any, val3 any) (any, error) + Callable func(ctx *sql.Context, paramsAndReturn [4]*pgtypes.DoltgresType, val1 any, val2 any, val3 any) (any, error) } // Function4 is a function that takes four parameters. The parameter and return types are passed into the Callable @@ -93,12 +93,12 @@ type Function3 struct { // last type in the array. type Function4 struct { Name string - Return pgtypes.DoltgresType - Parameters [4]pgtypes.DoltgresType + Return *pgtypes.DoltgresType + Parameters [4]*pgtypes.DoltgresType Variadic bool IsNonDeterministic bool Strict bool - Callable func(ctx *sql.Context, paramsAndReturn [5]pgtypes.DoltgresType, val1 any, val2 any, val3 any, val4 any) (any, error) + Callable func(ctx *sql.Context, paramsAndReturn [5]*pgtypes.DoltgresType, val1 any, val2 any, val3 any, val4 any) (any, error) } var _ FunctionInterface = Function0{} @@ -111,10 +111,10 @@ var _ FunctionInterface = Function4{} func (f Function0) GetName() string { return f.Name } // GetReturn implements the FunctionInterface interface. -func (f Function0) GetReturn() pgtypes.DoltgresType { return f.Return } +func (f Function0) GetReturn() *pgtypes.DoltgresType { return f.Return } // GetParameters implements the FunctionInterface interface. -func (f Function0) GetParameters() []pgtypes.DoltgresType { return nil } +func (f Function0) GetParameters() []*pgtypes.DoltgresType { return nil } func (f Function0) VariadicIndex() int { return -1 @@ -136,10 +136,10 @@ func (f Function0) enforceInterfaceInheritance(error) {} func (f Function1) GetName() string { return f.Name } // GetReturn implements the FunctionInterface interface. -func (f Function1) GetReturn() pgtypes.DoltgresType { return f.Return } +func (f Function1) GetReturn() *pgtypes.DoltgresType { return f.Return } // GetParameters implements the FunctionInterface interface. -func (f Function1) GetParameters() []pgtypes.DoltgresType { return f.Parameters[:] } +func (f Function1) GetParameters() []*pgtypes.DoltgresType { return f.Parameters[:] } // VariadicIndex implements the FunctionInterface interface. func (f Function1) VariadicIndex() int { @@ -166,10 +166,10 @@ func (f Function1) enforceInterfaceInheritance(error) {} func (f Function2) GetName() string { return f.Name } // GetReturn implements the FunctionInterface interface. -func (f Function2) GetReturn() pgtypes.DoltgresType { return f.Return } +func (f Function2) GetReturn() *pgtypes.DoltgresType { return f.Return } // GetParameters implements the FunctionInterface interface. -func (f Function2) GetParameters() []pgtypes.DoltgresType { return f.Parameters[:] } +func (f Function2) GetParameters() []*pgtypes.DoltgresType { return f.Parameters[:] } // VariadicIndex implements the FunctionInterface interface. func (f Function2) VariadicIndex() int { @@ -196,10 +196,10 @@ func (f Function2) enforceInterfaceInheritance(error) {} func (f Function3) GetName() string { return f.Name } // GetReturn implements the FunctionInterface interface. -func (f Function3) GetReturn() pgtypes.DoltgresType { return f.Return } +func (f Function3) GetReturn() *pgtypes.DoltgresType { return f.Return } // GetParameters implements the FunctionInterface interface. -func (f Function3) GetParameters() []pgtypes.DoltgresType { return f.Parameters[:] } +func (f Function3) GetParameters() []*pgtypes.DoltgresType { return f.Parameters[:] } // VariadicIndex implements the FunctionInterface interface. func (f Function3) VariadicIndex() int { @@ -226,10 +226,10 @@ func (f Function3) enforceInterfaceInheritance(error) {} func (f Function4) GetName() string { return f.Name } // GetReturn implements the FunctionInterface interface. -func (f Function4) GetReturn() pgtypes.DoltgresType { return f.Return } +func (f Function4) GetReturn() *pgtypes.DoltgresType { return f.Return } // GetParameters implements the FunctionInterface interface. -func (f Function4) GetParameters() []pgtypes.DoltgresType { return f.Parameters[:] } +func (f Function4) GetParameters() []*pgtypes.DoltgresType { return f.Parameters[:] } // VariadicIndex implements the FunctionInterface interface. func (f Function4) VariadicIndex() int { diff --git a/server/functions/framework/literal.go b/server/functions/framework/literal.go index 497a120674..7f3938b158 100644 --- a/server/functions/framework/literal.go +++ b/server/functions/framework/literal.go @@ -23,6 +23,6 @@ import ( // LiteralInterface is used to prevent import cycles, since we can't reference pgexprs.Literal from this package. type LiteralInterface interface { sql.Expression - GetDoltgresType() pgtypes.DoltgresType + GetDoltgresType() *pgtypes.DoltgresType ConformsToLiteralInterface() } diff --git a/server/functions/framework/operators.go b/server/functions/framework/operators.go index 7c8292b86b..7f3320d8ca 100644 --- a/server/functions/framework/operators.go +++ b/server/functions/framework/operators.go @@ -16,8 +16,6 @@ package framework import ( "fmt" - - pgtypes "github.com/dolthub/doltgresql/server/types" ) // Operator is a unary or binary operator. @@ -57,14 +55,14 @@ const ( // unaryFunction represents the signature for a unary function. type unaryFunction struct { Operator Operator - Type pgtypes.DoltgresTypeBaseID + TypeOid uint32 } // binaryFunction represents the signature for a binary function. type binaryFunction struct { Operator Operator - Left pgtypes.DoltgresTypeBaseID - Right pgtypes.DoltgresTypeBaseID + Left uint32 + Right uint32 } var ( @@ -94,7 +92,7 @@ func RegisterUnaryFunction(operator Operator, f Function1) { RegisterFunction(f) sig := unaryFunction{ Operator: operator, - Type: f.Parameters[0].BaseID(), + TypeOid: f.Parameters[0].OID, } if existingFunction, ok := unaryFunctions[sig]; ok { panic(fmt.Errorf("duplicate unary function for `%s`: `%s` and `%s`", @@ -113,8 +111,8 @@ func RegisterBinaryFunction(operator Operator, f Function2) { RegisterFunction(f) sig := binaryFunction{ Operator: operator, - Left: f.Parameters[0].BaseID(), - Right: f.Parameters[1].BaseID(), + Left: f.Parameters[0].OID, + Right: f.Parameters[1].OID, } if existingFunction, ok := binaryFunctions[sig]; ok { panic(fmt.Errorf("duplicate binary function for `%s`: `%s` and `%s`", diff --git a/server/functions/framework/overloads.go b/server/functions/framework/overloads.go index 1042e9dea2..4db9cf9e1d 100644 --- a/server/functions/framework/overloads.go +++ b/server/functions/framework/overloads.go @@ -47,7 +47,7 @@ func (o *Overloads) Add(function FunctionInterface) error { if function.VariadicIndex() >= 0 { varArgsType := function.GetParameters()[function.VariadicIndex()] - if _, ok := varArgsType.(pgtypes.DoltgresArrayType); !ok { + if !varArgsType.IsArrayType() { return fmt.Errorf("variadic parameter must be an array type for function `%s`", function.GetName()) } } @@ -58,19 +58,7 @@ func (o *Overloads) Add(function FunctionInterface) error { } // keyForParamTypes returns a string key to match an overload with the given parameter types. -func keyForParamTypes(types []pgtypes.DoltgresType) string { - sb := strings.Builder{} - for i, typ := range types { - if i > 0 { - sb.WriteByte(',') - } - sb.WriteString(typ.BaseID().String()) - } - return sb.String() -} - -// keyForParamTypes returns a string key to match an overload with the given parameter types. -func keyForBaseIds(types []pgtypes.DoltgresTypeBaseID) string { +func keyForParamTypes(types []*pgtypes.DoltgresType) string { sb := strings.Builder{} for i, typ := range types { if i > 0 { @@ -81,26 +69,17 @@ func keyForBaseIds(types []pgtypes.DoltgresTypeBaseID) string { return sb.String() } -// baseIdsForTypes returns the base IDs of the given types. -func (o *Overloads) baseIdsForTypes(types []pgtypes.DoltgresType) []pgtypes.DoltgresTypeBaseID { - baseIds := make([]pgtypes.DoltgresTypeBaseID, len(types)) - for i, t := range types { - baseIds[i] = t.BaseID() - } - return baseIds -} - // overloadsForParams returns all overloads matching the number of params given, without regard for types. func (o *Overloads) overloadsForParams(numParams int) []Overload { results := make([]Overload, 0, len(o.AllOverloads)) for _, overload := range o.AllOverloads { - params := o.baseIdsForTypes(overload.GetParameters()) + params := overload.GetParameters() variadicIndex := overload.VariadicIndex() if variadicIndex >= 0 && len(params) <= numParams { // Variadic functions may only match when the function is declared with parameters that are fewer or equal // to our target length. If our target length is less, then we cannot expand, so we do not treat it as // variadic. - extendedParams := make([]pgtypes.DoltgresTypeBaseID, numParams) + extendedParams := make([]*pgtypes.DoltgresType, numParams) copy(extendedParams, params[:variadicIndex]) // This is copying the parameters after the variadic index, so we need to add 1. We subtract the declared // parameter count from the target parameter count to obtain the additional parameter count. @@ -108,7 +87,7 @@ func (o *Overloads) overloadsForParams(numParams int) []Overload { copy(extendedParams[firstValueAfterVariadic:], params[variadicIndex+1:]) // ToArrayType immediately followed by BaseType is a way to get the base type without having to cast. // For array types, ToArrayType causes them to return themselves. - variadicBaseType := overload.GetParameters()[variadicIndex].ToArrayType().BaseType().BaseID() + variadicBaseType := overload.GetParameters()[variadicIndex].ToArrayType().ArrayBaseType() for variadicParamIdx := 0; variadicParamIdx < 1+(numParams-len(params)); variadicParamIdx++ { extendedParams[variadicParamIdx+variadicIndex] = variadicBaseType } @@ -132,30 +111,22 @@ func (o *Overloads) overloadsForParams(numParams int) []Overload { // ExactMatchForTypes returns the function that exactly matches the given parameter types, or nil if no overload with // those types exists. -func (o *Overloads) ExactMatchForTypes(types []pgtypes.DoltgresType) (FunctionInterface, bool) { +func (o *Overloads) ExactMatchForTypes(types ...*pgtypes.DoltgresType) (FunctionInterface, bool) { key := keyForParamTypes(types) fn, ok := o.ByParamType[key] return fn, ok } -// ExactMatchForBaseIds returns the function that exactly matches the given parameter types, or nil if no overload with -// those types exists. -func (o *Overloads) ExactMatchForBaseIds(types ...pgtypes.DoltgresTypeBaseID) (FunctionInterface, bool) { - key := keyForBaseIds(types) - fn, ok := o.ByParamType[key] - return fn, ok -} - // Overload is a single overload of a given function, used during evaluation to match the arguments provided // to a particular overload. type Overload struct { // function is the actual function to call to invoke this overload function FunctionInterface // paramTypes is the base IDs of the parameters that the function expects - paramTypes []pgtypes.DoltgresTypeBaseID + paramTypes []*pgtypes.DoltgresType // argTypes is the base IDs of the parameters that the function expects, extended to match the number of args // provided in the case of a variadic function. - argTypes []pgtypes.DoltgresTypeBaseID + argTypes []*pgtypes.DoltgresType // variadic is the index of the variadic parameter, or -1 if the function is not variadic variadic int } diff --git a/server/functions/framework/quick_function.go b/server/functions/framework/quick_function.go new file mode 100644 index 0000000000..0db4ad99e2 --- /dev/null +++ b/server/functions/framework/quick_function.go @@ -0,0 +1,358 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "fmt" + + "github.com/dolthub/go-mysql-server/sql" + + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// QuickFunction represents an optimized function expression that has specific criteria in order to streamline +// evaluation. This will only apply to very specific functions that are generally performance-critical. +type QuickFunction interface { + Function + // CallVariadic is the variadic form of the Call function that is specific to each implementation of QuickFunction. + // The implementation will not verify that the correct number of arguments have been passed. + CallVariadic(ctx *sql.Context, args ...any) (interface{}, error) + // ResolvedTypes returns the types that were resolved with this function. + ResolvedTypes() []*pgtypes.DoltgresType + // WithResolvedTypes returns a new QuickFunction with the replaced resolved types. The implementation will not + // verify that the new types are correct in any way. This returns a QuickFunction, however it's typed as "any" due + // to potential import cycles. + WithResolvedTypes(newTypes []*pgtypes.DoltgresType) any +} + +// QuickFunction1 is an implementation of QuickFunction that handles a single parameter. +type QuickFunction1 struct { + Name string + Argument sql.Expression + IsStrict bool + callResolved [2]*pgtypes.DoltgresType + function Function1 +} + +var _ QuickFunction = (*QuickFunction1)(nil) + +// FunctionName implements the interface sql.Expression. +func (q *QuickFunction1) FunctionName() string { + return q.Name +} + +// Description implements the interface sql.Expression. +func (q *QuickFunction1) Description() string { + return fmt.Sprintf("The PostgreSQL function `%s`", q.Name) +} + +// Resolved implements the interface sql.Expression. +func (q *QuickFunction1) Resolved() bool { + return true +} + +// String implements the interface sql.Expression. +func (q *QuickFunction1) String() string { + // We'll reuse the compiled function's output so that the logic is centralized + c := CompiledFunction{ + Name: q.Name, + Arguments: []sql.Expression{q.Argument}, + } + return c.String() +} + +// Type implements the interface sql.Expression. +func (q *QuickFunction1) Type() sql.Type { + return q.callResolved[1] +} + +// IsNullable implements the interface sql.Expression. +func (q *QuickFunction1) IsNullable() bool { + return true +} + +// IsNonDeterministic implements the interface sql.NonDeterministicExpression. +func (q *QuickFunction1) IsNonDeterministic() bool { + return q.function.IsNonDeterministic +} + +// Eval implements the interface sql.Expression. +func (q *QuickFunction1) Eval(ctx *sql.Context, row sql.Row) (interface{}, error) { + arg, err := q.Argument.Eval(ctx, row) + if err != nil { + return nil, err + } + if arg == nil && q.IsStrict { + return nil, nil + } + return q.function.Callable(ctx, q.callResolved, arg) +} + +// Call directly calls the underlying function with the given arguments. This does not perform any form of NULL checking +// as it is assumed that it was done prior to this call. It also does not validate any types. This exists purely for +// performance, when we can guarantee that the input is always valid and well-formed. +func (q *QuickFunction1) Call(ctx *sql.Context, arg0 any) (interface{}, error) { + return q.function.Callable(ctx, q.callResolved, arg0) +} + +// CallVariadic implements the interface QuickFunction. +func (q *QuickFunction1) CallVariadic(ctx *sql.Context, args ...any) (interface{}, error) { + return q.function.Callable(ctx, q.callResolved, args[0]) +} + +// ResolvedTypes implements the interface QuickFunction. +func (q *QuickFunction1) ResolvedTypes() []*pgtypes.DoltgresType { + return q.callResolved[:] +} + +// WithResolvedTypes implements the interface QuickFunction. +func (q *QuickFunction1) WithResolvedTypes(newTypes []*pgtypes.DoltgresType) any { + return &QuickFunction1{ + Name: q.Name, + Argument: q.Argument, + IsStrict: q.IsStrict, + callResolved: [2]*pgtypes.DoltgresType(newTypes), + function: q.function, + } +} + +// Children implements the interface sql.Expression. +func (q *QuickFunction1) Children() []sql.Expression { + return []sql.Expression{q.Argument} +} + +// WithChildren implements the interface sql.Expression. +func (q *QuickFunction1) WithChildren(children ...sql.Expression) (sql.Expression, error) { + return nil, fmt.Errorf("cannot change the children for `%T`", q) +} + +// specificFuncImpl implements the interface sql.Expression. +func (*QuickFunction1) specificFuncImpl() {} + +// QuickFunction2 is an implementation of QuickFunction that handles two parameters. +type QuickFunction2 struct { + Name string + Arguments [2]sql.Expression + IsStrict bool + callResolved [3]*pgtypes.DoltgresType + function Function2 +} + +var _ QuickFunction = (*QuickFunction2)(nil) + +// FunctionName implements the interface sql.Expression. +func (q *QuickFunction2) FunctionName() string { + return q.Name +} + +// Description implements the interface sql.Expression. +func (q *QuickFunction2) Description() string { + return fmt.Sprintf("The PostgreSQL function `%s`", q.Name) +} + +// Resolved implements the interface sql.Expression. +func (q *QuickFunction2) Resolved() bool { + return true +} + +// String implements the interface sql.Expression. +func (q *QuickFunction2) String() string { + // We'll reuse the compiled function's output so that the logic is centralized + c := CompiledFunction{ + Name: q.Name, + Arguments: q.Arguments[:], + } + return c.String() +} + +// Type implements the interface sql.Expression. +func (q *QuickFunction2) Type() sql.Type { + return q.callResolved[2] +} + +// IsNullable implements the interface sql.Expression. +func (q *QuickFunction2) IsNullable() bool { + return true +} + +// IsNonDeterministic implements the interface sql.NonDeterministicExpression. +func (q *QuickFunction2) IsNonDeterministic() bool { + return q.function.IsNonDeterministic +} + +// Eval implements the interface sql.Expression. +func (q *QuickFunction2) Eval(ctx *sql.Context, row sql.Row) (interface{}, error) { + var args [2]any + for i := range q.Arguments { + var err error + args[i], err = q.Arguments[i].Eval(ctx, row) + if err != nil { + return nil, err + } + if args[i] == nil && q.IsStrict { + return nil, nil + } + } + return q.function.Callable(ctx, q.callResolved, args[0], args[1]) +} + +// Call directly calls the underlying function with the given arguments. This does not perform any form of NULL checking +// as it is assumed that it was done prior to this call. It also does not validate any types. This exists purely for +// performance, when we can guarantee that the input is always valid and well-formed. +func (q *QuickFunction2) Call(ctx *sql.Context, arg0 any, arg1 any) (interface{}, error) { + return q.function.Callable(ctx, q.callResolved, arg0, arg1) +} + +// CallVariadic implements the interface QuickFunction. +func (q *QuickFunction2) CallVariadic(ctx *sql.Context, args ...any) (interface{}, error) { + return q.function.Callable(ctx, q.callResolved, args[0], args[1]) +} + +// ResolvedTypes implements the interface QuickFunction. +func (q *QuickFunction2) ResolvedTypes() []*pgtypes.DoltgresType { + return q.callResolved[:] +} + +// WithResolvedTypes implements the interface QuickFunction. +func (q *QuickFunction2) WithResolvedTypes(newTypes []*pgtypes.DoltgresType) any { + return &QuickFunction2{ + Name: q.Name, + Arguments: q.Arguments, + IsStrict: q.IsStrict, + callResolved: [3]*pgtypes.DoltgresType(newTypes), + function: q.function, + } +} + +// Children implements the interface sql.Expression. +func (q *QuickFunction2) Children() []sql.Expression { + return q.Arguments[:] +} + +// WithChildren implements the interface sql.Expression. +func (q *QuickFunction2) WithChildren(children ...sql.Expression) (sql.Expression, error) { + return nil, fmt.Errorf("cannot change the children for `%T`", q) +} + +// specificFuncImpl implements the interface sql.Expression. +func (*QuickFunction2) specificFuncImpl() {} + +// QuickFunction3 is an implementation of QuickFunction that handles three parameters. +type QuickFunction3 struct { + Name string + Arguments [3]sql.Expression + IsStrict bool + callResolved [4]*pgtypes.DoltgresType + function Function3 +} + +var _ QuickFunction = (*QuickFunction3)(nil) + +// FunctionName implements the interface sql.Expression. +func (q *QuickFunction3) FunctionName() string { + return q.Name +} + +// Description implements the interface sql.Expression. +func (q *QuickFunction3) Description() string { + return fmt.Sprintf("The PostgreSQL function `%s`", q.Name) +} + +// Resolved implements the interface sql.Expression. +func (q *QuickFunction3) Resolved() bool { + return true +} + +// String implements the interface sql.Expression. +func (q *QuickFunction3) String() string { + // We'll reuse the compiled function's output so that the logic is centralized + c := CompiledFunction{ + Name: q.Name, + Arguments: q.Arguments[:], + } + return c.String() +} + +// Type implements the interface sql.Expression. +func (q *QuickFunction3) Type() sql.Type { + return q.callResolved[3] +} + +// IsNullable implements the interface sql.Expression. +func (q *QuickFunction3) IsNullable() bool { + return true +} + +// IsNonDeterministic implements the interface sql.NonDeterministicExpression. +func (q *QuickFunction3) IsNonDeterministic() bool { + return q.function.IsNonDeterministic +} + +// Eval implements the interface sql.Expression. +func (q *QuickFunction3) Eval(ctx *sql.Context, row sql.Row) (interface{}, error) { + var args [3]any + for i := range q.Arguments { + var err error + args[i], err = q.Arguments[i].Eval(ctx, row) + if err != nil { + return nil, err + } + if args[i] == nil && q.IsStrict { + return nil, nil + } + } + return q.function.Callable(ctx, q.callResolved, args[0], args[1], args[2]) +} + +// Call directly calls the underlying function with the given arguments. This does not perform any form of NULL checking +// as it is assumed that it was done prior to this call. It also does not validate any types. This exists purely for +// performance, when we can guarantee that the input is always valid and well-formed. +func (q *QuickFunction3) Call(ctx *sql.Context, arg0 any, arg1 any, arg2 any) (interface{}, error) { + return q.function.Callable(ctx, q.callResolved, arg0, arg1, arg2) +} + +// CallVariadic implements the interface QuickFunction. +func (q *QuickFunction3) CallVariadic(ctx *sql.Context, args ...any) (interface{}, error) { + return q.function.Callable(ctx, q.callResolved, args[0], args[1], args[2]) +} + +// ResolvedTypes implements the interface QuickFunction. +func (q *QuickFunction3) ResolvedTypes() []*pgtypes.DoltgresType { + return q.callResolved[:] +} + +// WithResolvedTypes implements the interface QuickFunction. +func (q *QuickFunction3) WithResolvedTypes(newTypes []*pgtypes.DoltgresType) any { + return &QuickFunction3{ + Name: q.Name, + Arguments: q.Arguments, + IsStrict: q.IsStrict, + callResolved: [4]*pgtypes.DoltgresType(newTypes), + function: q.function, + } +} + +// Children implements the interface sql.Expression. +func (q *QuickFunction3) Children() []sql.Expression { + return q.Arguments[:] +} + +// WithChildren implements the interface sql.Expression. +func (q *QuickFunction3) WithChildren(children ...sql.Expression) (sql.Expression, error) { + return nil, fmt.Errorf("cannot change the children for `%T`", q) +} + +// specificFuncImpl implements the interface sql.Expression. +func (*QuickFunction3) specificFuncImpl() {} diff --git a/server/functions/gcd.go b/server/functions/gcd.go index 2bae0966aa..697983b580 100644 --- a/server/functions/gcd.go +++ b/server/functions/gcd.go @@ -31,9 +31,9 @@ func initGcd() { var gcd_int64_int64 = framework.Function2{ Name: "gcd", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1Interface any, val2Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1Interface any, val2Interface any) (any, error) { val1 := val1Interface.(int64) val2 := val2Interface.(int64) for val2 != 0 { diff --git a/server/functions/init.go b/server/functions/init.go index 184dfc2880..0dc3d69c6c 100644 --- a/server/functions/init.go +++ b/server/functions/init.go @@ -14,8 +14,48 @@ package functions +// initTypeFunctions initializes all functions related to types in this package. +func initTypeFunctions() { + initAny() + initAnyArray() + initAnyElement() + initAnyNonArray() + initArray() + initBool() + initBpChar() + initBytea() + initChar() + initDate() + initDomain() + initFloat4() + initFloat8() + initInt2() + initInt4() + initInt8() + initInternal() + initInterval() + initJson() + initJsonB() + initName() + initNumeric() + initOid() + initRegclass() + initRegproc() + initRegtype() + initText() + initTime() + initTimestamp() + initTimestampTZ() + initTimeTZ() + initUnknown() + initUuid() + initVarChar() + initXid() +} + // Init initializes all functions in this package. func Init() { + initTypeFunctions() initAbs() initAcos() initAcosd() diff --git a/server/functions/initcap.go b/server/functions/initcap.go index 07cc5ba07a..15b82a595a 100644 --- a/server/functions/initcap.go +++ b/server/functions/initcap.go @@ -33,9 +33,9 @@ func initInitcap() { var initcap_text = framework.Function1{ Name: "initcap", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return cases.Title(language.English).String(val1.(string)), nil }, } diff --git a/server/functions/int2.go b/server/functions/int2.go new file mode 100644 index 0000000000..8a7526cbb8 --- /dev/null +++ b/server/functions/int2.go @@ -0,0 +1,152 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "encoding/binary" + "strconv" + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initInt2 registers the functions to the catalog. +func initInt2() { + framework.RegisterFunction(int2in) + framework.RegisterFunction(int2out) + framework.RegisterFunction(int2recv) + framework.RegisterFunction(int2send) + framework.RegisterFunction(btint2cmp) + framework.RegisterFunction(btint24cmp) + framework.RegisterFunction(btint28cmp) +} + +// int2in represents the PostgreSQL function of int2 type IO input. +var int2in = framework.Function1{ + Name: "int2in", + Return: pgtypes.Int16, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + iVal, err := strconv.ParseInt(strings.TrimSpace(input), 10, 16) + if err != nil { + return nil, pgtypes.ErrInvalidSyntaxForType.New("int2", input) + } + if iVal > 32767 || iVal < -32768 { + return nil, pgtypes.ErrValueIsOutOfRangeForType.New(input, "int2") + } + return int16(iVal), nil + }, +} + +// int2out represents the PostgreSQL function of int2 type IO output. +var int2out = framework.Function1{ + Name: "int2out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int16}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return strconv.FormatInt(int64(val.(int16)), 10), nil + }, +} + +// int2recv represents the PostgreSQL function of int2 type IO receive. +var int2recv = framework.Function1{ + Name: "int2recv", + Return: pgtypes.Int16, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return int16(binary.BigEndian.Uint16(data) - (1 << 15)), nil + }, +} + +// int2send represents the PostgreSQL function of int2 type IO send. +var int2send = framework.Function1{ + Name: "int2send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int16}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + retVal := make([]byte, 2) + binary.BigEndian.PutUint16(retVal, uint16(val.(int16))+(1<<15)) + return retVal, nil + }, +} + +// btint2cmp represents the PostgreSQL function of int2 type compare. +var btint2cmp = framework.Function2{ + Name: "btint2cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(int16) + bb := val2.(int16) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// btint24cmp represents the PostgreSQL function of int2 type compare with int4. +var btint24cmp = framework.Function2{ + Name: "btint24cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := int32(val1.(int16)) + bb := val2.(int32) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// btint28cmp represents the PostgreSQL function of int2 type compare with int8. +var btint28cmp = framework.Function2{ + Name: "btint28cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int64}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := int64(val1.(int16)) + bb := val2.(int64) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} diff --git a/server/functions/int4.go b/server/functions/int4.go new file mode 100644 index 0000000000..3a9878e37d --- /dev/null +++ b/server/functions/int4.go @@ -0,0 +1,152 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "encoding/binary" + "strconv" + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initInt4 registers the functions to the catalog. +func initInt4() { + framework.RegisterFunction(int4in) + framework.RegisterFunction(int4out) + framework.RegisterFunction(int4recv) + framework.RegisterFunction(int4send) + framework.RegisterFunction(btint4cmp) + framework.RegisterFunction(btint42cmp) + framework.RegisterFunction(btint48cmp) +} + +// int4in represents the PostgreSQL function of int4 type IO input. +var int4in = framework.Function1{ + Name: "int4in", + Return: pgtypes.Int32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + iVal, err := strconv.ParseInt(strings.TrimSpace(input), 10, 32) + if err != nil { + return nil, pgtypes.ErrInvalidSyntaxForType.New("int4", input) + } + if iVal > 2147483647 || iVal < -2147483648 { + return nil, pgtypes.ErrValueIsOutOfRangeForType.New(input, "int4") + } + return int32(iVal), nil + }, +} + +// int4out represents the PostgreSQL function of int4 type IO output. +var int4out = framework.Function1{ + Name: "int4out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return strconv.FormatInt(int64(val.(int32)), 10), nil + }, +} + +// int4recv represents the PostgreSQL function of int4 type IO receive. +var int4recv = framework.Function1{ + Name: "int4recv", + Return: pgtypes.Int32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return int32(binary.BigEndian.Uint32(data) - (1 << 31)), nil + }, +} + +// int4send represents the PostgreSQL function of int4 type IO send. +var int4send = framework.Function1{ + Name: "int4send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + retVal := make([]byte, 4) + binary.BigEndian.PutUint32(retVal, uint32(val.(int32))+(1<<31)) + return retVal, nil + }, +} + +// btint4cmp represents the PostgreSQL function of int4 type compare. +var btint4cmp = framework.Function2{ + Name: "btint4cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(int32) + bb := val2.(int32) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// btint42cmp represents the PostgreSQL function of int4 type compare with int2. +var btint42cmp = framework.Function2{ + Name: "btint42cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int16}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(int32) + bb := int32(val2.(int16)) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// btint48cmp represents the PostgreSQL function of int4 type compare with int8. +var btint48cmp = framework.Function2{ + Name: "btint48cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int64}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := int64(val1.(int32)) + bb := val2.(int64) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} diff --git a/server/functions/int8.go b/server/functions/int8.go new file mode 100644 index 0000000000..15d401597f --- /dev/null +++ b/server/functions/int8.go @@ -0,0 +1,152 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "encoding/binary" + "strconv" + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initInt8 registers the functions to the catalog. +func initInt8() { + framework.RegisterFunction(int8in) + framework.RegisterFunction(int8out) + framework.RegisterFunction(int8recv) + framework.RegisterFunction(int8send) + framework.RegisterFunction(btint8cmp) + framework.RegisterFunction(btint82cmp) + framework.RegisterFunction(btint84cmp) +} + +// int8in represents the PostgreSQL function of int8 type IO input. +var int8in = framework.Function1{ + Name: "int8in", + Return: pgtypes.Int64, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + iVal, err := strconv.ParseInt(strings.TrimSpace(input), 10, 64) + if err != nil { + return nil, pgtypes.ErrInvalidSyntaxForType.New("int8", input) + } + return iVal, nil + }, +} + +// int8out represents the PostgreSQL function of int8 type IO output. +var int8out = framework.Function1{ + Name: "int8out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int64}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + if val == nil { + return nil, nil + } + return strconv.FormatInt(val.(int64), 10), nil + }, +} + +// int8recv represents the PostgreSQL function of int8 type IO receive. +var int8recv = framework.Function1{ + Name: "int8recv", + Return: pgtypes.Int64, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return int64(binary.BigEndian.Uint64(data) - (1 << 63)), nil + }, +} + +// int8send represents the PostgreSQL function of int8 type IO send. +var int8send = framework.Function1{ + Name: "int8send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int64}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + retVal := make([]byte, 8) + binary.BigEndian.PutUint64(retVal, uint64(val.(int64))+(1<<63)) + return retVal, nil + }, +} + +// btint8cmp represents the PostgreSQL function of int8 type compare. +var btint8cmp = framework.Function2{ + Name: "btint8cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(int64) + bb := val2.(int64) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// btint82cmp represents the PostgreSQL function of int8 type compare with int2. +var btint82cmp = framework.Function2{ + Name: "btint82cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int16}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(int64) + bb := int64(val2.(int16)) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// btint84cmp represents the PostgreSQL function of int8 type compare with int4. +var btint84cmp = framework.Function2{ + Name: "btint84cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(int64) + bb := int64(val2.(int32)) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} diff --git a/server/functions/internal.go b/server/functions/internal.go new file mode 100644 index 0000000000..5ff4006cb2 --- /dev/null +++ b/server/functions/internal.go @@ -0,0 +1,51 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initInternal registers the functions to the catalog. +func initInternal() { + framework.RegisterFunction(internal_in) + framework.RegisterFunction(internal_out) +} + +// internal_in represents the PostgreSQL function of internal type IO input. +var internal_in = framework.Function1{ + Name: "internal_in", + Return: pgtypes.Internal, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return []byte(val.(string)), nil + }, +} + +// internal_out represents the PostgreSQL function of internal type IO output. +var internal_out = framework.Function1{ + Name: "internal_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + return string(val.([]byte)), nil + }, +} diff --git a/server/functions/interval.go b/server/functions/interval.go new file mode 100644 index 0000000000..35267ab705 --- /dev/null +++ b/server/functions/interval.go @@ -0,0 +1,143 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/utils" + + "github.com/dolthub/doltgresql/postgres/parser/duration" + "github.com/dolthub/doltgresql/postgres/parser/sem/tree" + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initInterval registers the functions to the catalog. +func initInterval() { + framework.RegisterFunction(interval_in) + framework.RegisterFunction(interval_out) + framework.RegisterFunction(interval_recv) + framework.RegisterFunction(interval_send) + framework.RegisterFunction(intervaltypmodin) + framework.RegisterFunction(intervaltypmodout) + framework.RegisterFunction(interval_cmp) +} + +// interval_in represents the PostgreSQL function of interval type IO input. +var interval_in = framework.Function3{ + Name: "interval_in", + Return: pgtypes.Interval, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Cstring, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + input := val1.(string) + //oid := val2.(uint32) + //typmod := val3.(int32) + dInterval, err := tree.ParseDInterval(input) + if err != nil { + return nil, err + } + return dInterval.Duration, nil + }, +} + +// interval_out represents the PostgreSQL function of interval type IO output. +var interval_out = framework.Function1{ + Name: "interval_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Interval}, + Strict: true, + Callable: func(ctx *sql.Context, t [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(duration.Duration).String(), nil + }, +} + +// interval_recv represents the PostgreSQL function of interval type IO receive. +var interval_recv = framework.Function3{ + Name: "interval_recv", + Return: pgtypes.Interval, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Internal, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + data := val1.([]byte) + //oid := val2.(uint32) + //typmod := val3.(int32) // precision + if len(data) == 0 { + return nil, nil + } + reader := utils.NewReader(data) + sortNanos := reader.Int64() + months := reader.Int32() + days := reader.Int32() + return duration.Decode(sortNanos, int64(months), int64(days)) + }, +} + +// interval_send represents the PostgreSQL function of interval type IO send. +var interval_send = framework.Function1{ + Name: "interval_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Interval}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + sortNanos, months, days, err := val.(duration.Duration).Encode() + if err != nil { + return nil, err + } + writer := utils.NewWriter(0) + writer.Int64(sortNanos) + writer.Int32(int32(months)) + writer.Int32(int32(days)) + return writer.Data(), nil + }, +} + +// intervaltypmodin represents the PostgreSQL function of interval type IO typmod input. +var intervaltypmodin = framework.Function1{ + Name: "intervaltypmodin", + Return: pgtypes.Int32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.CstringArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO: implement interval fields and precision + return int32(0), nil + }, +} + +// intervaltypmodout represents the PostgreSQL function of interval type IO typmod output. +var intervaltypmodout = framework.Function1{ + Name: "intervaltypmodout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO: implement interval fields and precision + return "", nil + }, +} + +// interval_cmp represents the PostgreSQL function of interval type compare. +var interval_cmp = framework.Function2{ + Name: "interval_cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Interval}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(duration.Duration) + bb := val2.(duration.Duration) + return int32(ab.Compare(bb)), nil + }, +} diff --git a/server/functions/json.go b/server/functions/json.go new file mode 100644 index 0000000000..cd81f53e8c --- /dev/null +++ b/server/functions/json.go @@ -0,0 +1,85 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "unsafe" + + "github.com/dolthub/go-mysql-server/sql" + "github.com/goccy/go-json" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initJson registers the functions to the catalog. +func initJson() { + framework.RegisterFunction(json_in) + framework.RegisterFunction(json_out) + framework.RegisterFunction(json_recv) + framework.RegisterFunction(json_send) +} + +// json_in represents the PostgreSQL function of json type IO input. +var json_in = framework.Function1{ + Name: "json_in", + Return: pgtypes.Json, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + if json.Valid(unsafe.Slice(unsafe.StringData(input), len(input))) { + return input, nil + } + return nil, pgtypes.ErrInvalidSyntaxForType.New("json", input[:10]+"...") + }, +} + +// json_out represents the PostgreSQL function of json type IO output. +var json_out = framework.Function1{ + Name: "json_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Json}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(string), nil + }, +} + +// json_recv represents the PostgreSQL function of json type IO receive. +var json_recv = framework.Function1{ + Name: "json_recv", + Return: pgtypes.Json, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return string(data), nil + }, +} + +// json_send represents the PostgreSQL function of json type IO send. +var json_send = framework.Function1{ + Name: "json_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Json}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return []byte(val.(string)), nil + }, +} diff --git a/server/functions/jsonb.go b/server/functions/jsonb.go new file mode 100644 index 0000000000..302c4cecb5 --- /dev/null +++ b/server/functions/jsonb.go @@ -0,0 +1,110 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "strings" + "unsafe" + + "github.com/dolthub/go-mysql-server/sql" + "github.com/goccy/go-json" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" + "github.com/dolthub/doltgresql/utils" +) + +// initJsonB registers the functions to the catalog. +func initJsonB() { + framework.RegisterFunction(jsonb_in) + framework.RegisterFunction(jsonb_out) + framework.RegisterFunction(jsonb_recv) + framework.RegisterFunction(jsonb_send) + framework.RegisterFunction(jsonb_cmp) +} + +// jsonb_in represents the PostgreSQL function of jsonb type IO input. +var jsonb_in = framework.Function1{ + Name: "jsonb_in", + Return: pgtypes.JsonB, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + inputBytes := unsafe.Slice(unsafe.StringData(input), len(input)) + if json.Valid(inputBytes) { + doc, err := pgtypes.UnmarshalToJsonDocument(inputBytes) + return doc, err + } + return nil, pgtypes.ErrInvalidSyntaxForType.New("jsonb", input[:10]+"...") + }, +} + +// jsonb_out represents the PostgreSQL function of jsonb type IO output. +var jsonb_out = framework.Function1{ + Name: "jsonb_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.JsonB}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + sb := strings.Builder{} + sb.Grow(256) + pgtypes.JsonValueFormatter(&sb, val.(pgtypes.JsonDocument).Value) + return sb.String(), nil + }, +} + +// jsonb_recv represents the PostgreSQL function of jsonb type IO receive. +var jsonb_recv = framework.Function1{ + Name: "jsonb_recv", + Return: pgtypes.JsonB, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + reader := utils.NewReader(data) + jsonValue, err := pgtypes.JsonValueDeserialize(reader) + return pgtypes.JsonDocument{Value: jsonValue}, err + }, +} + +// jsonb_send represents the PostgreSQL function of jsonb type IO send. +var jsonb_send = framework.Function1{ + Name: "jsonb_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.JsonB}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + writer := utils.NewWriter(256) + pgtypes.JsonValueSerialize(writer, val.(pgtypes.JsonDocument).Value) + return writer.Data(), nil + }, +} + +// jsonb_cmp represents the PostgreSQL function of jsonb type compare. +var jsonb_cmp = framework.Function2{ + Name: "jsonb_cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.JsonB, pgtypes.JsonB}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(pgtypes.JsonDocument) + bb := val2.(pgtypes.JsonDocument) + return int32(pgtypes.JsonValueCompare(ab.Value, bb.Value)), nil + }, +} diff --git a/server/functions/lcm.go b/server/functions/lcm.go index 8107833f21..9307e48854 100644 --- a/server/functions/lcm.go +++ b/server/functions/lcm.go @@ -33,9 +33,9 @@ func initLcm() { var lcm_int64_int64 = framework.Function2{ Name: "lcm", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, dt [3]pgtypes.DoltgresType, val1Int any, val2Int any) (any, error) { + Callable: func(ctx *sql.Context, dt [3]*pgtypes.DoltgresType, val1Int any, val2Int any) (any, error) { val1 := val1Int.(int64) val2 := val2Int.(int64) if val1 == val2 { diff --git a/server/functions/left.go b/server/functions/left.go index 4e9e0cda1b..ecfd9e5fc6 100644 --- a/server/functions/left.go +++ b/server/functions/left.go @@ -30,9 +30,9 @@ func initLeft() { var left_text_int32 = framework.Function2{ Name: "left", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, strInt any, nInt any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, strInt any, nInt any) (any, error) { str := strInt.(string) n := nInt.(int32) if n >= 0 { diff --git a/server/functions/length.go b/server/functions/length.go index 77f2ede4a7..bd47ae0186 100644 --- a/server/functions/length.go +++ b/server/functions/length.go @@ -30,9 +30,9 @@ func initLength() { var length_text = framework.Function1{ Name: "length", Return: pgtypes.Int32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return int32(len([]rune(val1.(string)))), nil }, } diff --git a/server/functions/ln.go b/server/functions/ln.go index f83ff024ad..bd314d92a1 100644 --- a/server/functions/ln.go +++ b/server/functions/ln.go @@ -35,9 +35,9 @@ func initLn() { var ln_float64 = framework.Function1{ Name: "ln", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1.(float64) == 0 { return nil, fmt.Errorf("cannot take logarithm of zero") } else if val1.(float64) < 0 { @@ -51,9 +51,9 @@ var ln_float64 = framework.Function1{ var ln_numeric = framework.Function1{ Name: "ln", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1 == nil { return nil, nil } diff --git a/server/functions/log.go b/server/functions/log.go index 22e75cbd5b..79b33c9ffa 100644 --- a/server/functions/log.go +++ b/server/functions/log.go @@ -36,9 +36,9 @@ func initLog() { var log_float64 = framework.Function1{ Name: "log", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1Interface any) (any, error) { val1 := val1Interface.(float64) if val1 == 0 { return nil, fmt.Errorf("cannot take logarithm of zero") @@ -53,9 +53,9 @@ var log_float64 = framework.Function1{ var log_numeric = framework.Function1{ Name: "log", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1Interface any) (any, error) { if val1Interface == nil { return nil, nil } @@ -75,9 +75,9 @@ var log_numeric = framework.Function1{ var log_numeric_numeric = framework.Function2{ Name: "log", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1Interface any, val2Interface any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1Interface any, val2Interface any) (any, error) { if val1Interface == nil || val2Interface == nil { return nil, nil } diff --git a/server/functions/lower.go b/server/functions/lower.go index e03d1edc11..9d63496072 100644 --- a/server/functions/lower.go +++ b/server/functions/lower.go @@ -32,9 +32,9 @@ func initLower() { var lower_text = framework.Function1{ Name: "lower", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { //TODO: this doesn't respect collations return strings.ToLower(val1.(string)), nil }, diff --git a/server/functions/lpad.go b/server/functions/lpad.go index c568174441..d4438ba8fe 100644 --- a/server/functions/lpad.go +++ b/server/functions/lpad.go @@ -31,10 +31,10 @@ func initLpad() { var lpad_text_int32 = framework.Function2{ Name: "lpad", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { - var unusedTypes [4]pgtypes.DoltgresType + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + var unusedTypes [4]*pgtypes.DoltgresType return lpad_text_int32_text.Callable(ctx, unusedTypes, val1, val2, " ") }, } @@ -43,9 +43,9 @@ var lpad_text_int32 = framework.Function2{ var lpad_text_int32_text = framework.Function3{ Name: "lpad", Return: pgtypes.Text, - Parameters: [3]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32, pgtypes.Text}, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [4]pgtypes.DoltgresType, str any, length any, fill any) (any, error) { + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, str any, length any, fill any) (any, error) { if length.(int32) <= 0 { return "", nil } diff --git a/server/functions/ltrim.go b/server/functions/ltrim.go index e87b1f7fff..bd9b51b360 100644 --- a/server/functions/ltrim.go +++ b/server/functions/ltrim.go @@ -31,10 +31,10 @@ func initLtrim() { var ltrim_text = framework.Function1{ Name: "ltrim", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { - var unusedTypes [3]pgtypes.DoltgresType + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { + var unusedTypes [3]*pgtypes.DoltgresType return ltrim_text_text.Callable(ctx, unusedTypes, val1, " ") }, } @@ -43,9 +43,9 @@ var ltrim_text = framework.Function1{ var ltrim_text_text = framework.Function2{ Name: "ltrim", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, str any, characters any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, str any, characters any) (any, error) { runes := []rune(str.(string)) trimChars := make(map[rune]struct{}) for _, c := range characters.(string) { diff --git a/server/functions/md5.go b/server/functions/md5.go index 2b40c6ea0c..8adf71e21a 100644 --- a/server/functions/md5.go +++ b/server/functions/md5.go @@ -33,9 +33,9 @@ func initMd5() { var md5_text = framework.Function1{ Name: "md5", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return fmt.Sprintf("%x", md5_package.Sum([]byte(val1.(string)))), nil }, } diff --git a/server/functions/min_scale.go b/server/functions/min_scale.go index 75f2c0f37f..c0cd908bbb 100644 --- a/server/functions/min_scale.go +++ b/server/functions/min_scale.go @@ -33,9 +33,9 @@ func initMinScale() { var min_scale_numeric = framework.Function1{ Name: "min_scale", Return: pgtypes.Int32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { str := val1.(decimal.Decimal).String() if idx := strings.Index(str, "."); idx != -1 { str = str[idx+1:] diff --git a/server/functions/mod.go b/server/functions/mod.go index 04ab804153..e95c20cc14 100644 --- a/server/functions/mod.go +++ b/server/functions/mod.go @@ -36,9 +36,9 @@ func initMod() { var mod_int16_int16 = framework.Function2{ Name: "mod", Return: pgtypes.Int16, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int16, pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int16) == 0 { return nil, fmt.Errorf("division by zero") } @@ -50,9 +50,9 @@ var mod_int16_int16 = framework.Function2{ var mod_int32_int32 = framework.Function2{ Name: "mod", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int32) == 0 { return nil, fmt.Errorf("division by zero") } @@ -64,9 +64,9 @@ var mod_int32_int32 = framework.Function2{ var mod_int64_int64 = framework.Function2{ Name: "mod", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Int64, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(int64) == 0 { return nil, fmt.Errorf("division by zero") } @@ -78,9 +78,9 @@ var mod_int64_int64 = framework.Function2{ var mod_numeric_numeric = framework.Function2{ Name: "mod", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val2.(decimal.Decimal).Cmp(decimal.Zero) == 0 { return nil, fmt.Errorf("division by zero") } diff --git a/server/functions/name.go b/server/functions/name.go new file mode 100644 index 0000000000..4cb9ca53a6 --- /dev/null +++ b/server/functions/name.go @@ -0,0 +1,127 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/utils" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initName registers the functions to the catalog. +func initName() { + framework.RegisterFunction(namein) + framework.RegisterFunction(nameout) + framework.RegisterFunction(namerecv) + framework.RegisterFunction(namesend) + framework.RegisterFunction(btnamecmp) + framework.RegisterFunction(btnametextcmp) +} + +// namein represents the PostgreSQL function of name type IO input. +var namein = framework.Function1{ + Name: "namein", + Return: pgtypes.Name, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + input, _ = truncateString(input, pgtypes.NameLength) + return input, nil + }, +} + +// nameout represents the PostgreSQL function of name type IO output. +var nameout = framework.Function1{ + Name: "nameout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Name}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + str, _ := truncateString(val.(string), pgtypes.NameLength) + return str, nil + }, +} + +// namerecv represents the PostgreSQL function of name type IO receive. +var namerecv = framework.Function1{ + Name: "namerecv", + Return: pgtypes.Name, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + reader := utils.NewReader(data) + return reader.String(), nil + }, +} + +// namesend represents the PostgreSQL function of name type IO send. +var namesend = framework.Function1{ + Name: "namesend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Name}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + str := val.(string) + writer := utils.NewWriter(uint64(len(str) + 1)) + writer.String(str) + return writer.Data(), nil + }, +} + +// btnamecmp represents the PostgreSQL function of name type compare. +var btnamecmp = framework.Function2{ + Name: "btnamecmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Name}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(string) + bb := val2.(string) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// btnametextcmp represents the PostgreSQL function of name type compare with text. +var btnametextcmp = framework.Function2{ + Name: "btnametextcmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Name, pgtypes.Text}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(string) + bb := val2.(string) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} diff --git a/server/functions/nextval.go b/server/functions/nextval.go index a318a93a17..a9b43d214d 100644 --- a/server/functions/nextval.go +++ b/server/functions/nextval.go @@ -37,10 +37,10 @@ func initNextVal() { var nextval_text = framework.Function1{ Name: "nextval", Return: pgtypes.Int64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { schema, sequence, err := parseRelationName(ctx, val.(string)) if err != nil { return nil, err @@ -58,10 +58,10 @@ var nextval_text = framework.Function1{ var nextval_regclass = framework.Function1{ Name: "nextval", Return: pgtypes.Int64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Regclass}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regclass}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { relationName, err := pgtypes.Regclass.IoOutput(ctx, val) if err != nil { return nil, err diff --git a/server/functions/numeric.go b/server/functions/numeric.go new file mode 100644 index 0000000000..50e7fb9232 --- /dev/null +++ b/server/functions/numeric.go @@ -0,0 +1,161 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "fmt" + "strconv" + "strings" + + "github.com/dolthub/go-mysql-server/sql" + "github.com/shopspring/decimal" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initNumeric registers the functions to the catalog. +func initNumeric() { + framework.RegisterFunction(numeric_in) + framework.RegisterFunction(numeric_out) + framework.RegisterFunction(numeric_recv) + framework.RegisterFunction(numeric_send) + framework.RegisterFunction(numerictypmodin) + framework.RegisterFunction(numerictypmodout) + framework.RegisterFunction(numeric_cmp) +} + +// numeric_in represents the PostgreSQL function of numeric type IO input. +var numeric_in = framework.Function3{ + Name: "numeric_in", + Return: pgtypes.Numeric, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Cstring, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + input := val1.(string) + val, err := decimal.NewFromString(strings.TrimSpace(input)) + if err != nil { + return nil, pgtypes.ErrInvalidSyntaxForType.New("numeric", input) + } + typmod := val3.(int32) + return pgtypes.GetNumericValueWithTypmod(val, typmod) + }, +} + +// numeric_out represents the PostgreSQL function of numeric type IO output. +var numeric_out = framework.Function1{ + Name: "numeric_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, + Strict: true, + Callable: func(ctx *sql.Context, t [2]*pgtypes.DoltgresType, val any) (any, error) { + typ := t[0] + dec := val.(decimal.Decimal) + tm := typ.GetAttTypMod() + if tm == -1 { + return dec.StringFixed(dec.Exponent() * -1), nil + } else { + _, s := pgtypes.GetPrecisionAndScaleFromTypmod(tm) + return dec.StringFixed(s), nil + } + }, +} + +// numeric_recv represents the PostgreSQL function of numeric type IO receive. +var numeric_recv = framework.Function3{ + Name: "numeric_recv", + Return: pgtypes.Numeric, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Internal, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + data := val1.([]byte) + //typmod := val3.(int32) + //precision, scale := getPrecisionAndScaleFromTypmod(typmod) + if len(data) == 0 { + return nil, nil + } + retVal := decimal.NewFromInt(0) + err := retVal.UnmarshalBinary(data) + return retVal, err + }, +} + +// numeric_send represents the PostgreSQL function of numeric type IO send. +var numeric_send = framework.Function1{ + Name: "numeric_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(decimal.Decimal).MarshalBinary() + }, +} + +// numerictypmodin represents the PostgreSQL function of numeric type IO typmod input. +var numerictypmodin = framework.Function1{ + Name: "numerictypmodin", + Return: pgtypes.Int32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.CstringArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + arr := val.([]any) + if len(arr) == 0 { + return nil, pgtypes.ErrTypmodArrayMustBe1D.New() + } else if len(arr) > 2 { + return nil, pgtypes.ErrInvalidTypMod.New("NUMERIC") + } + + p, err := strconv.ParseInt(arr[0].(string), 10, 32) + if err != nil { + return nil, err + } + precision := int32(p) + scale := int32(0) + if len(arr) == 2 { + s, err := strconv.ParseInt(arr[1].(string), 10, 32) + if err != nil { + return nil, err + } + scale = int32(s) + } + return pgtypes.GetTypmodFromNumericPrecisionAndScale(precision, scale) + }, +} + +// numerictypmodout represents the PostgreSQL function of numeric type IO typmod output. +var numerictypmodout = framework.Function1{ + Name: "numerictypmodout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + typmod := val.(int32) + precision, scale := pgtypes.GetPrecisionAndScaleFromTypmod(typmod) + return fmt.Sprintf("(%v,%v)", precision, scale), nil + }, +} + +// numeric_cmp represents the PostgreSQL function of numeric type compare. +var numeric_cmp = framework.Function2{ + Name: "numeric_cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(decimal.Decimal) + bb := val2.(decimal.Decimal) + return int32(ab.Cmp(bb)), nil + }, +} diff --git a/server/functions/obj_description.go b/server/functions/obj_description.go index 79aba73896..96aa603bec 100644 --- a/server/functions/obj_description.go +++ b/server/functions/obj_description.go @@ -30,10 +30,10 @@ func initObjDescription() { var obj_description = framework.Function2{ Name: "obj_description", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Name}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: When we support comments this should return the comment for a // database object specified by its OID and the name of the containing // system catalog. diff --git a/server/functions/octet_length.go b/server/functions/octet_length.go index 0eee3a67e2..9714778a6d 100644 --- a/server/functions/octet_length.go +++ b/server/functions/octet_length.go @@ -30,9 +30,9 @@ func initOctetLength() { var octet_length_text = framework.Function1{ Name: "octet_length", Return: pgtypes.Int32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return int32(len(val1.(string))), nil }, } diff --git a/server/functions/oid.go b/server/functions/oid.go new file mode 100644 index 0000000000..aa404ec00c --- /dev/null +++ b/server/functions/oid.go @@ -0,0 +1,113 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "encoding/binary" + "strconv" + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initOid registers the functions to the catalog. +func initOid() { + framework.RegisterFunction(oidin) + framework.RegisterFunction(oidout) + framework.RegisterFunction(oidrecv) + framework.RegisterFunction(oidsend) + framework.RegisterFunction(btoidcmp) +} + +// oidin represents the PostgreSQL function of oid type IO input. +var oidin = framework.Function1{ + Name: "oidin", + Return: pgtypes.Oid, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + uVal, err := strconv.ParseInt(strings.TrimSpace(input), 10, 64) + if err != nil { + return nil, pgtypes.ErrInvalidSyntaxForType.New("oid", input) + } + // Note: This minimum is different (-4294967295) for Postgres 15.4 compiled by Visual C++ + if uVal > pgtypes.MaxUint32 || uVal < pgtypes.MinInt32 { + return nil, pgtypes.ErrValueIsOutOfRangeForType.New(input, "oid") + } + return uint32(uVal), nil + }, +} + +// oidout represents the PostgreSQL function of oid type IO output. +var oidout = framework.Function1{ + Name: "oidout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return strconv.FormatUint(uint64(val.(uint32)), 10), nil + }, +} + +// oidrecv represents the PostgreSQL function of oid type IO receive. +var oidrecv = framework.Function1{ + Name: "oidrecv", + Return: pgtypes.Oid, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return binary.BigEndian.Uint32(data), nil + }, +} + +// oidsend represents the PostgreSQL function of oid type IO send. +var oidsend = framework.Function1{ + Name: "oidsend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + retVal := make([]byte, 4) + binary.BigEndian.PutUint32(retVal, val.(uint32)) + return retVal, nil + }, +} + +// btoidcmp represents the PostgreSQL function of oid type compare. +var btoidcmp = framework.Function2{ + Name: "btoidcmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Oid}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(uint32) + bb := val2.(uint32) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} diff --git a/server/functions/pg_encoding_to_char.go b/server/functions/pg_encoding_to_char.go index 1efdf310ee..577c2fcac8 100644 --- a/server/functions/pg_encoding_to_char.go +++ b/server/functions/pg_encoding_to_char.go @@ -30,10 +30,10 @@ func initPgEncodingToChar() { var pg_encoding_to_char_int = framework.Function1{ Name: "pg_encoding_to_char", Return: pgtypes.Name, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int32}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { encoding := val.(int32) if encoding == int32(6) { return "UTF8", nil diff --git a/server/functions/pg_function_is_visible.go b/server/functions/pg_function_is_visible.go index 733490962d..98b878ab11 100644 --- a/server/functions/pg_function_is_visible.go +++ b/server/functions/pg_function_is_visible.go @@ -32,10 +32,10 @@ func initPgFunctionIsVisible() { var pg_function_is_visible_oid = framework.Function1{ Name: "pg_function_is_visible", Return: pgtypes.Bool, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { var found bool err := oid.RunCallback(ctx, val.(uint32), oid.Callbacks{ Function: func(ctx *sql.Context, function oid.ItemFunction) (cont bool, err error) { diff --git a/server/functions/pg_get_constraintdef.go b/server/functions/pg_get_constraintdef.go index 44986827e7..3cf143caf8 100644 --- a/server/functions/pg_get_constraintdef.go +++ b/server/functions/pg_get_constraintdef.go @@ -35,9 +35,9 @@ func initPgGetConstraintdef() { var pg_get_constraintdef_oid = framework.Function1{ Name: "pg_get_constraintdef", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { oidVal := val1.(uint32) def, err := getConstraintDef(ctx, oidVal) return def, err @@ -48,9 +48,9 @@ var pg_get_constraintdef_oid = framework.Function1{ var pg_get_constraintdef_oid_bool = framework.Function2{ Name: "pg_get_constraintdef", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Bool}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Bool}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { oidVal := val1.(uint32) pretty := val2.(bool) if pretty { diff --git a/server/functions/pg_get_expr.go b/server/functions/pg_get_expr.go index 2301edb9ca..a451239eca 100644 --- a/server/functions/pg_get_expr.go +++ b/server/functions/pg_get_expr.go @@ -33,9 +33,9 @@ func initPgGetExpr() { var pg_get_expr_pgnodetree_oid = framework.Function2{ Name: "pg_get_expr", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Oid}, // TODO: First parameter should be pg_node_tree + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Oid}, // TODO: First parameter should be pg_node_tree Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { // TODO: Implement this when the pg_node_tree type exists return nil, fmt.Errorf("pg_get_expr is not yet supported") }, @@ -45,9 +45,9 @@ var pg_get_expr_pgnodetree_oid = framework.Function2{ var pg_get_expr_pgnodetree_oid_bool = framework.Function3{ Name: "pg_get_expr", Return: pgtypes.Text, - Parameters: [3]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Oid, pgtypes.Bool}, // TODO: First parameter should be pg_node_tree + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Oid, pgtypes.Bool}, // TODO: First parameter should be pg_node_tree Strict: true, - Callable: func(ctx *sql.Context, _ [4]pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { // TODO: Implement this when the pg_node_tree type exists return nil, fmt.Errorf("pg_get_expr is not yet supported") }, diff --git a/server/functions/pg_get_function_identity_arguments.go b/server/functions/pg_get_function_identity_arguments.go index c6d4bb52fc..7c5fc030b8 100644 --- a/server/functions/pg_get_function_identity_arguments.go +++ b/server/functions/pg_get_function_identity_arguments.go @@ -31,10 +31,10 @@ func initPgGetFunctionIdentityArguments() { var pg_get_function_identity_arguments_oid = framework.Function1{ Name: "pg_get_function_identity_arguments", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { oidVal := val.(uint32) err := oid.RunCallback(ctx, oidVal, oid.Callbacks{ Function: func(ctx *sql.Context, function oid.ItemFunction) (cont bool, err error) { diff --git a/server/functions/pg_get_functiondef.go b/server/functions/pg_get_functiondef.go index 8c7a6c4437..e1c9201f91 100644 --- a/server/functions/pg_get_functiondef.go +++ b/server/functions/pg_get_functiondef.go @@ -31,10 +31,10 @@ func initPgGetFunctionDef() { var pg_get_functiondef_oid = framework.Function1{ Name: "pg_get_functiondef", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { err := oid.RunCallback(ctx, val.(uint32), oid.Callbacks{ Function: func(ctx *sql.Context, function oid.ItemFunction) (cont bool, err error) { // TODO: sql.Function does not have sufficient information to build CREATE FUNCTION statement diff --git a/server/functions/pg_get_indexdef.go b/server/functions/pg_get_indexdef.go index e6fbbbe72d..8b30672204 100644 --- a/server/functions/pg_get_indexdef.go +++ b/server/functions/pg_get_indexdef.go @@ -34,10 +34,10 @@ func initPgGetIndexDef() { var pg_get_indexdef_oid = framework.Function1{ Name: "pg_get_indexdef", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { oidVal := val.(uint32) err := oid.RunCallback(ctx, oidVal, oid.Callbacks{ Index: func(ctx *sql.Context, schema oid.ItemSchema, table oid.ItemTable, index oid.ItemIndex) (cont bool, err error) { @@ -56,10 +56,10 @@ var pg_get_indexdef_oid = framework.Function1{ var pg_get_indexdef_oid_integer_bool = framework.Function3{ Name: "pg_get_indexdef", Return: pgtypes.Text, - Parameters: [3]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Int32, pgtypes.Bool}, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Int32, pgtypes.Bool}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [4]pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { oidVal := val1.(uint32) colNo := val2.(int32) pretty := val3.(bool) diff --git a/server/functions/pg_get_partkeydef.go b/server/functions/pg_get_partkeydef.go index f6f66141ff..de34b57ef0 100644 --- a/server/functions/pg_get_partkeydef.go +++ b/server/functions/pg_get_partkeydef.go @@ -31,10 +31,10 @@ func initPgGetPartKeyDef() { var pg_get_partkeydef_oid = framework.Function1{ Name: "pg_get_partkeydef", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { err := oid.RunCallback(ctx, val.(uint32), oid.Callbacks{ Table: func(ctx *sql.Context, schema oid.ItemSchema, table oid.ItemTable) (cont bool, err error) { // TODO: sql.Table does not have sufficient information about partition diff --git a/server/functions/pg_get_triggerdef.go b/server/functions/pg_get_triggerdef.go index 3f08ca3560..26d8d96c8d 100644 --- a/server/functions/pg_get_triggerdef.go +++ b/server/functions/pg_get_triggerdef.go @@ -33,10 +33,10 @@ func initPgGetTriggerDef() { var pg_get_triggerdef_oid = framework.Function1{ Name: "pg_get_triggerdef", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { // TODO: triggers are not supported yet return "", nil }, @@ -46,10 +46,10 @@ var pg_get_triggerdef_oid = framework.Function1{ var pg_get_triggerdef_oid_bool = framework.Function2{ Name: "pg_get_triggerdef", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Bool}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Bool}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { pretty := val2.(bool) if pretty { return "", fmt.Errorf("pretty printing is not yet supported") diff --git a/server/functions/pg_get_userbyid.go b/server/functions/pg_get_userbyid.go index d81a86ae7c..13c06f1152 100644 --- a/server/functions/pg_get_userbyid.go +++ b/server/functions/pg_get_userbyid.go @@ -30,10 +30,10 @@ func initPgGetUserbyid() { var pg_get_userbyid_oid = framework.Function1{ Name: "pg_get_userbyid", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { // TODO: roles are not supported yet return "unknown OID()", nil }, diff --git a/server/functions/pg_get_viewdef.go b/server/functions/pg_get_viewdef.go index 35acdea276..dfbef74c87 100644 --- a/server/functions/pg_get_viewdef.go +++ b/server/functions/pg_get_viewdef.go @@ -37,10 +37,10 @@ func initPgGetViewDef() { var pg_get_viewdef_oid = framework.Function1{ Name: "pg_get_viewdef", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { oidVal := val.(uint32) return getViewDef(ctx, oidVal) }, @@ -50,10 +50,10 @@ var pg_get_viewdef_oid = framework.Function1{ var pg_get_viewdef_oid_bool = framework.Function2{ Name: "pg_get_viewdef", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Bool}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Bool}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { oidVal := val1.(uint32) // TODO: pretty printing is not yet supported return getViewDef(ctx, oidVal) @@ -64,10 +64,10 @@ var pg_get_viewdef_oid_bool = framework.Function2{ var pg_get_viewdef_oid_int = framework.Function2{ Name: "pg_get_viewdef", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Int64}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { // TODO: prettyprint is implied, not yet supported // TODO: lines with fields are wrapped to specified number of columns return "", fmt.Errorf("not yet supported") diff --git a/server/functions/pg_indexes_size.go b/server/functions/pg_indexes_size.go index a387bc36d7..00c015de39 100644 --- a/server/functions/pg_indexes_size.go +++ b/server/functions/pg_indexes_size.go @@ -30,10 +30,10 @@ func initPgIndexesSize() { var pg_indexes_size_regclass = framework.Function1{ Name: "pg_indexes_size", Return: pgtypes.Int64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Regclass}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regclass}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { // TODO: Total disk space used by indexes attached to the specified table return int64(0), nil }, diff --git a/server/functions/pg_relation_size.go b/server/functions/pg_relation_size.go index 4c4e483978..6bd5175617 100644 --- a/server/functions/pg_relation_size.go +++ b/server/functions/pg_relation_size.go @@ -31,10 +31,10 @@ func initPgRelationSize() { var pg_relation_size_regclass = framework.Function1{ Name: "pg_relation_size", Return: pgtypes.Int64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Regclass}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regclass}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { // TODO: on-disk size in bytes of one fork of that relation // used by 'main' by default. return int64(0), nil @@ -45,10 +45,10 @@ var pg_relation_size_regclass = framework.Function1{ var pg_relation_size_regclass_text = framework.Function2{ Name: "pg_relation_size", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Regclass, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Regclass, pgtypes.Text}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { // TODO: on-disk size in bytes of one fork of that relation // used by the specified fork ('main', 'fsm', 'vm', or 'init') return int64(0), nil diff --git a/server/functions/pg_stat_get_numscans.go b/server/functions/pg_stat_get_numscans.go index fa9780ec7d..8307dd417a 100644 --- a/server/functions/pg_stat_get_numscans.go +++ b/server/functions/pg_stat_get_numscans.go @@ -30,10 +30,10 @@ func initPgStatGetNumscans() { var pg_stat_get_numscans_oid = framework.Function1{ Name: "pg_stat_get_numscans", Return: pgtypes.Int64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { return int64(0), nil }, } diff --git a/server/functions/pg_table_is_visible.go b/server/functions/pg_table_is_visible.go index 6ab60e77cd..54c7423b98 100644 --- a/server/functions/pg_table_is_visible.go +++ b/server/functions/pg_table_is_visible.go @@ -32,10 +32,10 @@ func initPgTableIsVisible() { var pg_table_is_visible_oid = framework.Function1{ Name: "pg_table_is_visible", Return: pgtypes.Bool, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { oidVal := val.(uint32) paths, err := resolve.SearchPath(ctx) if err != nil { diff --git a/server/functions/pg_table_size.go b/server/functions/pg_table_size.go index 01199d215d..94a656cc1d 100644 --- a/server/functions/pg_table_size.go +++ b/server/functions/pg_table_size.go @@ -30,10 +30,10 @@ func initPgTableSize() { var pg_table_size_regclass = framework.Function1{ Name: "pg_table_size", Return: pgtypes.Int64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Regclass}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regclass}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { // TODO: Disk space used by the specified table, excluding indexes (but including TOAST, free space map, and visibility map) return int64(0), nil }, diff --git a/server/functions/pg_tablespace_location.go b/server/functions/pg_tablespace_location.go index 6820071166..aca8a4c19d 100644 --- a/server/functions/pg_tablespace_location.go +++ b/server/functions/pg_tablespace_location.go @@ -30,10 +30,10 @@ func initPgTablespaceLocation() { var pg_tablespace_location_oid = framework.Function1{ Name: "pg_tablespace_location", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Oid}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Oid}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { return "", nil }, } diff --git a/server/functions/pg_total_relation_size.go b/server/functions/pg_total_relation_size.go index 78148fea55..88f7c62862 100644 --- a/server/functions/pg_total_relation_size.go +++ b/server/functions/pg_total_relation_size.go @@ -30,10 +30,10 @@ func initPgTotalRelationSize() { var pg_total_relation_size_regclass = framework.Function1{ Name: "pg_total_relation_size", Return: pgtypes.Int64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Regclass}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regclass}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { // TODO: Total disk space used by the specified table, including all indexes and TOAST data return int64(0), nil }, diff --git a/server/functions/power.go b/server/functions/power.go index 4b46a88258..9fe1a24869 100644 --- a/server/functions/power.go +++ b/server/functions/power.go @@ -34,9 +34,9 @@ func initPower() { var power_float64_float64 = framework.Function2{ Name: "power", Return: pgtypes.Float64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return math.Pow(val1.(float64), val2.(float64)), nil }, } @@ -45,9 +45,9 @@ var power_float64_float64 = framework.Function2{ var power_numeric_numeric = framework.Function2{ Name: "power", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { if val1 == nil || val2 == nil { return nil, nil } diff --git a/server/functions/quote_ident.go b/server/functions/quote_ident.go index c49a740981..0014cc794e 100644 --- a/server/functions/quote_ident.go +++ b/server/functions/quote_ident.go @@ -33,9 +33,9 @@ func initQuoteIdent() { var quote_ident_text = framework.Function1{ Name: "quote_ident", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { return fmt.Sprintf(`"%s"`, strings.Replace(val.(string), "\"", "\"\"", -1)), nil }, } diff --git a/server/functions/radians.go b/server/functions/radians.go index 44efc02b49..11d01da3d8 100644 --- a/server/functions/radians.go +++ b/server/functions/radians.go @@ -32,9 +32,9 @@ func initRadians() { var radians_float64 = framework.Function1{ Name: "radians", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return toRadians(val1.(float64)), nil }, } diff --git a/server/functions/regclass.go b/server/functions/regclass.go new file mode 100644 index 0000000000..0b480ff90c --- /dev/null +++ b/server/functions/regclass.go @@ -0,0 +1,82 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "encoding/binary" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initRegclass registers the functions to the catalog. +func initRegclass() { + framework.RegisterFunction(regclassin) + framework.RegisterFunction(regclassout) + framework.RegisterFunction(regclassrecv) + framework.RegisterFunction(regclasssend) +} + +// regclassin represents the PostgreSQL function of regclass type IO input. +var regclassin = framework.Function1{ + Name: "regclassin", + Return: pgtypes.Regclass, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return pgtypes.Regclass_IoInput(ctx, val.(string)) + }, +} + +// regclassout represents the PostgreSQL function of regclass type IO output. +var regclassout = framework.Function1{ + Name: "regclassout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regclass}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return pgtypes.Regclass_IoOutput(ctx, val.(uint32)) + }, +} + +// regclassrecv represents the PostgreSQL function of regclass type IO receive. +var regclassrecv = framework.Function1{ + Name: "regclassrecv", + Return: pgtypes.Regclass, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return binary.BigEndian.Uint32(data), nil + }, +} + +// regclasssend represents the PostgreSQL function of regclass type IO send. +var regclasssend = framework.Function1{ + Name: "regclasssend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regclass}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + retVal := make([]byte, 4) + binary.BigEndian.PutUint32(retVal, val.(uint32)) + return retVal, nil + }, +} diff --git a/server/functions/regproc.go b/server/functions/regproc.go new file mode 100644 index 0000000000..ba3887aed0 --- /dev/null +++ b/server/functions/regproc.go @@ -0,0 +1,82 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "encoding/binary" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initRegproc registers the functions to the catalog. +func initRegproc() { + framework.RegisterFunction(regprocin) + framework.RegisterFunction(regprocout) + framework.RegisterFunction(regprocrecv) + framework.RegisterFunction(regprocsend) +} + +// regprocin represents the PostgreSQL function of regproc type IO input. +var regprocin = framework.Function1{ + Name: "regprocin", + Return: pgtypes.Regproc, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return pgtypes.Regproc_IoInput(ctx, val.(string)) + }, +} + +// regprocout represents the PostgreSQL function of regproc type IO output. +var regprocout = framework.Function1{ + Name: "regprocout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regproc}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return pgtypes.Regproc_IoOutput(ctx, val.(uint32)) + }, +} + +// regprocrecv represents the PostgreSQL function of regproc type IO receive. +var regprocrecv = framework.Function1{ + Name: "regprocrecv", + Return: pgtypes.Regproc, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return binary.BigEndian.Uint32(data), nil + }, +} + +// regprocsend represents the PostgreSQL function of regproc type IO send. +var regprocsend = framework.Function1{ + Name: "regprocsend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regproc}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + retVal := make([]byte, 4) + binary.BigEndian.PutUint32(retVal, val.(uint32)) + return retVal, nil + }, +} diff --git a/server/functions/regtype.go b/server/functions/regtype.go new file mode 100644 index 0000000000..e8a5d505ad --- /dev/null +++ b/server/functions/regtype.go @@ -0,0 +1,82 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "encoding/binary" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initRegtype registers the functions to the catalog. +func initRegtype() { + framework.RegisterFunction(regtypein) + framework.RegisterFunction(regtypeout) + framework.RegisterFunction(regtyperecv) + framework.RegisterFunction(regtypesend) +} + +// regtypein represents the PostgreSQL function of regtype type IO input. +var regtypein = framework.Function1{ + Name: "regtypein", + Return: pgtypes.Regtype, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return pgtypes.Regtype_IoInput(ctx, val.(string)) + }, +} + +// regtypeout represents the PostgreSQL function of regtype type IO output. +var regtypeout = framework.Function1{ + Name: "regtypeout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regtype}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return pgtypes.Regtype_IoOutput(ctx, val.(uint32)) + }, +} + +// regtyperecv represents the PostgreSQL function of regtype type IO receive. +var regtyperecv = framework.Function1{ + Name: "regtyperecv", + Return: pgtypes.Regtype, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return binary.BigEndian.Uint32(data), nil + }, +} + +// regtypesend represents the PostgreSQL function of regtype type IO send. +var regtypesend = framework.Function1{ + Name: "regtypesend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Regtype}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + retVal := make([]byte, 4) + binary.BigEndian.PutUint32(retVal, val.(uint32)) + return retVal, nil + }, +} diff --git a/server/functions/repeat.go b/server/functions/repeat.go index 050a7403fd..77265d28e1 100644 --- a/server/functions/repeat.go +++ b/server/functions/repeat.go @@ -32,9 +32,9 @@ func initRepeat() { var repeat_text_int32 = framework.Function2{ Name: "repeat", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, str any, num any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, str any, num any) (any, error) { return strings.Repeat(str.(string), int(num.(int32))), nil }, } diff --git a/server/functions/replace.go b/server/functions/replace.go index f8ffb5eaec..365c6a8be9 100644 --- a/server/functions/replace.go +++ b/server/functions/replace.go @@ -32,9 +32,9 @@ func initReplace() { var replace_text_text_text = framework.Function3{ Name: "replace", Return: pgtypes.Text, - Parameters: [3]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text, pgtypes.Text}, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [4]pgtypes.DoltgresType, str any, from any, to any) (any, error) { + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, str any, from any, to any) (any, error) { if len(from.(string)) == 0 { return str, nil } diff --git a/server/functions/reverse.go b/server/functions/reverse.go index b14ab7c796..cfeffc3dcd 100644 --- a/server/functions/reverse.go +++ b/server/functions/reverse.go @@ -30,9 +30,9 @@ func initReverse() { var reverse_text = framework.Function1{ Name: "reverse", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { runes := []rune(val1.(string)) for i, j := 0, len(runes)-1; i < j; i, j = i+1, j-1 { runes[i], runes[j] = runes[j], runes[i] diff --git a/server/functions/right.go b/server/functions/right.go index 6f4b06ebfd..7811d9ddab 100644 --- a/server/functions/right.go +++ b/server/functions/right.go @@ -30,9 +30,9 @@ func initRight() { var right_text_int32 = framework.Function2{ Name: "right", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, strInt any, nInt any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, strInt any, nInt any) (any, error) { str := strInt.(string) n := nInt.(int32) if n >= 0 { diff --git a/server/functions/round.go b/server/functions/round.go index 44aa5203ed..7529d51d1e 100644 --- a/server/functions/round.go +++ b/server/functions/round.go @@ -35,9 +35,9 @@ func initRound() { var round_float64 = framework.Function1{ Name: "round", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1 == nil { return nil, nil } @@ -49,9 +49,9 @@ var round_float64 = framework.Function1{ var round_numeric = framework.Function1{ Name: "round", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1 == nil { return nil, nil } @@ -63,9 +63,9 @@ var round_numeric = framework.Function1{ var round_numeric_int64 = framework.Function2{ Name: "round", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { return val1.(decimal.Decimal).Round(int32(val2.(int64))), nil }, } diff --git a/server/functions/rpad.go b/server/functions/rpad.go index dac0891254..4a43b87ef2 100644 --- a/server/functions/rpad.go +++ b/server/functions/rpad.go @@ -31,10 +31,10 @@ func initRpad() { var rpad_text_int32 = framework.Function2{ Name: "rpad", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { - var unusedTypes [4]pgtypes.DoltgresType + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + var unusedTypes [4]*pgtypes.DoltgresType return rpad_text_int32_text.Callable(ctx, unusedTypes, val1, val2, " ") }, } @@ -43,9 +43,9 @@ var rpad_text_int32 = framework.Function2{ var rpad_text_int32_text = framework.Function3{ Name: "rpad", Return: pgtypes.Text, - Parameters: [3]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32, pgtypes.Text}, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [4]pgtypes.DoltgresType, str any, length any, fill any) (any, error) { + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, str any, length any, fill any) (any, error) { if length.(int32) <= 0 { return "", nil } diff --git a/server/functions/rtrim.go b/server/functions/rtrim.go index 7b791d7175..2ec8d3a4a7 100644 --- a/server/functions/rtrim.go +++ b/server/functions/rtrim.go @@ -31,10 +31,10 @@ func initRtrim() { var rtrim_text = framework.Function1{ Name: "rtrim", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { - var unusedTypes [3]pgtypes.DoltgresType + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { + var unusedTypes [3]*pgtypes.DoltgresType return rtrim_text_text.Callable(ctx, unusedTypes, val1, " ") }, } @@ -43,9 +43,9 @@ var rtrim_text = framework.Function1{ var rtrim_text_text = framework.Function2{ Name: "rtrim", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, str any, characters any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, str any, characters any) (any, error) { runes := []rune(str.(string)) trimChars := make(map[rune]struct{}) for _, c := range characters.(string) { diff --git a/server/functions/scale.go b/server/functions/scale.go index ef60a0a3a2..0280742fc6 100644 --- a/server/functions/scale.go +++ b/server/functions/scale.go @@ -31,9 +31,9 @@ func initScale() { var scale_numeric = framework.Function1{ Name: "scale", Return: pgtypes.Int32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, dt [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, dt [2]*pgtypes.DoltgresType, val1 any) (any, error) { res, err := min_scale_numeric.Callable(ctx, dt, val1) if res != nil { return res.(int32), err diff --git a/server/functions/set_config.go b/server/functions/set_config.go index 4c8d18b726..089f7c1d07 100644 --- a/server/functions/set_config.go +++ b/server/functions/set_config.go @@ -33,8 +33,8 @@ func initSetConfig() { var set_config_text_text_boolean = framework.Function3{ Name: "set_config", Return: pgtypes.Text, - Parameters: [3]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text, pgtypes.Bool}, - Callable: func(ctx *sql.Context, _ [4]pgtypes.DoltgresType, settingName any, newValue any, isLocal any) (any, error) { + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text, pgtypes.Bool}, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, settingName any, newValue any, isLocal any) (any, error) { if settingName == nil { return nil, fmt.Errorf("NULL value not allowed for configuration setting name") } diff --git a/server/functions/setval.go b/server/functions/setval.go index 9e9376947c..bdb16054d3 100644 --- a/server/functions/setval.go +++ b/server/functions/setval.go @@ -35,11 +35,11 @@ func initSetVal() { var setval_text_int64 = framework.Function2{ Name: "setval", Return: pgtypes.Int64, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int64}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int64}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { - var unusedTypes [4]pgtypes.DoltgresType + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + var unusedTypes [4]*pgtypes.DoltgresType return setval_text_int64_boolean.Callable(ctx, unusedTypes, val1, val2, true) }, } @@ -48,10 +48,10 @@ var setval_text_int64 = framework.Function2{ var setval_text_int64_boolean = framework.Function3{ Name: "setval", Return: pgtypes.Int64, - Parameters: [3]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int64, pgtypes.Bool}, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int64, pgtypes.Bool}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [4]pgtypes.DoltgresType, val1 any, val2 any, val3 any) (any, error) { + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1 any, val2 any, val3 any) (any, error) { collection, err := core.GetSequencesCollectionFromContext(ctx) if err != nil { return nil, err diff --git a/server/functions/shobj_description.go b/server/functions/shobj_description.go index 1b2f355a2b..f8736e61a7 100644 --- a/server/functions/shobj_description.go +++ b/server/functions/shobj_description.go @@ -30,10 +30,10 @@ func initShobjDescription() { var shobj_description = framework.Function2{ Name: "shobj_description", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Name}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Oid, pgtypes.Name}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1 any, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1 any, val2 any) (any, error) { // TODO: When we support comments this should return the comment for a // shared database object specified by its OID and the name of the // containing system catalog. diff --git a/server/functions/sign.go b/server/functions/sign.go index 5d12ffc877..17d8af2ffa 100644 --- a/server/functions/sign.go +++ b/server/functions/sign.go @@ -33,9 +33,9 @@ func initSign() { var sign_float64 = framework.Function1{ Name: "sign", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1.(float64) < 0 { return float64(-1), nil } else if val1.(float64) > 0 { @@ -50,9 +50,9 @@ var sign_float64 = framework.Function1{ var sign_numeric = framework.Function1{ Name: "sign", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return decimal.NewFromInt(int64(val1.(decimal.Decimal).Cmp(decimal.Zero))), nil }, } diff --git a/server/functions/sin.go b/server/functions/sin.go index 69f926c822..079d1254e5 100644 --- a/server/functions/sin.go +++ b/server/functions/sin.go @@ -32,9 +32,9 @@ func initSin() { var sin_float64 = framework.Function1{ Name: "sin", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Sin(val1.(float64)), nil }, } diff --git a/server/functions/sind.go b/server/functions/sind.go index a06ff9b93d..56745d7fef 100644 --- a/server/functions/sind.go +++ b/server/functions/sind.go @@ -32,9 +32,9 @@ func initSind() { var sind_float64 = framework.Function1{ Name: "sind", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Sin(toRadians(val1.(float64))), nil }, } diff --git a/server/functions/sinh.go b/server/functions/sinh.go index 8fc989ea01..286c677f1c 100644 --- a/server/functions/sinh.go +++ b/server/functions/sinh.go @@ -32,9 +32,9 @@ func initSinh() { var sinh_float64 = framework.Function1{ Name: "sinh", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Sinh(val1.(float64)), nil }, } diff --git a/server/functions/split_part.go b/server/functions/split_part.go index 06f65d6776..7e2b455109 100644 --- a/server/functions/split_part.go +++ b/server/functions/split_part.go @@ -34,9 +34,9 @@ func initSplitPart() { var split_part_text_text_int32 = framework.Function3{ Name: "split_part", Return: pgtypes.Text, - Parameters: [3]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text, pgtypes.Int32}, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [4]pgtypes.DoltgresType, str any, delimiter any, n any) (any, error) { + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, str any, delimiter any, n any) (any, error) { if n.(int32) == 0 { return nil, fmt.Errorf("field position must not be zero") } diff --git a/server/functions/sqrt.go b/server/functions/sqrt.go index bef067ff00..8bf016d438 100644 --- a/server/functions/sqrt.go +++ b/server/functions/sqrt.go @@ -35,9 +35,9 @@ func initSqrt() { var sqrt_float64 = framework.Function1{ Name: "sqrt", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1 == nil { return nil, nil } @@ -52,9 +52,9 @@ var sqrt_float64 = framework.Function1{ var sqrt_numeric = framework.Function1{ Name: "sqrt", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1.(decimal.Decimal).Cmp(decimal.Zero) == -1 { return nil, fmt.Errorf("cannot take square root of a negative number") } diff --git a/server/functions/strpos.go b/server/functions/strpos.go index e1ecbb3ee0..cf131969ec 100644 --- a/server/functions/strpos.go +++ b/server/functions/strpos.go @@ -32,9 +32,9 @@ func initStrpos() { var strpos_varchar = framework.Function2{ Name: "strpos", Return: pgtypes.Int32, - Parameters: [2]pgtypes.DoltgresType{pgtypes.VarChar, pgtypes.VarChar}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.VarChar, pgtypes.VarChar}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, str any, substring any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, str any, substring any) (any, error) { idx := strings.Index(str.(string), substring.(string)) if idx == -1 { return int32(0), nil diff --git a/server/functions/substr.go b/server/functions/substr.go index 7fbfac2a1e..3a92cee9bf 100644 --- a/server/functions/substr.go +++ b/server/functions/substr.go @@ -33,9 +33,9 @@ func initSubstr() { var substr_text_int32 = framework.Function2{ Name: "substr", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, str any, start any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, str any, start any) (any, error) { runes := []rune(str.(string)) if start.(int32) < 1 { start = int32(1) @@ -53,9 +53,9 @@ var substr_text_int32 = framework.Function2{ var substr_text_int32_int32 = framework.Function3{ Name: "substr", Return: pgtypes.Text, - Parameters: [3]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32, pgtypes.Int32}, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Int32, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [4]pgtypes.DoltgresType, str any, startInt any, countInt any) (any, error) { + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, str any, startInt any, countInt any) (any, error) { start := startInt.(int32) count := countInt.(int32) runes := []rune(str.(string)) diff --git a/server/functions/tan.go b/server/functions/tan.go index f6d451d4f1..4d49d7e61b 100644 --- a/server/functions/tan.go +++ b/server/functions/tan.go @@ -32,9 +32,9 @@ func initTan() { var tan_float64 = framework.Function1{ Name: "tan", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Tan(val1.(float64)), nil }, } diff --git a/server/functions/tand.go b/server/functions/tand.go index 92c58c91d1..d18ac1cbd7 100644 --- a/server/functions/tand.go +++ b/server/functions/tand.go @@ -32,9 +32,9 @@ func initTand() { var tand_float64 = framework.Function1{ Name: "tand", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Tan(toRadians(val1.(float64))), nil }, } diff --git a/server/functions/tanh.go b/server/functions/tanh.go index b830a6bf2f..430cdd374f 100644 --- a/server/functions/tanh.go +++ b/server/functions/tanh.go @@ -32,9 +32,9 @@ func initTanh() { var tanh_float64 = framework.Function1{ Name: "tanh", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return math.Tanh(val1.(float64)), nil }, } diff --git a/server/functions/text.go b/server/functions/text.go new file mode 100644 index 0000000000..8c419daad6 --- /dev/null +++ b/server/functions/text.go @@ -0,0 +1,124 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/utils" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initText registers the functions to the catalog. +func initText() { + framework.RegisterFunction(textin) + framework.RegisterFunction(textout) + framework.RegisterFunction(textrecv) + framework.RegisterFunction(textsend) + framework.RegisterFunction(bttextcmp) + framework.RegisterFunction(bttextnamecmp) +} + +// textin represents the PostgreSQL function of text type IO input. +var textin = framework.Function1{ + Name: "textin", + Return: pgtypes.Text, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(string), nil + }, +} + +// textout represents the PostgreSQL function of text type IO output. +var textout = framework.Function1{ + Name: "textout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(string), nil + }, +} + +// textrecv represents the PostgreSQL function of text type IO receive. +var textrecv = framework.Function1{ + Name: "textrecv", + Return: pgtypes.Text, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + reader := utils.NewReader(data) + return reader.String(), nil + }, +} + +// textsend represents the PostgreSQL function of text type IO send. +var textsend = framework.Function1{ + Name: "textsend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + str := val.(string) + writer := utils.NewWriter(uint64(len(str) + 4)) + writer.String(str) + return writer.Data(), nil + }, +} + +// bttextcmp represents the PostgreSQL function of text type compare. +var bttextcmp = framework.Function2{ + Name: "bttextcmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(string) + bb := val2.(string) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} + +// bttextnamecmp represents the PostgreSQL function of text type compare with name. +var bttextnamecmp = framework.Function2{ + Name: "bttextnamecmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(string) + bb := val2.(string) + if ab == bb { + return int32(0), nil + } else if ab < bb { + return int32(-1), nil + } else { + return int32(1), nil + } + }, +} diff --git a/server/functions/time.go b/server/functions/time.go new file mode 100644 index 0000000000..483a79ffd0 --- /dev/null +++ b/server/functions/time.go @@ -0,0 +1,143 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "time" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/postgres/parser/sem/tree" + "github.com/dolthub/doltgresql/postgres/parser/timeofday" + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initTime registers the functions to the catalog. +func initTime() { + framework.RegisterFunction(time_in) + framework.RegisterFunction(time_out) + framework.RegisterFunction(time_recv) + framework.RegisterFunction(time_send) + framework.RegisterFunction(timetypmodin) + framework.RegisterFunction(timetypmodout) + framework.RegisterFunction(time_cmp) +} + +// time_in represents the PostgreSQL function of time type IO input. +var time_in = framework.Function3{ + Name: "time_in", + Return: pgtypes.Time, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Cstring, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + input := val1.(string) + //oid := val2.(uint32) + //typmod := val3.(int32) + // TODO: decode typmod to precision + p := 6 + //if b.Precision == -1 { + // p = b.Precision + //} + t, _, err := tree.ParseDTime(nil, input, tree.TimeFamilyPrecisionToRoundDuration(int32(p))) + if err != nil { + return nil, err + } + return timeofday.TimeOfDay(*t).ToTime(), nil + }, +} + +// time_out represents the PostgreSQL function of time type IO output. +var time_out = framework.Function1{ + Name: "time_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Time}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(time.Time).Format("15:04:05.999999999"), nil + }, +} + +// time_recv represents the PostgreSQL function of time type IO receive. +var time_recv = framework.Function3{ + Name: "time_recv", + Return: pgtypes.Time, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Internal, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + data := val1.([]byte) + //oid := val2.(uint32) + //typmod := val3.(int32) + // TODO: decode typmod to precision + if len(data) == 0 { + return nil, nil + } + t := time.Time{} + if err := t.UnmarshalBinary(data); err != nil { + return nil, err + } + return t, nil + }, +} + +// time_send represents the PostgreSQL function of time type IO send. +var time_send = framework.Function1{ + Name: "time_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Time}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(time.Time).MarshalBinary() + }, +} + +// timetypmodin represents the PostgreSQL function of time type IO typmod input. +var timetypmodin = framework.Function1{ + Name: "timetypmodin", + Return: pgtypes.Int32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.CstringArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO: typmod=(precision<<16)∣scale + return nil, nil + }, +} + +// timetypmodout represents the PostgreSQL function of time type IO typmod output. +var timetypmodout = framework.Function1{ + Name: "timetypmodout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + // Precision = typmod & 0xFFFF + // Scale = (typmod >> 16) & 0xFFFF + return nil, nil + }, +} + +// time_cmp represents the PostgreSQL function of time type compare. +var time_cmp = framework.Function2{ + Name: "time_cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Time, pgtypes.Time}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(time.Time) + bb := val2.(time.Time) + return int32(ab.Compare(bb)), nil + }, +} diff --git a/server/functions/timestamp.go b/server/functions/timestamp.go new file mode 100644 index 0000000000..11a37604e5 --- /dev/null +++ b/server/functions/timestamp.go @@ -0,0 +1,142 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "time" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/postgres/parser/sem/tree" + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initTimestamp registers the functions to the catalog. +func initTimestamp() { + framework.RegisterFunction(timestamp_in) + framework.RegisterFunction(timestamp_out) + framework.RegisterFunction(timestamp_recv) + framework.RegisterFunction(timestamp_send) + framework.RegisterFunction(timestamptypmodin) + framework.RegisterFunction(timestamptypmodout) + framework.RegisterFunction(timestamp_cmp) +} + +// timestamp_in represents the PostgreSQL function of timestamp type IO input. +var timestamp_in = framework.Function3{ + Name: "timestamp_in", + Return: pgtypes.Timestamp, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Cstring, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + input := val1.(string) + //oid := val2.(uint32) + //typmod := val3.(int32) + // TODO: decode typmod to precision + p := 6 + //if b.Precision == -1 { + // p = b.Precision + //} + t, _, err := tree.ParseDTimestamp(nil, input, tree.TimeFamilyPrecisionToRoundDuration(int32(p))) + if err != nil { + return nil, err + } + return t.Time, nil + }, +} + +// timestamp_out represents the PostgreSQL function of timestamp type IO output. +var timestamp_out = framework.Function1{ + Name: "timestamp_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Timestamp}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(time.Time).Format("2006-01-02 15:04:05.999999999"), nil + }, +} + +// timestamp_recv represents the PostgreSQL function of timestamp type IO receive. +var timestamp_recv = framework.Function3{ + Name: "timestamp_recv", + Return: pgtypes.Timestamp, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Internal, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + data := val1.([]byte) + //oid := val2.(uint32) + //typmod := val3.(int32) + // TODO: decode typmod to precision + if len(data) == 0 { + return nil, nil + } + t := time.Time{} + if err := t.UnmarshalBinary(data); err != nil { + return nil, err + } + return t, nil + }, +} + +// timestamp_send represents the PostgreSQL function of timestamp type IO send. +var timestamp_send = framework.Function1{ + Name: "timestamp_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Timestamp}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(time.Time).MarshalBinary() + }, +} + +// timestamptypmodin represents the PostgreSQL function of timestamp type IO typmod input. +var timestamptypmodin = framework.Function1{ + Name: "timestamptypmodin", + Return: pgtypes.Int32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.CstringArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO: typmod=(precision<<16)∣scale + return nil, nil + }, +} + +// timestamptypmodout represents the PostgreSQL function of timestamp type IO typmod output. +var timestamptypmodout = framework.Function1{ + Name: "timestamptypmodout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + // Precision = typmod & 0xFFFF + // Scale = (typmod >> 16) & 0xFFFF + return nil, nil + }, +} + +// timestamp_cmp represents the PostgreSQL function of timestamp type compare. +var timestamp_cmp = framework.Function2{ + Name: "timestamp_cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Timestamp}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(time.Time) + bb := val2.(time.Time) + return int32(ab.Compare(bb)), nil + }, +} diff --git a/server/functions/timestamptz.go b/server/functions/timestamptz.go new file mode 100644 index 0000000000..a8a3bbe5d2 --- /dev/null +++ b/server/functions/timestamptz.go @@ -0,0 +1,156 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "time" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/postgres/parser/sem/tree" + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initTimestampTZ registers the functions to the catalog. +func initTimestampTZ() { + framework.RegisterFunction(timestamptz_in) + framework.RegisterFunction(timestamptz_out) + framework.RegisterFunction(timestamptz_recv) + framework.RegisterFunction(timestamptz_send) + framework.RegisterFunction(timestamptztypmodin) + framework.RegisterFunction(timestamptztypmodout) + framework.RegisterFunction(timestamptz_cmp) +} + +// timestamptz_in represents the PostgreSQL function of timestamptz type IO input. +var timestamptz_in = framework.Function3{ + Name: "timestamptz_in", + Return: pgtypes.TimestampTZ, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Cstring, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + input := val1.(string) + //oid := val2.(uint32) + //typmod := val3.(int32) + // TODO: decode typmod to precision + p := 6 + //if b.Precision == -1 { + // p = b.Precision + //} + loc, err := GetServerLocation(ctx) + if err != nil { + return nil, err + } + t, _, err := tree.ParseDTimestampTZ(nil, input, tree.TimeFamilyPrecisionToRoundDuration(int32(p)), loc) + if err != nil { + return nil, err + } + return t.Time, nil + }, +} + +// timestamptz_out represents the PostgreSQL function of timestamptz type IO output. +var timestamptz_out = framework.Function1{ + Name: "timestamptz_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.TimestampTZ}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + serverLoc, err := GetServerLocation(ctx) + if err != nil { + return "", err + } + t := val.(time.Time).In(serverLoc) + _, offset := t.Zone() + if offset%3600 != 0 { + return t.Format("2006-01-02 15:04:05.999999999-07:00"), nil + } else { + return t.Format("2006-01-02 15:04:05.999999999-07"), nil + } + }, +} + +// timestamptz_recv represents the PostgreSQL function of timestamptz type IO receive. +var timestamptz_recv = framework.Function3{ + Name: "timestamptz_recv", + Return: pgtypes.TimestampTZ, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Internal, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + data := val1.([]byte) + //oid := val2.(uint32) + //typmod := val3.(int32) + // TODO: decode typmod to precision + if len(data) == 0 { + return nil, nil + } + t := time.Time{} + if err := t.UnmarshalBinary(data); err != nil { + return nil, err + } + return t, nil + }, +} + +// timestamptz_send represents the PostgreSQL function of timestamptz type IO send. +var timestamptz_send = framework.Function1{ + Name: "timestamptz_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.TimestampTZ}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(time.Time).MarshalBinary() + }, +} + +// timestamptztypmodin represents the PostgreSQL function of timestamptz type IO typmod input. +var timestamptztypmodin = framework.Function1{ + Name: "timestamptztypmodin", + Return: pgtypes.Int32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.CstringArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO: typmod=(precision<<16)∣scale + return nil, nil + }, +} + +// timestamptztypmodout represents the PostgreSQL function of timestamptz type IO typmod output. +var timestamptztypmodout = framework.Function1{ + Name: "timestamptztypmodout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + // Precision = typmod & 0xFFFF + // Scale = (typmod >> 16) & 0xFFFF + return nil, nil + }, +} + +// timestamptz_cmp represents the PostgreSQL function of timestamptz type compare. +var timestamptz_cmp = framework.Function2{ + Name: "timestamptz_cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimestampTZ, pgtypes.TimestampTZ}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(time.Time) + bb := val2.(time.Time) + return int32(ab.Compare(bb)), nil + }, +} diff --git a/server/functions/timetz.go b/server/functions/timetz.go new file mode 100644 index 0000000000..5fba121dce --- /dev/null +++ b/server/functions/timetz.go @@ -0,0 +1,176 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "fmt" + "time" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/postgres/parser/sem/tree" + "github.com/dolthub/doltgresql/postgres/parser/timetz" + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initTimeTZ registers the functions to the catalog. +func initTimeTZ() { + framework.RegisterFunction(timetz_in) + framework.RegisterFunction(timetz_out) + framework.RegisterFunction(timetz_recv) + framework.RegisterFunction(timetz_send) + framework.RegisterFunction(timetztypmodin) + framework.RegisterFunction(timetztypmodout) + framework.RegisterFunction(timetz_cmp) +} + +// timetz_in represents the PostgreSQL function of timetz type IO input. +var timetz_in = framework.Function3{ + Name: "timetz_in", + Return: pgtypes.TimeTZ, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Cstring, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + input := val1.(string) + //oid := val2.(uint32) + //typmod := val3.(int32) + // TODO: decode typmod to precision + p := 6 + //if b.Precision == -1 { + // p = b.Precision + //} + loc, err := GetServerLocation(ctx) + if err != nil { + return nil, err + } + t, _, err := timetz.ParseTimeTZ(time.Now().In(loc), input, tree.TimeFamilyPrecisionToRoundDuration(int32(p))) + if err != nil { + return nil, err + } + return t.ToTime(), nil + }, +} + +// timetz_out represents the PostgreSQL function of timetz type IO output. +var timetz_out = framework.Function1{ + Name: "timetz_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.TimeTZ}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO: this always displays the time with an offset relevant to the server location + return timetz.MakeTimeTZFromTime(val.(time.Time)).String(), nil + }, +} + +// timetz_recv represents the PostgreSQL function of timetz type IO receive. +var timetz_recv = framework.Function3{ + Name: "timetz_recv", + Return: pgtypes.TimeTZ, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Internal, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + data := val1.([]byte) + //oid := val2.(uint32) + //typmod := val3.(int32) + // TODO: decode typmod to precision + if len(data) == 0 { + return nil, nil + } + t := time.Time{} + if err := t.UnmarshalBinary(data); err != nil { + return nil, err + } + return t, nil + }, +} + +// timetz_send represents the PostgreSQL function of timetz type IO send. +var timetz_send = framework.Function1{ + Name: "timetz_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.TimeTZ}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(time.Time).MarshalBinary() + }, +} + +// timetztypmodin represents the PostgreSQL function of timetz type IO typmod input. +var timetztypmodin = framework.Function1{ + Name: "timetztypmodin", + Return: pgtypes.Int32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.CstringArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO: typmod=(precision<<16)∣scale + return nil, nil + }, +} + +// timetztypmodout represents the PostgreSQL function of timetz type IO typmod output. +var timetztypmodout = framework.Function1{ + Name: "timetztypmodout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + // TODO + // Precision = typmod & 0xFFFF + // Scale = (typmod >> 16) & 0xFFFF + return nil, nil + }, +} + +// timetz_cmp represents the PostgreSQL function of timetz type compare. +var timetz_cmp = framework.Function2{ + Name: "timetz_cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.TimeTZ, pgtypes.TimeTZ}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(time.Time) + bb := val2.(time.Time) + return int32(ab.Compare(bb)), nil + }, +} + +// GetServerLocation returns timezone value set for the server. +func GetServerLocation(ctx *sql.Context) (*time.Location, error) { + if ctx == nil { + return time.Local, nil + } + val, err := ctx.GetSessionVariable(ctx, "timezone") + if err != nil { + return nil, err + } + + tz := val.(string) + loc, err := time.LoadLocation(tz) + if err == nil { + return loc, nil + } + + var t time.Time + if t, err = time.Parse("Z07", tz); err == nil { + } else if t, err = time.Parse("Z07:00", tz); err == nil { + } else if t, err = time.Parse("Z07:00:00", tz); err != nil { + return nil, err + } + + _, offsetSecsUnconverted := t.Zone() + return time.FixedZone(fmt.Sprintf("fixed offset:%d", offsetSecsUnconverted), -offsetSecsUnconverted), nil +} diff --git a/server/functions/timezone.go b/server/functions/timezone.go index 639506a048..025346c584 100644 --- a/server/functions/timezone.go +++ b/server/functions/timezone.go @@ -41,10 +41,10 @@ func initTimezone() { var timezone_interval_timestamptz = framework.Function2{ Name: "timezone", Return: pgtypes.Timestamp, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.TimestampTZ}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { dur := val1.(duration.Duration) t := val2.(time.Time) return t.UTC().Add(time.Duration(dur.Nanos())), nil @@ -55,10 +55,10 @@ var timezone_interval_timestamptz = framework.Function2{ var timezone_text_timestamptz = framework.Function2{ Name: "timezone", Return: pgtypes.Timestamp, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.TimestampTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.TimestampTZ}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { tz := val1.(string) timeVal := val2.(time.Time) newOffset, err := convertTzToOffsetSecs(tz) @@ -73,10 +73,10 @@ var timezone_text_timestamptz = framework.Function2{ var timezone_text_timetz = framework.Function2{ Name: "timezone", Return: pgtypes.TimeTZ, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.TimeTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.TimeTZ}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { tz := val1.(string) timeVal := val2.(time.Time) newOffset, err := convertTzToOffsetSecs(tz) @@ -93,10 +93,10 @@ var timezone_text_timetz = framework.Function2{ var timezone_interval_timetz = framework.Function2{ Name: "timezone", Return: pgtypes.TimeTZ, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.TimeTZ}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.TimeTZ}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { dur := val1.(duration.Duration) timeVal := val2.(time.Time) newOffset := int32(dur.Nanos() / NanosPerSec) @@ -110,17 +110,17 @@ var timezone_interval_timetz = framework.Function2{ var timezone_text_timestamp = framework.Function2{ Name: "timezone", Return: pgtypes.TimestampTZ, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Timestamp}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { tz := val1.(string) timeVal := val2.(time.Time) newOffset, err := convertTzToOffsetSecs(tz) if err != nil { return nil, err } - serverLoc, err := pgtypes.GetServerLocation(ctx) + serverLoc, err := GetServerLocation(ctx) if err != nil { return nil, err } @@ -132,13 +132,13 @@ var timezone_text_timestamp = framework.Function2{ var timezone_interval_timestamp = framework.Function2{ Name: "timezone", Return: pgtypes.TimestampTZ, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Timestamp}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Interval, pgtypes.Timestamp}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { dur := val1.(duration.Duration) timeVal := val2.(time.Time) - serverLoc, err := pgtypes.GetServerLocation(ctx) + serverLoc, err := GetServerLocation(ctx) if err != nil { return nil, err } diff --git a/server/functions/to_char.go b/server/functions/to_char.go index cfa572230b..9e61fda8bc 100644 --- a/server/functions/to_char.go +++ b/server/functions/to_char.go @@ -35,9 +35,9 @@ func initToChar() { var to_char_timestamp = framework.Function2{ Name: "to_char", Return: pgtypes.Text, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Text}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Timestamp, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, val1, val2 any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { timestamp := val1.(time.Time) format := val2.(string) diff --git a/server/functions/to_hex.go b/server/functions/to_hex.go index 0e19c825e4..09c00b20bc 100644 --- a/server/functions/to_hex.go +++ b/server/functions/to_hex.go @@ -33,9 +33,9 @@ func initToHex() { var to_hex_int32 = framework.Function1{ Name: "to_hex", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int32}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return fmt.Sprintf("%x", uint64(val1.(int32))), nil }, } @@ -44,9 +44,9 @@ var to_hex_int32 = framework.Function1{ var to_hex_int64 = framework.Function1{ Name: "to_hex", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return fmt.Sprintf("%x", uint64(val1.(int64))), nil }, } diff --git a/server/functions/to_regclass.go b/server/functions/to_regclass.go index b289793fe5..2b88191b5d 100644 --- a/server/functions/to_regclass.go +++ b/server/functions/to_regclass.go @@ -33,10 +33,10 @@ func initToRegclass() { var to_regclass_text = framework.Function1{ Name: "to_regclass", Return: pgtypes.Regclass, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { // If the string just represents a number, then we return nil. if _, err := strconv.ParseUint(val1.(string), 10, 32); err == nil { return nil, nil diff --git a/server/functions/to_regproc.go b/server/functions/to_regproc.go index ff39386443..ed6357b34f 100644 --- a/server/functions/to_regproc.go +++ b/server/functions/to_regproc.go @@ -33,10 +33,10 @@ func initToRegproc() { var to_regproc_text = framework.Function1{ Name: "to_regproc", Return: pgtypes.Regproc, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { // If the string just represents a number, then we return nil. if _, err := strconv.ParseUint(val1.(string), 10, 32); err == nil { return nil, nil diff --git a/server/functions/to_regtype.go b/server/functions/to_regtype.go index a2f9e049f9..74113443f6 100644 --- a/server/functions/to_regtype.go +++ b/server/functions/to_regtype.go @@ -33,10 +33,10 @@ func initToRegtype() { var to_regtype_text = framework.Function1{ Name: "to_regtype", Return: pgtypes.Regtype, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, IsNonDeterministic: true, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { // If the string just represents a number, then we return nil. if _, err := strconv.ParseUint(val1.(string), 10, 32); err == nil { return nil, nil diff --git a/server/functions/translate.go b/server/functions/translate.go index 9ab1b157d5..26780e1663 100644 --- a/server/functions/translate.go +++ b/server/functions/translate.go @@ -30,9 +30,9 @@ func initTranslate() { var translate_text_text_text = framework.Function3{ Name: "translate", Return: pgtypes.Text, - Parameters: [3]pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text, pgtypes.Text}, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Text, pgtypes.Text, pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [4]pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { str := val1.(string) from := []rune(val2.(string)) to := []rune(val3.(string)) diff --git a/server/functions/trim_scale.go b/server/functions/trim_scale.go index 07bce588dd..4de98277a2 100644 --- a/server/functions/trim_scale.go +++ b/server/functions/trim_scale.go @@ -32,9 +32,9 @@ func initTrimScale() { var trim_scale_numeric = framework.Function1{ Name: "trim_scale", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { // We don't store the scale in the value, so I'm not sure if this is functionally correct. // Seems like we'd need to modify the type of the return value (by trimming the scale), rather than the value itself. return val1.(decimal.Decimal), nil diff --git a/server/functions/trunc.go b/server/functions/trunc.go index 3eba055bd2..73eb83ca4d 100644 --- a/server/functions/trunc.go +++ b/server/functions/trunc.go @@ -35,9 +35,9 @@ func initTrunc() { var trunc_float64 = framework.Function1{ Name: "trunc", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1 == nil { return nil, nil } @@ -49,9 +49,9 @@ var trunc_float64 = framework.Function1{ var trunc_numeric = framework.Function1{ Name: "trunc", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { if val1 == nil { return nil, nil } @@ -63,9 +63,9 @@ var trunc_numeric = framework.Function1{ var trunc_numeric_int64 = framework.Function2{ Name: "trunc", Return: pgtypes.Numeric, - Parameters: [2]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Int32}, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [3]pgtypes.DoltgresType, num any, places any) (any, error) { + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, num any, places any) (any, error) { //TODO: test for negative values in places return num.(decimal.Decimal).Truncate(places.(int32)), nil }, diff --git a/server/functions/unary/minus.go b/server/functions/unary/minus.go index e614eb2415..4a85214a1d 100644 --- a/server/functions/unary/minus.go +++ b/server/functions/unary/minus.go @@ -42,9 +42,9 @@ func initUnaryMinus() { var float4um = framework.Function1{ Name: "float4um", Return: pgtypes.Float32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float32}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return -(val1.(float32)), nil }, } @@ -53,9 +53,9 @@ var float4um = framework.Function1{ var float8um = framework.Function1{ Name: "float8um", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return -(val1.(float64)), nil }, } @@ -64,9 +64,9 @@ var float8um = framework.Function1{ var int2um = framework.Function1{ Name: "int2um", Return: pgtypes.Int16, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int16}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return -(val1.(int16)), nil }, } @@ -75,9 +75,9 @@ var int2um = framework.Function1{ var int4um = framework.Function1{ Name: "int4um", Return: pgtypes.Int32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int32}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return -(val1.(int32)), nil }, } @@ -86,9 +86,9 @@ var int4um = framework.Function1{ var int8um = framework.Function1{ Name: "int8um", Return: pgtypes.Int64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return -(val1.(int64)), nil }, } @@ -97,9 +97,9 @@ var int8um = framework.Function1{ var interval_um = framework.Function1{ Name: "interval_um", Return: pgtypes.Interval, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Interval}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Interval}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { dur := val1.(duration.Duration) return dur.Mul(-1), nil }, @@ -109,9 +109,9 @@ var interval_um = framework.Function1{ var numeric_uminus = framework.Function1{ Name: "numeric_uminus", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return val1.(decimal.Decimal).Neg(), nil }, } diff --git a/server/functions/unary/plus.go b/server/functions/unary/plus.go index a2b37a0b03..1f59c82f1d 100644 --- a/server/functions/unary/plus.go +++ b/server/functions/unary/plus.go @@ -38,9 +38,9 @@ func initUnaryPlus() { var float4up = framework.Function1{ Name: "float4up", Return: pgtypes.Float32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float32}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float32}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return val1, nil }, } @@ -49,9 +49,9 @@ var float4up = framework.Function1{ var float8up = framework.Function1{ Name: "float8up", Return: pgtypes.Float64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Float64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Float64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return val1, nil }, } @@ -60,9 +60,9 @@ var float8up = framework.Function1{ var int2up = framework.Function1{ Name: "int2up", Return: pgtypes.Int16, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int16}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int16}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return val1, nil }, } @@ -71,9 +71,9 @@ var int2up = framework.Function1{ var int4up = framework.Function1{ Name: "int4up", Return: pgtypes.Int32, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int32}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return val1, nil }, } @@ -82,9 +82,9 @@ var int4up = framework.Function1{ var int8up = framework.Function1{ Name: "int8up", Return: pgtypes.Int64, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Int64}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int64}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return val1, nil }, } @@ -93,9 +93,9 @@ var int8up = framework.Function1{ var numeric_uplus = framework.Function1{ Name: "numeric_uplus", Return: pgtypes.Numeric, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Numeric}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Numeric}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { return val1, nil }, } diff --git a/server/functions/unknown.go b/server/functions/unknown.go new file mode 100644 index 0000000000..0a739da6fb --- /dev/null +++ b/server/functions/unknown.go @@ -0,0 +1,81 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/utils" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initUnknown registers the functions to the catalog. +func initUnknown() { + framework.RegisterFunction(unknownin) + framework.RegisterFunction(unknownout) + framework.RegisterFunction(unknownrecv) + framework.RegisterFunction(unknownsend) +} + +// unknownin represents the PostgreSQL function of unknown type IO input. +var unknownin = framework.Function1{ + Name: "unknownin", + Return: pgtypes.Unknown, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(string), nil + }, +} + +// unknownout represents the PostgreSQL function of unknown type IO output. +var unknownout = framework.Function1{ + Name: "unknownout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Unknown}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(string), nil + }, +} + +// unknownrecv represents the PostgreSQL function of unknown type IO receive. +var unknownrecv = framework.Function1{ + Name: "unknownrecv", + Return: pgtypes.Unknown, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + reader := utils.NewReader(data) + return reader.String(), nil + }, +} + +// unknownsend represents the PostgreSQL function of unknown type IO send. +var unknownsend = framework.Function1{ + Name: "unknownsend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Unknown}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + str := val.(string) + writer := utils.NewWriter(uint64(len(str) + 4)) + writer.String(str) + return writer.Data(), nil + }, +} diff --git a/server/functions/unnest.go b/server/functions/unnest.go index 55c74edbe0..b338e2292b 100644 --- a/server/functions/unnest.go +++ b/server/functions/unnest.go @@ -32,9 +32,9 @@ func initUnnest() { var unnest = framework.Function1{ Name: "unnest", Return: pgtypes.AnyElement, // TODO: Should return setof AnyElement - Parameters: [1]pgtypes.DoltgresType{pgtypes.AnyArray}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.AnyArray}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { valArr := val1.([]interface{}) if len(valArr) == 0 { return nil, nil diff --git a/server/functions/upper.go b/server/functions/upper.go index bc60bcad80..2e4e582f4d 100644 --- a/server/functions/upper.go +++ b/server/functions/upper.go @@ -32,9 +32,9 @@ func initUpper() { var upper_text = framework.Function1{ Name: "upper", Return: pgtypes.Text, - Parameters: [1]pgtypes.DoltgresType{pgtypes.Text}, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Text}, Strict: true, - Callable: func(ctx *sql.Context, _ [2]pgtypes.DoltgresType, val1 any) (any, error) { + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val1 any) (any, error) { //TODO: this doesn't respect collations return strings.ToUpper(val1.(string)), nil }, diff --git a/server/functions/uuid.go b/server/functions/uuid.go new file mode 100644 index 0000000000..4e80f2960d --- /dev/null +++ b/server/functions/uuid.go @@ -0,0 +1,95 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "bytes" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/postgres/parser/uuid" + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initUuid registers the functions to the catalog. +func initUuid() { + framework.RegisterFunction(uuid_in) + framework.RegisterFunction(uuid_out) + framework.RegisterFunction(uuid_recv) + framework.RegisterFunction(uuid_send) + framework.RegisterFunction(uuid_cmp) +} + +// uuid_in represents the PostgreSQL function of uuid type IO input. +var uuid_in = framework.Function1{ + Name: "uuid_in", + Return: pgtypes.Uuid, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return uuid.FromString(val.(string)) + }, +} + +// uuid_out represents the PostgreSQL function of uuid type IO output. +var uuid_out = framework.Function1{ + Name: "uuid_out", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Uuid}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(uuid.UUID).String(), nil + }, +} + +// uuid_recv represents the PostgreSQL function of uuid type IO receive. +var uuid_recv = framework.Function1{ + Name: "uuid_recv", + Return: pgtypes.Uuid, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return uuid.FromBytes(data) + }, +} + +// uuid_send represents the PostgreSQL function of uuid type IO send. +var uuid_send = framework.Function1{ + Name: "uuid_send", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Uuid}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return val.(uuid.UUID).GetBytes(), nil + }, +} + +// uuid_cmp represents the PostgreSQL function of uuid type compare. +var uuid_cmp = framework.Function2{ + Name: "uuid_cmp", + Return: pgtypes.Int32, + Parameters: [2]*pgtypes.DoltgresType{pgtypes.Uuid, pgtypes.Uuid}, + Strict: true, + Callable: func(ctx *sql.Context, _ [3]*pgtypes.DoltgresType, val1, val2 any) (any, error) { + ab := val1.(uuid.UUID) + bb := val2.(uuid.UUID) + return int32(bytes.Compare(ab.GetBytesMut(), bb.GetBytesMut())), nil + }, +} diff --git a/server/functions/varchar.go b/server/functions/varchar.go new file mode 100644 index 0000000000..7c15b6a2f6 --- /dev/null +++ b/server/functions/varchar.go @@ -0,0 +1,133 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "fmt" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" + "github.com/dolthub/doltgresql/utils" +) + +// initVarChar registers the functions to the catalog. +func initVarChar() { + framework.RegisterFunction(varcharin) + framework.RegisterFunction(varcharout) + framework.RegisterFunction(varcharrecv) + framework.RegisterFunction(varcharsend) + framework.RegisterFunction(varchartypmodin) + framework.RegisterFunction(varchartypmodout) +} + +// varcharin represents the PostgreSQL function of varchar type IO input. +var varcharin = framework.Function3{ + Name: "varcharin", + Return: pgtypes.VarChar, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Cstring, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + input := val1.(string) + typmod := val3.(int32) + maxChars := pgtypes.GetCharLengthFromTypmod(typmod) + if maxChars < pgtypes.StringUnbounded { + return input, nil + } + input, runeLength := truncateString(input, maxChars) + if runeLength > maxChars { + return input, fmt.Errorf("value too long for type varying(%v)", maxChars) + } else { + return input, nil + } + }, +} + +// varcharout represents the PostgreSQL function of varchar type IO output. +var varcharout = framework.Function1{ + Name: "varcharout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.VarChar}, + Strict: true, + Callable: func(ctx *sql.Context, t [2]*pgtypes.DoltgresType, val any) (any, error) { + v := val.(string) + typ := t[0] + tm := typ.GetAttTypMod() + if tm != -1 { + str, _ := truncateString(v, pgtypes.GetCharLengthFromTypmod(tm)) + return str, nil + } else { + return v, nil + } + }, +} + +// varcharrecv represents the PostgreSQL function of varchar type IO receive. +var varcharrecv = framework.Function3{ + Name: "varcharrecv", + Return: pgtypes.VarChar, + Parameters: [3]*pgtypes.DoltgresType{pgtypes.Internal, pgtypes.Oid, pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [4]*pgtypes.DoltgresType, val1, val2, val3 any) (any, error) { + data := val1.([]byte) + if len(data) == 0 { + return nil, nil + } + reader := utils.NewReader(data) + return reader.String(), nil + }, +} + +// varcharsend represents the PostgreSQL function of varchar type IO send. +var varcharsend = framework.Function1{ + Name: "varcharsend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.VarChar}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + str := val.(string) + writer := utils.NewWriter(uint64(len(str) + 4)) + writer.String(str) + return writer.Data(), nil + }, +} + +// varchartypmodin represents the PostgreSQL function of varchar type IO typmod input. +var varchartypmodin = framework.Function1{ + Name: "varchartypmodin", + Return: pgtypes.Int32, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.CstringArray}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return getTypModFromStringArr("varchar", val.([]any)) + }, +} + +// varchartypmodout represents the PostgreSQL function of varchar type IO typmod output. +var varchartypmodout = framework.Function1{ + Name: "varchartypmodout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Int32}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + typmod := val.(int32) + if typmod < 5 { + return "", nil + } + maxChars := pgtypes.GetCharLengthFromTypmod(typmod) + return fmt.Sprintf("(%v)", maxChars), nil + }, +} diff --git a/server/functions/width_bucket.go b/server/functions/width_bucket.go index ee20455cfb..b90edcc8d4 100644 --- a/server/functions/width_bucket.go +++ b/server/functions/width_bucket.go @@ -35,9 +35,9 @@ func initWidthBucket() { var width_bucket_float64_float64_float64_int64 = framework.Function4{ Name: "width_bucket", Return: pgtypes.Int32, - Parameters: [4]pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64, pgtypes.Float64, pgtypes.Int32}, + Parameters: [4]*pgtypes.DoltgresType{pgtypes.Float64, pgtypes.Float64, pgtypes.Float64, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [5]pgtypes.DoltgresType, operandInterface any, lowInterface any, highInterface any, countInterface any) (any, error) { + Callable: func(ctx *sql.Context, _ [5]*pgtypes.DoltgresType, operandInterface any, lowInterface any, highInterface any, countInterface any) (any, error) { operand := operandInterface.(float64) low := lowInterface.(float64) high := highInterface.(float64) @@ -68,9 +68,9 @@ var width_bucket_float64_float64_float64_int64 = framework.Function4{ var width_bucket_numeric_numeric_numeric_int64 = framework.Function4{ Name: "width_bucket", Return: pgtypes.Int32, - Parameters: [4]pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric, pgtypes.Numeric, pgtypes.Int32}, + Parameters: [4]*pgtypes.DoltgresType{pgtypes.Numeric, pgtypes.Numeric, pgtypes.Numeric, pgtypes.Int32}, Strict: true, - Callable: func(ctx *sql.Context, _ [5]pgtypes.DoltgresType, operandInterface any, lowInterface any, highInterface any, countInterface any) (any, error) { + Callable: func(ctx *sql.Context, _ [5]*pgtypes.DoltgresType, operandInterface any, lowInterface any, highInterface any, countInterface any) (any, error) { operand := operandInterface.(decimal.Decimal) low := lowInterface.(decimal.Decimal) high := highInterface.(decimal.Decimal) diff --git a/server/functions/xid.go b/server/functions/xid.go new file mode 100644 index 0000000000..cabaeb81fa --- /dev/null +++ b/server/functions/xid.go @@ -0,0 +1,89 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package functions + +import ( + "encoding/binary" + "strconv" + "strings" + + "github.com/dolthub/go-mysql-server/sql" + + "github.com/dolthub/doltgresql/server/functions/framework" + pgtypes "github.com/dolthub/doltgresql/server/types" +) + +// initXid registers the functions to the catalog. +func initXid() { + framework.RegisterFunction(xidin) + framework.RegisterFunction(xidout) + framework.RegisterFunction(xidrecv) + framework.RegisterFunction(xidsend) +} + +// xidin represents the PostgreSQL function of xid type IO input. +var xidin = framework.Function1{ + Name: "xidin", + Return: pgtypes.Xid, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Cstring}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + input := val.(string) + uVal, err := strconv.ParseInt(strings.TrimSpace(input), 10, 64) + if err != nil { + return uint32(0), nil + } + return uint32(uVal), nil + }, +} + +// xidout represents the PostgreSQL function of xid type IO output. +var xidout = framework.Function1{ + Name: "xidout", + Return: pgtypes.Cstring, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Xid}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + return strconv.FormatUint(uint64(val.(uint32)), 10), nil + }, +} + +// xidrecv represents the PostgreSQL function of xid type IO receive. +var xidrecv = framework.Function1{ + Name: "xidrecv", + Return: pgtypes.Xid, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Internal}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + data := val.([]byte) + if len(data) == 0 { + return nil, nil + } + return binary.BigEndian.Uint32(data), nil + }, +} + +// xidsend represents the PostgreSQL function of xid type IO send. +var xidsend = framework.Function1{ + Name: "xidsend", + Return: pgtypes.Bytea, + Parameters: [1]*pgtypes.DoltgresType{pgtypes.Xid}, + Strict: true, + Callable: func(ctx *sql.Context, _ [2]*pgtypes.DoltgresType, val any) (any, error) { + retVal := make([]byte, 4) + binary.BigEndian.PutUint32(retVal, val.(uint32)) + return retVal, nil + }, +} diff --git a/server/handler.go b/server/handler.go index c9d7851ac0..c83c00cb86 100644 --- a/server/handler.go +++ b/server/handler.go @@ -25,7 +25,7 @@ import ( type Handler interface { // ComBind is called when a connection receives a request to bind a prepared statement to a set of values. - ComBind(ctx context.Context, c *mysql.Conn, query string, parsedQuery mysql.ParsedQuery, bindVars map[string]sqlparser.Expr) (mysql.BoundQuery, []pgproto3.FieldDescription, error) + ComBind(ctx context.Context, c *mysql.Conn, query string, parsedQuery mysql.ParsedQuery, bindVars BindVariables) (mysql.BoundQuery, []pgproto3.FieldDescription, error) // ComExecuteBound is called when a connection receives a request to execute a prepared statement that has already bound to a set of values. ComExecuteBound(ctx context.Context, conn *mysql.Conn, query string, boundQuery mysql.BoundQuery, callback func(*Result) error) error // ComPrepareParsed is called when a connection receives a prepared statement query that has already been parsed. diff --git a/server/index/index_builder.go b/server/index/index_builder.go index a5ca801f02..2fb0f3888f 100644 --- a/server/index/index_builder.go +++ b/server/index/index_builder.go @@ -58,7 +58,7 @@ func NewIndexBuilder(ctx *sql.Context, indexes []sql.Index) (*IndexBuilder, erro columnName := strings.Replace(columnType.Expression, index.Table()+".", "", 1) columnMap[columnName] = columnIndex var ok bool - columns[columnIndex].typ, ok = columnType.Type.(pgtypes.DoltgresType) + columns[columnIndex].typ, ok = columnType.Type.(*pgtypes.DoltgresType) if !ok { return nil, fmt.Errorf("encountered a GMS type in the index `%s` on table `%s`", index.ID(), index.Table()) } @@ -190,7 +190,7 @@ func (ib *IndexBuilder) convertBinaryExpression(ctx *sql.Context, expr *pgexprs. operator = framework.Operator_BinaryGreaterOrEqual } } - valueType, ok := valueExpr.Type().(pgtypes.DoltgresType) + valueType, ok := valueExpr.Type().(*pgtypes.DoltgresType) if !ok { return indexBuilderExpr{isValid: false} } diff --git a/server/index/index_builder_column.go b/server/index/index_builder_column.go index cdde50d141..8f3ca654e7 100644 --- a/server/index/index_builder_column.go +++ b/server/index/index_builder_column.go @@ -16,9 +16,10 @@ package index import pgtypes "github.com/dolthub/doltgresql/server/types" -// indexBuilderColumn is a column within an indexBuilderElement, containing all of the expressions that should be -// applied to a column while iterating over the index. +// indexBuilderColumn is a column within an indexBuilderElement, +// containing all expressions that should be applied +// to a column while iterating over the index. type indexBuilderColumn struct { exprs []indexBuilderExpr - typ pgtypes.DoltgresType + typ *pgtypes.DoltgresType } diff --git a/server/index/split.go b/server/index/split.go index b8294bf101..13a6136f01 100644 --- a/server/index/split.go +++ b/server/index/split.go @@ -38,7 +38,7 @@ func SplitDisjunction(expr sql.Expression) []sql.Expression { // We should check to see if we need to preserve the cast on each child individually split := SplitDisjunction(expr.Child()) for i := range split { - if _, ok := split[i].Type().(pgtypes.DoltgresType); !ok { + if _, ok := split[i].Type().(*pgtypes.DoltgresType); !ok { split[i] = pgexprs.NewGMSCast(split[i]) } } @@ -65,7 +65,7 @@ func SplitConjunction(expr sql.Expression) []sql.Expression { // We should check to see if we need to preserve the cast on each child individually split := SplitConjunction(expr.Child()) for i := range split { - if _, ok := split[i].Type().(pgtypes.DoltgresType); !ok { + if _, ok := split[i].Type().(*pgtypes.DoltgresType); !ok { split[i] = pgexprs.NewGMSCast(split[i]) } } diff --git a/server/initialization/initialization.go b/server/initialization/initialization.go index c8992e6214..28471cccda 100644 --- a/server/initialization/initialization.go +++ b/server/initialization/initialization.go @@ -35,7 +35,6 @@ import ( "github.com/dolthub/doltgresql/server/tables/dtables" "github.com/dolthub/doltgresql/server/tables/information_schema" "github.com/dolthub/doltgresql/server/tables/pgcatalog" - pgtypes "github.com/dolthub/doltgresql/server/types" "github.com/dolthub/doltgresql/server/types/oid" doltgresservercfg "github.com/dolthub/doltgresql/servercfg" ) @@ -49,7 +48,6 @@ func Initialize(dEnv *env.DoltEnv) { auth.Init(dEnv) analyzer.Init() config.Init() - pgtypes.Init() oid.Init() binary.Init() unary.Init() diff --git a/server/node/create_domain.go b/server/node/create_domain.go index 2e66a3f615..69f0c47ec9 100644 --- a/server/node/create_domain.go +++ b/server/node/create_domain.go @@ -30,7 +30,7 @@ import ( type CreateDomain struct { SchemaName string Name string - AsType types.DoltgresType + AsType *types.DoltgresType Collation string HasDefault bool DefaultExpr sql.Expression @@ -67,7 +67,7 @@ func (c *CreateDomain) RowIter(ctx *sql.Context, r sql.Row) (sql.RowIter, error) return nil, fmt.Errorf(`role "%s" does not exist`, ctx.Client().User) } - // TODO: create array type with this type as base type? + // TODO: create array type with this type as base type var defExpr string if c.DefaultExpr != nil { defExpr = c.DefaultExpr.String() @@ -81,10 +81,6 @@ func (c *CreateDomain) RowIter(ctx *sql.Context, r sql.Row) (sql.RowIter, error) } } - newType, err := types.NewDomainType(ctx, c.SchemaName, c.Name, c.AsType, defExpr, c.IsNotNull, checkDefs, "") - if err != nil { - return nil, err - } schema, err := core.GetSchemaName(ctx, nil, c.SchemaName) if err != nil { return nil, err @@ -93,6 +89,8 @@ func (c *CreateDomain) RowIter(ctx *sql.Context, r sql.Row) (sql.RowIter, error) if err != nil { return nil, err } + + newType := types.NewDomainType(ctx, c.SchemaName, c.Name, c.AsType, defExpr, c.IsNotNull, checkDefs, "") err = collection.CreateType(schema, newType) if err != nil { return nil, err @@ -168,7 +166,7 @@ func (c *CreateDomain) WithResolvedChildren(children []any) (any, error) { // It is a placeholder column reference later // used for column defined as the domain type. type DomainColumn struct { - Typ types.DoltgresType + Typ *types.DoltgresType } var _ vitess.Injectable = (*DomainColumn)(nil) diff --git a/server/node/drop_domain.go b/server/node/drop_domain.go index 491726620b..3fbd135ec7 100644 --- a/server/node/drop_domain.go +++ b/server/node/drop_domain.go @@ -116,7 +116,7 @@ func (c *DropDomain) RowIter(ctx *sql.Context, r sql.Row) (sql.RowIter, error) { } if ok { for _, col := range t.Schema() { - if dt, isDomainType := col.Type.(types.DomainType); isDomainType { + if dt, isDoltgresType := col.Type.(*types.DoltgresType); isDoltgresType && dt.TypType == types.TypeType_Domain { if dt.Name == domain.Name { // TODO: issue a detail (list of all columns and tables that uses this domain) // and a hint (when we support CASCADE) diff --git a/server/tables/dtables/constraint_violations.go b/server/tables/dtables/constraint_violations.go index 41957df060..64a13ab7c3 100644 --- a/server/tables/dtables/constraint_violations.go +++ b/server/tables/dtables/constraint_violations.go @@ -25,7 +25,7 @@ import ( func getDoltConstraintViolationsBaseSqlSchema() sql.Schema { return []*sql.Column{ {Name: "from_root_ish", Type: pgtypes.Text, PrimaryKey: false, Nullable: true}, - {Name: "violation_type", Type: pgtypes.VarCharType{MaxChars: 16}, PrimaryKey: true}, + {Name: "violation_type", Type: pgtypes.MustCreateNewVarCharType(16), PrimaryKey: true}, } } diff --git a/server/tables/dtables/rebase.go b/server/tables/dtables/rebase.go index 6027942d3c..29f8d5ee35 100644 --- a/server/tables/dtables/rebase.go +++ b/server/tables/dtables/rebase.go @@ -30,7 +30,7 @@ import ( func getRebaseSchema() sql.Schema { return []*sql.Column{ {Name: "rebase_order", Type: pgtypes.Float32, Nullable: false, PrimaryKey: true}, // TODO: cannot have numeric key - {Name: "action", Type: pgtypes.VarCharType{MaxChars: 6}, Nullable: false}, // TODO: Should be enum(pick, squash, fixup, drop, reword) + {Name: "action", Type: pgtypes.MustCreateNewVarCharType(6), Nullable: false}, // TODO: Should be enum(pick, squash, fixup, drop, reword) {Name: "commit_hash", Type: pgtypes.Text, Nullable: false}, {Name: "commit_message", Type: pgtypes.Text, Nullable: false}, } diff --git a/server/tables/information_schema/columns_table.go b/server/tables/information_schema/columns_table.go index db9e03a2ee..3d2550b033 100644 --- a/server/tables/information_schema/columns_table.go +++ b/server/tables/information_schema/columns_table.go @@ -300,20 +300,17 @@ func getRowsFromDatabase(ctx *sql.Context, db information_schema.DbWithNames, al func getDataAndUdtType(colType sql.Type, colName string) (string, string) { udtName := "" dataType := "" - dgType, ok := colType.(pgtypes.DoltgresType) + dgType, ok := colType.(*pgtypes.DoltgresType) if ok { - udtName = dgType.BaseName() - if udtName == `"char"` { - udtName = `char` - } - if t, ok := partypes.OidToType[oid.Oid(dgType.OID())]; ok { + udtName = dgType.Name + if t, ok := partypes.OidToType[oid.Oid(dgType.OID)]; ok { dataType = t.SQLStandardName() } } else { dtdId := strings.Split(strings.Split(colType.String(), " COLLATE")[0], " CHARACTER SET")[0] // The DATA_TYPE value is the type name only with no other information - dataType := strings.Split(dtdId, "(")[0] + dataType = strings.Split(dtdId, "(")[0] dataType = strings.Split(dataType, " ")[0] udtName = dataType } @@ -323,22 +320,20 @@ func getDataAndUdtType(colType sql.Type, colName string) (string, string) { // getColumnPrecisionAndScale returns the precision or a number of postgres type. For non-numeric or decimal types this // function should return nil,nil. func getColumnPrecisionAndScale(colType sql.Type) (interface{}, interface{}, interface{}) { - dgt, ok := colType.(pgtypes.DoltgresType) + dgt, ok := colType.(*pgtypes.DoltgresType) if ok { - switch t := dgt.(type) { + switch oid.Oid(dgt.OID) { // TODO: BitType - case pgtypes.Float32Type, pgtypes.Float64Type: + case oid.T_float4, oid.T_float8: return typeToNumericPrecision[colType.Type()], int32(2), nil - case pgtypes.Int16Type, pgtypes.Int32Type, pgtypes.Int64Type: + case oid.T_int2, oid.T_int4, oid.T_int8: return typeToNumericPrecision[colType.Type()], int32(2), int32(0) - case pgtypes.NumericType: + case oid.T_numeric: var precision interface{} var scale interface{} - if t.Precision >= 0 { - precision = int32(t.Precision) - } - if t.Scale >= 0 { - scale = int32(t.Scale) + tm := dgt.GetAttTypMod() + if tm != -1 { + precision, scale = pgtypes.GetPrecisionAndScaleFromTypmod(tm) } return precision, int32(10), scale default: @@ -369,21 +364,16 @@ func getCharAndCollNamesAndCharMaxAndOctetLens(ctx *sql.Context, colType sql.Typ } switch t := colType.(type) { - case pgtypes.TextType: - charOctetLen = int32(maxCharacterOctetLength) - case pgtypes.VarCharType: - if t.IsUnbounded() { - charOctetLen = int32(maxCharacterOctetLength) - } else { - charOctetLen = int32(t.MaxChars) * 4 - charMaxLen = int32(t.MaxChars) - } - case pgtypes.CharType: - if t.IsUnbounded() { - charOctetLen = int32(maxCharacterOctetLength) - } else { - charOctetLen = int32(t.Length) * 4 - charMaxLen = int32(t.Length) + case *pgtypes.DoltgresType: + if t.TypCategory == pgtypes.TypeCategory_StringTypes { + tm := t.GetAttTypMod() + if tm == -1 { + charOctetLen = int32(maxCharacterOctetLength) + } else { + l := pgtypes.GetCharLengthFromTypmod(tm) + charOctetLen = l * 4 + charMaxLen = l + } } } @@ -391,11 +381,11 @@ func getCharAndCollNamesAndCharMaxAndOctetLens(ctx *sql.Context, colType sql.Typ } func getDatetimePrecision(colType sql.Type) interface{} { - if dgType, ok := colType.(pgtypes.DoltgresType); ok { - switch dgType.(type) { - case pgtypes.DateType: + if dgType, ok := colType.(*pgtypes.DoltgresType); ok { + switch oid.Oid(dgType.OID) { + case oid.T_date: return int32(0) - case pgtypes.TimeType, pgtypes.TimeTZType, pgtypes.TimestampType, pgtypes.TimestampTZType: + case oid.T_time, oid.T_timetz, oid.T_timestamp, oid.T_timestamptz: // TODO: TIME length not yet supported return int32(6) default: diff --git a/server/tables/information_schema/types.go b/server/tables/information_schema/types.go index f786be0cca..1aef3185d0 100644 --- a/server/tables/information_schema/types.go +++ b/server/tables/information_schema/types.go @@ -21,5 +21,5 @@ import ( // information_schema columns are one of these 5 types https://www.postgresql.org/docs/current/infoschema-datatypes.html var cardinal_number = pgtypes.Int32 var character_data = pgtypes.Text -var sql_identifier = pgtypes.VarCharType{MaxChars: 64} -var yes_or_no = pgtypes.VarCharType{MaxChars: 3} +var sql_identifier = pgtypes.MustCreateNewVarCharType(64) +var yes_or_no = pgtypes.MustCreateNewVarCharType(3) diff --git a/server/tables/pgcatalog/pg_attribute.go b/server/tables/pgcatalog/pg_attribute.go index f8487273b7..a735241cd1 100644 --- a/server/tables/pgcatalog/pg_attribute.go +++ b/server/tables/pgcatalog/pg_attribute.go @@ -152,12 +152,12 @@ func (iter *pgAttributeRowIter) Next(ctx *sql.Context) (sql.Row, error) { hasDefault := col.Default != nil typeOid := uint32(0) - if doltgresType, ok := col.Type.(pgtypes.DoltgresType); ok { - typeOid = doltgresType.OID() + if doltgresType, ok := col.Type.(*pgtypes.DoltgresType); ok { + typeOid = doltgresType.OID } else { // TODO: Remove once all information_schema tables are converted to use DoltgresType - doltgresType := pgtypes.FromGmsType(col.Type) - typeOid = doltgresType.OID() + dt := pgtypes.FromGmsType(col.Type) + typeOid = dt.OID } // TODO: Fill in the rest of the pg_attribute columns diff --git a/server/tables/pgcatalog/pg_catalog_cache.go b/server/tables/pgcatalog/pg_catalog_cache.go index cd060046b0..f21860eefd 100644 --- a/server/tables/pgcatalog/pg_catalog_cache.go +++ b/server/tables/pgcatalog/pg_catalog_cache.go @@ -69,7 +69,7 @@ type pgCatalogCache struct { viewSchemas []string // pg_types - types []pgtypes.DoltgresType + types []*pgtypes.DoltgresType pgCatalogOid uint32 // pg_tables diff --git a/server/tables/pgcatalog/pg_conversion.go b/server/tables/pgcatalog/pg_conversion.go index 1dc7f95f0f..ae829a1987 100644 --- a/server/tables/pgcatalog/pg_conversion.go +++ b/server/tables/pgcatalog/pg_conversion.go @@ -63,7 +63,7 @@ var PgConversionSchema = sql.Schema{ {Name: "conowner", Type: pgtypes.Oid, Default: nil, Nullable: false, Source: PgConversionName}, {Name: "conforencoding", Type: pgtypes.Int32, Default: nil, Nullable: false, Source: PgConversionName}, {Name: "contoencoding", Type: pgtypes.Int32, Default: nil, Nullable: false, Source: PgConversionName}, - {Name: "conproc", Type: pgtypes.Text, Default: nil, Nullable: false, Source: PgConversionName}, // TODDO: regproc type + {Name: "conproc", Type: pgtypes.Text, Default: nil, Nullable: false, Source: PgConversionName}, // TODO: regproc type {Name: "condefault", Type: pgtypes.Bool, Default: nil, Nullable: false, Source: PgConversionName}, } diff --git a/server/tables/pgcatalog/pg_type.go b/server/tables/pgcatalog/pg_type.go index e073718bad..9e428df7b9 100644 --- a/server/tables/pgcatalog/pg_type.go +++ b/server/tables/pgcatalog/pg_type.go @@ -15,9 +15,7 @@ package pgcatalog import ( - "fmt" "io" - "math" "github.com/dolthub/go-mysql-server/sql" @@ -67,7 +65,7 @@ func (p PgTypeHandler) RowIter(ctx *sql.Context) (sql.RowIter, error) { return nil, err } - var types []pgtypes.DoltgresType + var types []*pgtypes.DoltgresType err = oid.IterateCurrentDatabase(ctx, oid.Callbacks{ Type: func(ctx *sql.Context, typ oid.ItemType) (cont bool, err error) { types = append(types, typ.Item) @@ -135,7 +133,7 @@ var pgTypeSchema = sql.Schema{ // pgTypeRowIter is the sql.RowIter for the pg_type table. type pgTypeRowIter struct { pgCatalogOid uint32 - types []pgtypes.DoltgresType + types []*pgtypes.DoltgresType idx int } @@ -148,118 +146,42 @@ func (iter *pgTypeRowIter) Next(ctx *sql.Context) (sql.Row, error) { } iter.idx++ typ := iter.types[iter.idx-1] + // TODO: typ.Acl is stored as []string + typAcl := []any(nil) - var ( - typName = typ.BaseName() - typLen int16 - typByVal = false - typType = "b" - typCat = typ.Category() - typAlign = string(typ.Alignment()) - typStorage = "p" - typSubscript = "-" - typConvFnPrefix = typ.BaseName() - typConvFnSep = "" - typAnalyze = "-" - typModIn = "-" - typModOut = "-" - ) - - if l := typ.MaxTextResponseByteLength(ctx); l == math.MaxUint32 { - typLen = -1 - } else { - typLen = int16(l) - // TODO: below can be of different value for some exceptions - typByVal = true - typStorage = "x" - } - - // TODO: use the type information to fill these rather than manually doing it - switch t := typ.(type) { - case pgtypes.UnknownType: - typLen = -2 - case pgtypes.NumericType: - typStorage = "m" - case pgtypes.JsonType: - typConvFnSep = "_" - typStorage = "x" - case pgtypes.UuidType: - typConvFnSep = "_" - case pgtypes.DoltgresArrayType: - typStorage = "x" - typConvFnSep = "_" - if _, ok := typ.(pgtypes.DoltgresPolymorphicType); !ok { - typSubscript = "array_subscript_handler" - typConvFnPrefix = "array" - typAnalyze = "array_typanalyze" - typName = fmt.Sprintf("_%s", typName) - } else { - typType = "p" - } - if _, ok := t.BaseType().(pgtypes.InternalCharType); ok { - typName = "_char" - } - case pgtypes.InternalCharType: - typName = "char" - typConvFnPrefix = "char" - typStorage = "p" - case pgtypes.CharType: - typModIn = "bpchartypmodin" - typModOut = "bpchartypmodout" - typStorage = "x" - case pgtypes.DoltgresPolymorphicType: - typType = "p" - typConvFnSep = "_" - typByVal = true - } - - typIn := fmt.Sprintf("%s%sin", typConvFnPrefix, typConvFnSep) - typOut := fmt.Sprintf("%s%sout", typConvFnPrefix, typConvFnSep) - typRec := fmt.Sprintf("%s%srecv", typConvFnPrefix, typConvFnSep) - typSend := fmt.Sprintf("%s%ssend", typConvFnPrefix, typConvFnSep) - - // Non array polymorphic types do not have a receive or send functions - if _, ok := typ.(pgtypes.DoltgresPolymorphicType); ok { - if _, ok := typ.(pgtypes.DoltgresArrayType); !ok { - typRec = "-" - typSend = "-" - } - } - - // TODO: not all columns are populated return sql.Row{ - typ.OID(), //oid - typName, //typname - iter.pgCatalogOid, //typnamespace - uint32(0), //typowner - typLen, //typlen - typByVal, //typbyval - typType, //typtype - string(typCat), //typcategory - typ.IsPreferredType(), //typispreferred - true, //typisdefined - ",", //typdelim - uint32(0), //typrelid - typSubscript, //typsubscript - uint32(0), //typelem - uint32(0), //typarray - typIn, //typinput - typOut, //typoutput - typRec, //typreceive - typSend, //typsend - typModIn, //typmodin - typModOut, //typmodout - typAnalyze, //typanalyze - typAlign, //typalign - typStorage, //typstorage - false, //typnotnull - uint32(0), //typbasetype - int32(0), //typtypmod - int32(0), //typndims - uint32(0), //typcollation - nil, //typdefaultbin - nil, //typdefault - nil, //typacl + typ.OID, //oid + typ.Name, //typname + iter.pgCatalogOid, //typnamespace + uint32(0), //typowner + typ.TypLength, //typlen + typ.PassedByVal, //typbyval + string(typ.TypType), //typtype + string(typ.TypCategory), //typcategory + typ.IsPreferred, //typispreferred + typ.IsDefined, //typisdefined + typ.Delimiter, //typdelim + typ.RelID, //typrelid + typ.SubscriptFuncName(), //typsubscript + typ.Elem, //typelem + typ.Array, //typarray + typ.InputFuncName(), //typinput + typ.OutputFuncName(), //typoutput + typ.ReceiveFuncName(), //typreceive + typ.SendFuncName(), //typsend + typ.ModInFuncName(), //typmodin + typ.ModOutFuncName(), //typmodout + typ.AnalyzeFuncName(), //typanalyze + string(typ.Align), //typalign + string(typ.Storage), //typstorage + typ.NotNull, //typnotnull + typ.BaseTypeOID, //typbasetype + typ.TypMod, //typtypmod + typ.NDims, //typndims + typ.TypCollation, //typcollation + typ.DefaulBin, //typdefaultbin + typ.Default, //typdefault + typAcl, //typacl }, nil } diff --git a/server/types/any.go b/server/types/any.go new file mode 100644 index 0000000000..bceb8bd803 --- /dev/null +++ b/server/types/any.go @@ -0,0 +1,57 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package types + +import ( + "github.com/lib/pq/oid" +) + +// Any is a type that may contain any type. +var Any = &DoltgresType{ + OID: uint32(oid.T_any), + Name: "any", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Pseudo, + TypCategory: TypeCategory_PseudoTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: 0, + InputFunc: toFuncID("any_in", oid.T_cstring), + OutputFunc: toFuncID("any_out", oid.T_any), + ReceiveFunc: toFuncID("-"), + SendFunc: toFuncID("-"), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), +} diff --git a/server/types/any_array.go b/server/types/any_array.go index 9a9a87bd3b..3977ce82d1 100644 --- a/server/types/any_array.go +++ b/server/types/any_array.go @@ -15,187 +15,44 @@ package types import ( - "fmt" - "math" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) -// AnyArray is an array that may contain elements of any type. -var AnyArray = AnyArrayType{} - -// AnyArrayType is the extended type implementation of the PostgreSQL anyarray. -type AnyArrayType struct{} - -var _ DoltgresType = AnyArrayType{} -var _ DoltgresArrayType = AnyArrayType{} -var _ DoltgresPolymorphicType = AnyArrayType{} - -// Alignment implements the DoltgresType interface. -func (aa AnyArrayType) Alignment() TypeAlignment { - return TypeAlignment_Double -} - -// BaseID implements the DoltgresType interface. -func (aa AnyArrayType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_AnyArray -} - -// BaseName implements the DoltgresType interface. -func (aa AnyArrayType) BaseName() string { - return "anyarray" -} - -// BaseType implements the DoltgresArrayType interface. -func (aa AnyArrayType) BaseType() DoltgresType { - return Unknown -} - -// Category implements the DoltgresType interface. -func (aa AnyArrayType) Category() TypeCategory { - return TypeCategory_PseudoTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (aa AnyArrayType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (aa AnyArrayType) Compare(v1 any, v2 any) (int, error) { - return 0, fmt.Errorf("%s cannot compare values", aa.String()) -} - -// Convert implements the DoltgresType interface. -func (aa AnyArrayType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case []any: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", aa.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (aa AnyArrayType) Equals(otherType sql.Type) bool { - _, ok := otherType.(AnyArrayType) - return ok -} - -// FormatValue implements the DoltgresType interface. -func (aa AnyArrayType) FormatValue(val any) (string, error) { - return "", fmt.Errorf("%s cannot format values", aa.String()) -} - -// GetSerializationID implements the DoltgresType interface. -func (aa AnyArrayType) GetSerializationID() SerializationID { - return SerializationID_Invalid -} - -// IoInput implements the DoltgresType interface. -func (aa AnyArrayType) IoInput(ctx *sql.Context, input string) (any, error) { - return "", fmt.Errorf("%s cannot receive I/O input", aa.String()) -} - -// IoOutput implements the DoltgresType interface. -func (aa AnyArrayType) IoOutput(ctx *sql.Context, output any) (string, error) { - return "", fmt.Errorf("%s cannot produce I/O output", aa.String()) -} - -// IsPreferredType implements the DoltgresType interface. -func (aa AnyArrayType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (aa AnyArrayType) IsUnbounded() bool { - return true -} - -// IsValid implements the DoltgresPolymorphicType interface. -func (aa AnyArrayType) IsValid(target DoltgresType) bool { - _, ok := target.(DoltgresArrayType) - return ok -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (aa AnyArrayType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_Unbounded -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (aa AnyArrayType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return math.MaxUint32 -} - -// OID implements the DoltgresType interface. -func (aa AnyArrayType) OID() uint32 { - return uint32(oid.T_anyarray) -} - -// Promote implements the DoltgresType interface. -func (aa AnyArrayType) Promote() sql.Type { - return aa -} - -// SerializedCompare implements the DoltgresType interface. -func (aa AnyArrayType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - return 0, fmt.Errorf("%s cannot compare serialized values", aa.String()) -} - -// SQL implements the DoltgresType interface. -func (aa AnyArrayType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - return sqltypes.Value{}, fmt.Errorf("%s cannot output values in the wire format", aa.String()) -} - -// String implements the DoltgresType interface. -func (aa AnyArrayType) String() string { - return "anyarray" -} - -// ToArrayType implements the DoltgresType interface. -func (aa AnyArrayType) ToArrayType() DoltgresArrayType { - return aa -} - -// Type implements the DoltgresType interface. -func (aa AnyArrayType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (aa AnyArrayType) ValueType() reflect.Type { - return reflect.TypeOf([]any{}) -} - -// Zero implements the DoltgresType interface. -func (aa AnyArrayType) Zero() any { - return []any{} -} - -// SerializeType implements the DoltgresType interface. -func (aa AnyArrayType) SerializeType() ([]byte, error) { - return nil, fmt.Errorf("%s cannot be serialized", aa.String()) -} - -// deserializeType implements the DoltgresType interface. -func (aa AnyArrayType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - return nil, fmt.Errorf("%s cannot be deserialized", aa.String()) -} - -// SerializeValue implements the DoltgresType interface. -func (aa AnyArrayType) SerializeValue(val any) ([]byte, error) { - return nil, fmt.Errorf("%s cannot serialize values", aa.String()) -} - -// DeserializeValue implements the DoltgresType interface. -func (aa AnyArrayType) DeserializeValue(val []byte) (any, error) { - return nil, fmt.Errorf("%s cannot deserialize values", aa.String()) +// AnyArray is a pseudo-type that can represent any type +// that is an array type that may contain elements of any type. +var AnyArray = &DoltgresType{ + OID: uint32(oid.T_anyarray), + Name: "anyarray", + Schema: "pg_catalog", + TypLength: int16(-1), + PassedByVal: false, + TypType: TypeType_Pseudo, + TypCategory: TypeCategory_PseudoTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: 0, + InputFunc: toFuncID("anyarray_in", oid.T_cstring), + OutputFunc: toFuncID("anyarray_out", oid.T_anyarray), + ReceiveFunc: toFuncID("anyarray_recv", oid.T_internal), + SendFunc: toFuncID("anyarray_send", oid.T_anyarray), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Double, + Storage: TypeStorage_Extended, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btarraycmp", oid.T_anyarray, oid.T_anyarray), } diff --git a/server/types/any_element.go b/server/types/any_element.go index 3b90c40b5a..d1241e1503 100644 --- a/server/types/any_element.go +++ b/server/types/any_element.go @@ -15,175 +15,43 @@ package types import ( - "fmt" - "math" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // AnyElement is a pseudo-type that can represent any type. -var AnyElement = AnyElementType{} - -// AnyElementType is the extended type implementation of the PostgreSQL anyelement. -type AnyElementType struct{} - -var _ DoltgresType = AnyElementType{} -var _ DoltgresPolymorphicType = AnyElementType{} - -// Alignment implements the DoltgresType interface. -func (ae AnyElementType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (ae AnyElementType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_AnyElement -} - -// BaseName implements the DoltgresType interface. -func (ae AnyElementType) BaseName() string { - return "anyelement" -} - -// Category implements the DoltgresType interface. -func (ae AnyElementType) Category() TypeCategory { - return TypeCategory_PseudoTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (ae AnyElementType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (ae AnyElementType) Compare(v1 any, v2 any) (int, error) { - return 0, fmt.Errorf("%s cannot compare values", ae.String()) -} - -// Convert implements the DoltgresType interface. -func (ae AnyElementType) Convert(val any) (any, sql.ConvertInRange, error) { - return val, sql.InRange, nil -} - -// Equals implements the DoltgresType interface. -func (ae AnyElementType) Equals(otherType sql.Type) bool { - _, ok := otherType.(AnyElementType) - return ok -} - -// FormatValue implements the DoltgresType interface. -func (ae AnyElementType) FormatValue(val any) (string, error) { - return "", fmt.Errorf("%s cannot format values", ae.String()) -} - -// GetSerializationID implements the DoltgresType interface. -func (ae AnyElementType) GetSerializationID() SerializationID { - return SerializationID_Invalid -} - -// IoInput implements the DoltgresType interface. -func (ae AnyElementType) IoInput(ctx *sql.Context, input string) (any, error) { - return "", fmt.Errorf("%s cannot receive I/O input", ae.String()) -} - -// IoOutput implements the DoltgresType interface. -func (ae AnyElementType) IoOutput(ctx *sql.Context, output any) (string, error) { - return "", fmt.Errorf("%s cannot produce I/O output", ae.String()) -} - -// IsPreferredType implements the DoltgresType interface. -func (ae AnyElementType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (ae AnyElementType) IsUnbounded() bool { - return true -} - -// IsValid implements the DoltgresPolymorphicType interface. -func (ae AnyElementType) IsValid(target DoltgresType) bool { - return true -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (ae AnyElementType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_Unbounded -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (ae AnyElementType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return math.MaxUint32 -} - -// OID implements the DoltgresType interface. -func (ae AnyElementType) OID() uint32 { - return uint32(oid.T_anyelement) -} - -// Promote implements the DoltgresType interface. -func (ae AnyElementType) Promote() sql.Type { - return ae -} - -// SerializedCompare implements the DoltgresType interface. -func (ae AnyElementType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - return 0, fmt.Errorf("%s cannot compare serialized values", ae.String()) -} - -// SQL implements the DoltgresType interface. -func (ae AnyElementType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - return sqltypes.Value{}, fmt.Errorf("%s cannot output values in the wire format", ae.String()) -} - -// String implements the DoltgresType interface. -func (ae AnyElementType) String() string { - return "anyelement" -} - -// ToArrayType implements the DoltgresType interface. -func (ae AnyElementType) ToArrayType() DoltgresArrayType { - return Unknown -} - -// Type implements the DoltgresType interface. -func (ae AnyElementType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (ae AnyElementType) ValueType() reflect.Type { - var val any - return reflect.TypeOf(val) -} - -// Zero implements the DoltgresType interface. -func (ae AnyElementType) Zero() any { - var val any - return val -} - -// SerializeType implements the DoltgresType interface. -func (ae AnyElementType) SerializeType() ([]byte, error) { - return nil, fmt.Errorf("%s cannot be serialized", ae.String()) -} - -// deserializeType implements the DoltgresType interface. -func (ae AnyElementType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - return nil, fmt.Errorf("%s cannot be deserialized", ae.String()) -} - -// SerializeValue implements the DoltgresType interface. -func (ae AnyElementType) SerializeValue(val any) ([]byte, error) { - return nil, fmt.Errorf("%s cannot serialize values", ae.String()) -} - -// DeserializeValue implements the DoltgresType interface. -func (ae AnyElementType) DeserializeValue(val []byte) (any, error) { - return nil, fmt.Errorf("%s cannot deserialize values", ae.String()) +var AnyElement = &DoltgresType{ + OID: uint32(oid.T_anyelement), + Name: "anyelement", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Pseudo, + TypCategory: TypeCategory_PseudoTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: 0, + InputFunc: toFuncID("anyelement_in", oid.T_cstring), + OutputFunc: toFuncID("anyelement_out", oid.T_anyelement), + ReceiveFunc: toFuncID("-"), + SendFunc: toFuncID("-"), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), } diff --git a/server/types/any_nonarray.go b/server/types/any_nonarray.go index c7caa1aeff..82cd4865c6 100644 --- a/server/types/any_nonarray.go +++ b/server/types/any_nonarray.go @@ -15,181 +15,43 @@ package types import ( - "fmt" - "math" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // AnyNonArray is a pseudo-type that can represent any type that isn't an array type. -var AnyNonArray = AnyNonArrayType{} - -// AnyNonArrayType is the extended type implementation of the PostgreSQL anynonarray. -type AnyNonArrayType struct{} - -var _ DoltgresType = AnyNonArrayType{} -var _ DoltgresPolymorphicType = AnyNonArrayType{} - -// Alignment implements the DoltgresType interface. -func (ana AnyNonArrayType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (ana AnyNonArrayType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_AnyNonArray -} - -// BaseName implements the DoltgresType interface. -func (ana AnyNonArrayType) BaseName() string { - return "anynonarray" -} - -// Category implements the DoltgresType interface. -func (ana AnyNonArrayType) Category() TypeCategory { - return TypeCategory_PseudoTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (ana AnyNonArrayType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (ana AnyNonArrayType) Compare(v1 any, v2 any) (int, error) { - return 0, fmt.Errorf("%s cannot compare values", ana.String()) -} - -// Convert implements the DoltgresType interface. -func (ana AnyNonArrayType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case []any: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", ana.String(), val) - default: - return val, sql.InRange, nil - } -} - -// Equals implements the DoltgresType interface. -func (ana AnyNonArrayType) Equals(otherType sql.Type) bool { - _, ok := otherType.(AnyNonArrayType) - return ok -} - -// FormatValue implements the DoltgresType interface. -func (ana AnyNonArrayType) FormatValue(val any) (string, error) { - return "", fmt.Errorf("%s cannot format values", ana.String()) -} - -// GetSerializationID implements the DoltgresType interface. -func (ana AnyNonArrayType) GetSerializationID() SerializationID { - return SerializationID_Invalid -} - -// IoInput implements the DoltgresType interface. -func (ana AnyNonArrayType) IoInput(ctx *sql.Context, input string) (any, error) { - return "", fmt.Errorf("%s cannot receive I/O input", ana.String()) -} - -// IoOutput implements the DoltgresType interface. -func (ana AnyNonArrayType) IoOutput(ctx *sql.Context, output any) (string, error) { - return "", fmt.Errorf("%s cannot produce I/O output", ana.String()) -} - -// IsPreferredType implements the DoltgresType interface. -func (ana AnyNonArrayType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (ana AnyNonArrayType) IsUnbounded() bool { - return true -} - -// IsValid implements the DoltgresPolymorphicType interface. -func (ana AnyNonArrayType) IsValid(target DoltgresType) bool { - _, ok := target.(DoltgresArrayType) - return !ok -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (ana AnyNonArrayType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_Unbounded -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (ana AnyNonArrayType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return math.MaxUint32 -} - -// OID implements the DoltgresType interface. -func (ana AnyNonArrayType) OID() uint32 { - return uint32(oid.T_anynonarray) -} - -// Promote implements the DoltgresType interface. -func (ana AnyNonArrayType) Promote() sql.Type { - return ana -} - -// SerializedCompare implements the DoltgresType interface. -func (ana AnyNonArrayType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - return 0, fmt.Errorf("%s cannot compare serialized values", ana.String()) -} - -// SQL implements the DoltgresType interface. -func (ana AnyNonArrayType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - return sqltypes.Value{}, fmt.Errorf("%s cannot output values in the wire format", ana.String()) -} - -// String implements the DoltgresType interface. -func (ana AnyNonArrayType) String() string { - return "anynonarray" -} - -// ToArrayType implements the DoltgresType interface. -func (ana AnyNonArrayType) ToArrayType() DoltgresArrayType { - return Unknown -} - -// Type implements the DoltgresType interface. -func (ana AnyNonArrayType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (ana AnyNonArrayType) ValueType() reflect.Type { - var val any - return reflect.TypeOf(val) -} - -// Zero implements the DoltgresType interface. -func (ana AnyNonArrayType) Zero() any { - var val any - return val -} - -// SerializeType implements the DoltgresType interface. -func (ana AnyNonArrayType) SerializeType() ([]byte, error) { - return nil, fmt.Errorf("%s cannot be serialized", ana.String()) -} - -// deserializeType implements the DoltgresType interface. -func (ana AnyNonArrayType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - return nil, fmt.Errorf("%s cannot be deserialized", ana.String()) -} - -// SerializeValue implements the DoltgresType interface. -func (ana AnyNonArrayType) SerializeValue(val any) ([]byte, error) { - return nil, fmt.Errorf("%s cannot serialize values", ana.String()) -} - -// DeserializeValue implements the DoltgresType interface. -func (ana AnyNonArrayType) DeserializeValue(val []byte) (any, error) { - return nil, fmt.Errorf("%s cannot deserialize values", ana.String()) +var AnyNonArray = &DoltgresType{ + OID: uint32(oid.T_anynonarray), + Name: "anynonarray", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Pseudo, + TypCategory: TypeCategory_PseudoTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: 0, + InputFunc: toFuncID("anynonarray_in", oid.T_cstring), + OutputFunc: toFuncID("anynonarray_out", oid.T_anynonarray), + ReceiveFunc: toFuncID("-"), + SendFunc: toFuncID("-"), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), } diff --git a/server/types/array.go b/server/types/array.go index 0bb860bd14..cbdb2dc9df 100644 --- a/server/types/array.go +++ b/server/types/array.go @@ -15,505 +15,52 @@ package types import ( - "bytes" - "encoding/binary" "fmt" - "math" - "reflect" - "strings" - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" - - "github.com/dolthub/doltgresql/utils" ) -// arrayContainer is a type that wraps non-array types, giving them array functionality without requiring a bespoke -// implementation. -type arrayContainer struct { - innerType DoltgresType - serializationID SerializationID - oid oid.Oid - funcs arrayContainerFunctions -} - -// arrayContainerFunctions are overrides for the default array implementations of specific functions. If they are left -// nil, then it uses the default implementation. -type arrayContainerFunctions struct { - // SQL is similar to the function with the same name that is found on sql.Type. This just takes an additional - // arrayContainer parameter. - SQL func(ctx *sql.Context, ac arrayContainer, dest []byte, valInterface any) (sqltypes.Value, error) -} - -var _ DoltgresType = arrayContainer{} -var _ DoltgresArrayType = arrayContainer{} - -// createArrayType creates an array variant of the given type. Uses the default array implementations for all possible -// overrides. -func createArrayType(innerType DoltgresType, serializationID SerializationID, arrayOid oid.Oid) DoltgresArrayType { - return createArrayTypeWithFuncs(innerType, serializationID, arrayOid, arrayContainerFunctions{}) -} - -// createArrayTypeWithFuncs creates an array variant of the given type. Uses the provided function overrides if they're -// not nil. If any are nil, then they use the default array implementations. -func createArrayTypeWithFuncs(innerType DoltgresType, serializationID SerializationID, arrayOid oid.Oid, funcs arrayContainerFunctions) DoltgresArrayType { - if funcs.SQL == nil { - funcs.SQL = arrayContainerSQL - } - return arrayContainer{ - innerType: innerType, - serializationID: serializationID, - oid: arrayOid, - funcs: funcs, - } -} - -// Alignment implements the DoltgresType interface. -func (ac arrayContainer) Alignment() TypeAlignment { - return ac.innerType.Alignment() -} - -// BaseID implements the DoltgresType interface. -func (ac arrayContainer) BaseID() DoltgresTypeBaseID { - // The serializationID might be enough, but it's technically possible for us to use the same serialization ID with - // different inner types, so this ensures uniqueness. It is safe to change base IDs in the future (unlike - // serialization IDs, which must never be changed, only added to), so we can change this at any time if we feel it - // is necessary to. - return (1 << 31) | (DoltgresTypeBaseID(ac.serializationID) << 16) | ac.innerType.BaseID() -} - -// BaseName implements the DoltgresType interface. -func (ac arrayContainer) BaseName() string { - return ac.innerType.BaseName() -} - -// BaseType implements the DoltgresArrayType interface. -func (ac arrayContainer) BaseType() DoltgresType { - return ac.innerType -} - -// Category implements the DoltgresType interface. -func (ac arrayContainer) Category() TypeCategory { - return TypeCategory_ArrayTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (ac arrayContainer) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (ac arrayContainer) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ab, ok := v1.([]any) - if !ok { - return 0, fmt.Errorf("%s: unhandled type: %T", ac.String(), v1) - } - bb, ok := v2.([]any) - if !ok { - return 0, fmt.Errorf("%s: unhandled type: %T", ac.String(), v2) - } - - minLength := utils.Min(len(ab), len(bb)) - for i := 0; i < minLength; i++ { - res, err := ac.innerType.Compare(ab[i], bb[i]) - if err != nil { - return 0, err - } - if res != 0 { - return res, nil - } - } - if len(ab) == len(bb) { - return 0, nil - } else if len(ab) < len(bb) { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (ac arrayContainer) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case []any: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", ac.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (ac arrayContainer) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(ac), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (ac arrayContainer) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return ac.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (ac arrayContainer) GetSerializationID() SerializationID { - return ac.serializationID -} - -// IoInput implements the DoltgresType interface. -func (ac arrayContainer) IoInput(ctx *sql.Context, input string) (any, error) { - if len(input) < 2 || input[0] != '{' || input[len(input)-1] != '}' { - // This error is regarded as a critical error, and thus we immediately return the error alongside a nil - // value. Returning a nil value is a signal to not ignore the error. - return nil, fmt.Errorf(`malformed array literal: "%s"`, input) - } - // We'll remove the surrounding braces since we've already verified that they're there - input = input[1 : len(input)-1] - var values []any - var err error - sb := strings.Builder{} - quoteStartCount := 0 - quoteEndCount := 0 - escaped := false - // Iterate over each rune in the input to collect and process the rune elements - for _, r := range input { - if escaped { - sb.WriteRune(r) - escaped = false - } else if quoteStartCount > quoteEndCount { - switch r { - case '\\': - escaped = true - case '"': - quoteEndCount++ - default: - sb.WriteRune(r) - } - } else { - switch r { - case ' ', '\t', '\n', '\r': - continue - case '\\': - escaped = true - case '"': - quoteStartCount++ - case ',': - if quoteStartCount >= 2 { - // This is a malformed string, thus we treat it as a critical error. - return nil, fmt.Errorf(`malformed array literal: "%s"`, input) - } - str := sb.String() - var innerValue any - if quoteStartCount == 0 && strings.EqualFold(str, "null") { - // An unquoted case-insensitive NULL is treated as an actual null value - innerValue = nil - } else { - var nErr error - innerValue, nErr = ac.innerType.IoInput(ctx, str) - if nErr != nil && err == nil { - // This is a non-critical error, therefore the error may be ignored at a higher layer (such as - // an explicit cast) and the inner type will still return a valid result, so we must allow the - // values to propagate. - err = nErr - } - } - values = append(values, innerValue) - sb.Reset() - quoteStartCount = 0 - quoteEndCount = 0 - default: - sb.WriteRune(r) - } - } - } - // Use anything remaining in the buffer as the last element - if sb.Len() > 0 { - if escaped || quoteStartCount > quoteEndCount || quoteStartCount >= 2 { - // These errors are regarded as critical errors, and thus we immediately return the error alongside a nil - // value. Returning a nil value is a signal to not ignore the error. - return nil, fmt.Errorf(`malformed array literal: "%s"`, input) - } else { - str := sb.String() - var innerValue any - if quoteStartCount == 0 && strings.EqualFold(str, "NULL") { - // An unquoted case-insensitive NULL is treated as an actual null value - innerValue = nil - } else { - var nErr error - innerValue, nErr = ac.innerType.IoInput(ctx, str) - if nErr != nil && err == nil { - // This is a non-critical error, therefore the error may be ignored at a higher layer (such as - // an explicit cast) and the inner type will still return a valid result, so we must allow the - // values to propagate. - err = nErr - } - } - values = append(values, innerValue) - } - } - - return values, err -} - -// IoOutput implements the DoltgresType interface. -func (ac arrayContainer) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := ac.Convert(output) - if err != nil { - return "", err - } - sb := strings.Builder{} - sb.WriteRune('{') - for i, v := range converted.([]any) { - if i > 0 { - sb.WriteString(",") - } - if v != nil { - str, err := ac.innerType.IoOutput(ctx, v) - if err != nil { - return "", err - } - shouldQuote := false - for _, r := range str { - switch r { - case ' ', ',', '{', '}', '\\', '"': - shouldQuote = true - } - } - if shouldQuote || strings.EqualFold(str, "NULL") { - sb.WriteRune('"') - sb.WriteString(strings.ReplaceAll(str, `"`, `\"`)) - sb.WriteRune('"') - } else { - sb.WriteString(str) - } - } else { - sb.WriteString("NULL") - } - } - sb.WriteRune('}') - return sb.String(), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (ac arrayContainer) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (ac arrayContainer) IsUnbounded() bool { - return true -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (ac arrayContainer) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_Unbounded -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (ac arrayContainer) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return math.MaxUint32 -} - -// OID implements the DoltgresType interface. -func (ac arrayContainer) OID() uint32 { - return uint32(ac.oid) -} - -// Promote implements the DoltgresType interface. -func (ac arrayContainer) Promote() sql.Type { - return ac -} - -// SerializedCompare implements the DoltgresType interface. -func (ac arrayContainer) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - //TODO: write a far more optimized version of this that does not deserialize the entire arrays at once - dv1, err := ac.DeserializeValue(v1) - if err != nil { - return 0, err - } - dv2, err := ac.DeserializeValue(v2) - if err != nil { - return 0, err - } - return ac.Compare(dv1, dv2) -} - -// SQL implements the DoltgresType interface. -func (ac arrayContainer) SQL(ctx *sql.Context, dest []byte, valInterface any) (sqltypes.Value, error) { - return ac.funcs.SQL(ctx, ac, dest, valInterface) -} - -// String implements the DoltgresType interface. -func (ac arrayContainer) String() string { - return ac.innerType.String() + "[]" -} - -// ToArrayType implements the DoltgresType interface. -func (ac arrayContainer) ToArrayType() DoltgresArrayType { - return ac -} - -// Type implements the DoltgresType interface. -func (ac arrayContainer) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (ac arrayContainer) ValueType() reflect.Type { - return reflect.TypeOf([]any{}) -} - -// Zero implements the DoltgresType interface. -func (ac arrayContainer) Zero() any { - return []any{} -} - -// SerializeType implements the DoltgresType interface. -func (ac arrayContainer) SerializeType() ([]byte, error) { - innerSerialized, err := ac.innerType.SerializeType() - if err != nil { - return nil, err - } - serialized := make([]byte, serializationIDHeaderSize+len(innerSerialized)) - copy(serialized, ac.serializationID.ToByteSlice(0)) - copy(serialized[serializationIDHeaderSize:], innerSerialized) - return serialized, nil -} - -// deserializeType implements the DoltgresType interface. -func (ac arrayContainer) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - innerType, err := DeserializeType(metadata) - if err != nil { - return nil, err - } - return innerType.(DoltgresType).ToArrayType(), nil - default: - return nil, fmt.Errorf("version %d is not yet supported for arrays", version) - } -} - -// SerializeValue implements the DoltgresType interface. -func (ac arrayContainer) SerializeValue(valInterface any) ([]byte, error) { - // The binary format is as follows: - // The first value is always the number of serialized elements (uint32). - // The next section contains offsets to the start of each element (uint32). There are N+1 offsets to elements. - // The last offset contains the length of the slice. - // The last section is the data section, where all elements store their data. - // Each element comprises two values: a single byte stating if it's null, and the data itself. - // You may determine the length of the data by using the following offset, as the data occupies all bytes up to the next offset. - // The last element is a special case, as its data simply occupies all bytes up to the end of the slice. - // The data may have a length of zero, which is distinct from null for some types. - // In addition, a null value will always have a data length of zero. - // This format allows for O(1) point lookups. - - // Check for a nil value and convert to the expected type - if valInterface == nil { - return nil, nil - } - converted, _, err := ac.Convert(valInterface) - if err != nil { - return nil, err - } - vals := converted.([]any) - - bb := bytes.Buffer{} - // Write the element count to a buffer. We're using an array since it's stack-allocated, so no need for pooling. - var elementCount [4]byte - binary.LittleEndian.PutUint32(elementCount[:], uint32(len(vals))) - bb.Write(elementCount[:]) - // Create an array that contains the offsets for each value. Since we can't update the offset portion of the buffer - // as we determine the offsets, we have to track them outside the buffer. We'll overwrite the buffer later with the - // correct offsets. The last offset represents the end of the slice, which simplifies the logic for reading elements - // using the "current offset to next offset" strategy. We use a byte slice since the buffer only works with byte - // slices. - offsets := make([]byte, (len(vals)+1)*4) - bb.Write(offsets) - // The starting offset for the first element is Count(uint32) + (NumberOfElementOffsets * sizeof(uint32)) - currentOffset := uint32(4 + (len(vals)+1)*4) - for i := range vals { - // Write the current offset - binary.LittleEndian.PutUint32(offsets[i*4:], currentOffset) - // Handle serialization of the value - // TODO: ARRAYs may be multidimensional, such as ARRAY[[4,2],[6,3]], which isn't accounted for here - serializedVal, err := ac.innerType.SerializeValue(vals[i]) - if err != nil { - return nil, err - } - // Handle the nil case and non-nil case - if serializedVal == nil { - bb.WriteByte(1) - currentOffset += 1 - } else { - bb.WriteByte(0) - bb.Write(serializedVal) - currentOffset += 1 + uint32(len(serializedVal)) - } - } - // Write the final offset, which will equal the length of the serialized slice - binary.LittleEndian.PutUint32(offsets[len(offsets)-4:], currentOffset) - // Get the final output, and write the updated offsets to it - outputBytes := bb.Bytes() - copy(outputBytes[4:], offsets) - return outputBytes, nil -} - -// DeserializeValue implements the DoltgresType interface. -func (ac arrayContainer) DeserializeValue(serializedVals []byte) (_ any, err error) { - // Check for the nil value, then ensure the minimum length of the slice - if serializedVals == nil { - return nil, nil - } - if len(serializedVals) < 4 { - return nil, fmt.Errorf("deserializing non-nil array value has invalid length of %d", len(serializedVals)) - } - // Grab the number of elements and construct an output slice of the appropriate size - elementCount := binary.LittleEndian.Uint32(serializedVals) - output := make([]any, elementCount) - // Read all elements - for i := uint32(0); i < elementCount; i++ { - // We read from i+1 to account for the element count at the beginning - offset := binary.LittleEndian.Uint32(serializedVals[(i+1)*4:]) - // If the value is null, then we can skip it, since the output slice default initializes all values to nil - if serializedVals[offset] == 1 { - continue - } - // The element data is everything from the offset to the next offset, excluding the null determinant - nextOffset := binary.LittleEndian.Uint32(serializedVals[(i+2)*4:]) - output[i], err = ac.innerType.DeserializeValue(serializedVals[offset+1 : nextOffset]) - if err != nil { - return nil, err - } - } - // Returns all of the read elements - return output, nil -} - -// arrayContainerSQL implements the default SQL function for arrayContainer. -func arrayContainerSQL(ctx *sql.Context, ac arrayContainer, dest []byte, value any) (sqltypes.Value, error) { - if value == nil { - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(""))), nil - } - str, err := ac.IoOutput(ctx, value) - if err != nil { - return sqltypes.Value{}, err +// CreateArrayTypeFromBaseType create array type from given type. +func CreateArrayTypeFromBaseType(baseType *DoltgresType) *DoltgresType { + align := TypeAlignment_Int + if baseType.Align == TypeAlignment_Double { + align = TypeAlignment_Double + } + return &DoltgresType{ + OID: baseType.Array, + Name: fmt.Sprintf("_%s", baseType.Name), + Schema: "pg_catalog", + TypLength: int16(-1), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_ArrayTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("array_subscript_handler", oid.T_internal), + Elem: baseType.OID, + Array: 0, + InputFunc: toFuncID("array_in", oid.T_cstring, oid.T_oid, oid.T_int4), + OutputFunc: toFuncID("array_out", oid.T_anyarray), + ReceiveFunc: toFuncID("array_recv", oid.T_internal, oid.T_oid, oid.T_int4), + SendFunc: toFuncID("array_send", oid.T_anyarray), + ModInFunc: baseType.ModInFunc, + ModOutFunc: baseType.ModOutFunc, + AnalyzeFunc: toFuncID("array_typanalyze", oid.T_internal), + Align: align, + Storage: TypeStorage_Extended, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: baseType.TypCollation, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + InternalName: fmt.Sprintf("%s[]", baseType.Name), // This will be set to the proper name in ToArrayType + attTypMod: baseType.attTypMod, // TODO: check + CompareFunc: toFuncID("btarraycmp", oid.T_anyarray, oid.T_anyarray), } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(str))), nil } diff --git a/server/types/bool.go b/server/types/bool.go index da3f4a6cfe..ad213e074f 100644 --- a/server/types/bool.go +++ b/server/types/bool.go @@ -15,268 +15,45 @@ package types import ( - "bytes" - "fmt" - "reflect" - "strings" - "github.com/lib/pq/oid" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" ) -// Bool is the standard boolean. -var Bool = BoolType{} - -// BoolType is the extended type implementation of the PostgreSQL boolean. -type BoolType struct{} - -var _ DoltgresType = BoolType{} - -// Alignment implements the DoltgresType interface. -func (b BoolType) Alignment() TypeAlignment { - return TypeAlignment_Char -} - -// BaseID implements the DoltgresType interface. -func (b BoolType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Bool -} - -// BaseName implements the DoltgresType interface. -func (b BoolType) BaseName() string { - return "bool" -} - -// Category implements the DoltgresType interface. -func (b BoolType) Category() TypeCategory { - return TypeCategory_BooleanTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b BoolType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b BoolType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(bool) - bb := bc.(bool) - if ab == bb { - return 0, nil - } else if !ab { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b BoolType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case bool: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b BoolType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b BoolType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b BoolType) GetSerializationID() SerializationID { - return SerializationID_Bool -} - -// IoInput implements the DoltgresType interface. -func (b BoolType) IoInput(ctx *sql.Context, input string) (any, error) { - input = strings.TrimSpace(strings.ToLower(input)) - if input == "true" || input == "t" || input == "yes" || input == "on" || input == "1" { - return true, nil - } else if input == "false" || input == "f" || input == "no" || input == "off" || input == "0" { - return false, nil - } else { - return nil, fmt.Errorf("invalid input syntax for type %s: %q", b.String(), input) - } -} - -// IoOutput implements the DoltgresType interface. -func (b BoolType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - if converted.(bool) { - return "true", nil - } else { - return "false", nil - } -} - -// IsPreferredType implements the DoltgresType interface. -func (b BoolType) IsPreferredType() bool { - return true -} - -// IsUnbounded implements the DoltgresType interface. -func (b BoolType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b BoolType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b BoolType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 1 -} - -// OID implements the DoltgresType interface. -func (b BoolType) OID() uint32 { - return uint32(oid.T_bool) -} - -// Promote implements the DoltgresType interface. -func (b BoolType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b BoolType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - if v1[0] == v2[0] { - return 0, nil - } else if v1[0] == 0 { - return -1, nil - } else { - return 1, nil - } -} - -// SQL implements the DoltgresType interface. -func (b BoolType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, _, err := b.Convert(v) - if err != nil { - return sqltypes.Value{}, err - } - var valBytes []byte - if value.(bool) { - //TODO: use Wireshark and check whether we're returning these strings or something else - valBytes = types.AppendAndSliceBytes(dest, []byte{'t'}) - } else { - valBytes = types.AppendAndSliceBytes(dest, []byte{'f'}) - } - return sqltypes.MakeTrusted(sqltypes.Text, valBytes), nil -} - -// String implements the DoltgresType interface. -func (b BoolType) String() string { - return "boolean" -} - -// ToArrayType implements the DoltgresType interface. -func (b BoolType) ToArrayType() DoltgresArrayType { - return BoolArray -} - -// Type implements the DoltgresType interface. -func (b BoolType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b BoolType) ValueType() reflect.Type { - return reflect.TypeOf(bool(false)) -} - -// Zero implements the DoltgresType interface. -func (b BoolType) Zero() any { - return false -} - -// SerializeType implements the DoltgresType interface. -func (b BoolType) SerializeType() ([]byte, error) { - return SerializationID_Bool.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b BoolType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Bool, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b BoolType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - if converted.(bool) { - return []byte{1}, nil - } else { - return []byte{0}, nil - } -} - -// DeserializeValue implements the DoltgresType interface. -func (b BoolType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - return val[0] != 0, nil +// Bool is the bool type. +var Bool = &DoltgresType{ + OID: uint32(oid.T_bool), + Name: "bool", + Schema: "pg_catalog", + Owner: "doltgres", + TypLength: int16(1), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_BooleanTypes, + IsPreferred: true, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__bool), + InputFunc: toFuncID("boolin", oid.T_cstring), + OutputFunc: toFuncID("boolout", oid.T_bool), + ReceiveFunc: toFuncID("boolrecv", oid.T_internal), + SendFunc: toFuncID("boolsend", oid.T_bool), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Char, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btboolcmp", oid.T_bool, oid.T_bool), + InternalName: "boolean", } diff --git a/server/types/bool_array.go b/server/types/bool_array.go index 5b17d975e2..72e4164d63 100644 --- a/server/types/bool_array.go +++ b/server/types/bool_array.go @@ -14,41 +14,5 @@ package types -import ( - "bytes" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/lib/pq/oid" -) - // BoolArray is the array variant of Bool. -var BoolArray = createArrayTypeWithFuncs(Bool, SerializationID_BoolArray, oid.T__bool, arrayContainerFunctions{ - SQL: func(ctx *sql.Context, ac arrayContainer, dest []byte, valInterface any) (sqltypes.Value, error) { - if valInterface == nil { - return sqltypes.NULL, nil - } - converted, _, err := ac.Convert(valInterface) - if err != nil { - return sqltypes.Value{}, err - } - vals := converted.([]any) - bb := bytes.Buffer{} - bb.WriteRune('{') - for i := range vals { - if i > 0 { - bb.WriteRune(',') - } - if vals[i] == nil { - bb.WriteString("NULL") - } else if vals[i].(bool) { - bb.WriteRune('t') - } else { - bb.WriteRune('f') - } - } - bb.WriteRune('}') - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, bb.Bytes())), nil - }, -}) +var BoolArray = CreateArrayTypeFromBaseType(Bool) diff --git a/server/types/bytea.go b/server/types/bytea.go index 974ce6de4f..4a6cdc5912 100644 --- a/server/types/bytea.go +++ b/server/types/bytea.go @@ -15,244 +15,43 @@ package types import ( - "bytes" - "encoding/hex" - "fmt" - "math" - "reflect" - "strings" - - "github.com/dolthub/doltgresql/utils" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Bytea is the byte string type. -var Bytea = ByteaType{} - -// ByteaType is the extended type implementation of the PostgreSQL bytea. -type ByteaType struct{} - -var _ DoltgresType = ByteaType{} - -// Alignment implements the DoltgresType interface. -func (b ByteaType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b ByteaType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Bytea -} - -// BaseName implements the DoltgresType interface. -func (b ByteaType) BaseName() string { - return "bytea" -} - -// Category implements the DoltgresType interface. -func (b ByteaType) Category() TypeCategory { - return TypeCategory_UserDefinedTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b ByteaType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b ByteaType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.([]byte) - bb := bc.([]byte) - return bytes.Compare(ab, bb), nil -} - -// Convert implements the DoltgresType interface. -func (b ByteaType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case []byte: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b ByteaType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b ByteaType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b ByteaType) GetSerializationID() SerializationID { - return SerializationID_Bytea -} - -// IoInput implements the DoltgresType interface. -func (b ByteaType) IoInput(ctx *sql.Context, input string) (any, error) { - if strings.HasPrefix(input, `\x`) { - return hex.DecodeString(input[2:]) - } else { - return []byte(input), nil - } -} - -// IoOutput implements the DoltgresType interface. -func (b ByteaType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return `\x` + hex.EncodeToString(converted.([]byte)), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b ByteaType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b ByteaType) IsUnbounded() bool { - return true -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b ByteaType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_Unbounded -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b ByteaType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return math.MaxUint32 -} - -// OID implements the DoltgresType interface. -func (b ByteaType) OID() uint32 { - return uint32(oid.T_bytea) -} - -// Promote implements the DoltgresType interface. -func (b ByteaType) Promote() sql.Type { - return Bytea -} - -// SerializedCompare implements the DoltgresType interface. -func (b ByteaType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - return serializedStringCompare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b ByteaType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Blob, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b ByteaType) String() string { - return "bytea" -} - -// ToArrayType implements the DoltgresType interface. -func (b ByteaType) ToArrayType() DoltgresArrayType { - return ByteaArray -} - -// Type implements the DoltgresType interface. -func (b ByteaType) Type() query.Type { - return sqltypes.Blob -} - -// ValueType implements the DoltgresType interface. -func (b ByteaType) ValueType() reflect.Type { - return reflect.TypeOf([]byte{}) -} - -// Zero implements the DoltgresType interface. -func (b ByteaType) Zero() any { - return []byte{} -} - -// SerializeType implements the DoltgresType interface. -func (b ByteaType) SerializeType() ([]byte, error) { - return SerializationID_Bytea.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b ByteaType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Bytea, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b ByteaType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - str := converted.([]byte) - writer := utils.NewWriter(uint64(len(str) + 4)) - writer.ByteSlice(str) - return writer.Data(), nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b ByteaType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - reader := utils.NewReader(val) - return reader.ByteSlice(), nil +var Bytea = &DoltgresType{ + OID: uint32(oid.T_bytea), + Name: "bytea", + Schema: "pg_catalog", + TypLength: int16(-1), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_UserDefinedTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__bytea), + InputFunc: toFuncID("byteain", oid.T_cstring), + OutputFunc: toFuncID("byteaout", oid.T_bytea), + ReceiveFunc: toFuncID("bytearecv", oid.T_internal), + SendFunc: toFuncID("byteasend", oid.T_bytea), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Extended, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("byteacmp", oid.T_bytea, oid.T_bytea), } diff --git a/server/types/bytea_array.go b/server/types/bytea_array.go index ceb9c9dd7c..4c5e9975cd 100644 --- a/server/types/bytea_array.go +++ b/server/types/bytea_array.go @@ -14,9 +14,5 @@ package types -import ( - "github.com/lib/pq/oid" -) - // ByteaArray is the array variant of Bytea. -var ByteaArray = createArrayType(Bytea, SerializationID_ByteaArray, oid.T__bytea) +var ByteaArray = CreateArrayTypeFromBaseType(Bytea) diff --git a/server/types/char.go b/server/types/char.go index 8cf4fb3b40..880a644e65 100644 --- a/server/types/char.go +++ b/server/types/char.go @@ -15,282 +15,53 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "math" - "reflect" - "strings" - - "github.com/dolthub/doltgresql/utils" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // BpChar is a char that has an unbounded length. -var BpChar = CharType{Length: stringUnbounded} - -// CharType is the type implementation of the PostgreSQL bpchar. -type CharType struct { - // Length represents the maximum number of characters that the type may hold. - // When this is set to unbounded, then it becomes recognized as bpchar. - Length uint32 -} - -var _ DoltgresType = CharType{} - -// Alignment implements the DoltgresType interface. -func (b CharType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b CharType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Char -} - -// BaseName implements the DoltgresType interface. -func (b CharType) BaseName() string { - return "bpchar" -} - -// Category implements the DoltgresType interface. -func (b CharType) Category() TypeCategory { - return TypeCategory_StringTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b CharType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b CharType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := strings.TrimRight(ac.(string), " ") - bb := strings.TrimRight(bc.(string), " ") - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b CharType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case string: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b CharType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b CharType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b CharType) GetSerializationID() SerializationID { - return SerializationID_Char -} - -// IoInput implements the DoltgresType interface. -func (b CharType) IoInput(ctx *sql.Context, input string) (any, error) { - if b.IsUnbounded() { - return input, nil - } else { - input, runeLength := truncateString(input, b.Length) - if runeLength > b.Length { - return input, fmt.Errorf("value too long for type %s", b.String()) - } else if runeLength < b.Length { - return input + strings.Repeat(" ", int(b.Length-runeLength)), nil - } else { - return input, nil - } - } -} - -// IoOutput implements the DoltgresType interface. -func (b CharType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - if b.IsUnbounded() { - return converted.(string), nil - } else { - str, runeCount := truncateString(converted.(string), b.Length) - if runeCount < b.Length { - return str + strings.Repeat(" ", int(b.Length-runeCount)), nil - } - return str, nil - } -} - -// IsPreferredType implements the DoltgresType interface. -func (b CharType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b CharType) IsUnbounded() bool { - return b.Length == stringUnbounded -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b CharType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - if b.Length != stringUnbounded && b.Length <= stringInline { - return types.ExtendedTypeSerializedWidth_64K - } else { - return types.ExtendedTypeSerializedWidth_Unbounded - } -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b CharType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - if b.Length == stringUnbounded { - return math.MaxUint32 - } else { - return b.Length * 4 - } -} - -// OID implements the DoltgresType interface. -func (b CharType) OID() uint32 { - return uint32(oid.T_bpchar) -} - -// Promote implements the DoltgresType interface. -func (b CharType) Promote() sql.Type { - return BpChar -} - -// SerializedCompare implements the DoltgresType interface. -func (b CharType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - return serializedStringCompare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b CharType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b CharType) String() string { - return fmt.Sprintf("character(%d)", b.Length) -} - -// ToArrayType implements the DoltgresType interface. -func (b CharType) ToArrayType() DoltgresArrayType { - return createArrayType(b, SerializationID_CharArray, oid.T__bpchar) -} - -// Type implements the DoltgresType interface. -func (b CharType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b CharType) ValueType() reflect.Type { - return reflect.TypeOf("") -} - -// Zero implements the DoltgresType interface. -func (b CharType) Zero() any { - return "" -} - -// SerializeType implements the DoltgresType interface. -func (b CharType) SerializeType() ([]byte, error) { - t := make([]byte, serializationIDHeaderSize+4) - copy(t, SerializationID_Char.ToByteSlice(0)) - binary.LittleEndian.PutUint32(t[serializationIDHeaderSize:], b.Length) - return t, nil -} - -// deserializeType implements the DoltgresType interface. -func (b CharType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return CharType{ - Length: binary.LittleEndian.Uint32(metadata), - }, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b CharType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) +var BpChar = &DoltgresType{ + OID: uint32(oid.T_bpchar), + Name: "bpchar", + Schema: "pg_catalog", + TypLength: int16(-1), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_StringTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__bpchar), + InputFunc: toFuncID("bpcharin", oid.T_cstring, oid.T_oid, oid.T_int4), + OutputFunc: toFuncID("bpcharout", oid.T_bpchar), + ReceiveFunc: toFuncID("bpcharrecv", oid.T_internal, oid.T_oid, oid.T_int4), + SendFunc: toFuncID("bpcharsend", oid.T_bpchar), + ModInFunc: toFuncID("bpchartypmodin", oid.T__cstring), + ModOutFunc: toFuncID("bpchartypmodout", oid.T_int4), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Extended, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 100, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("bpcharcmp", oid.T_bpchar, oid.T_bpchar), +} + +// NewCharType returns BpChar type with typmod set. +func NewCharType(length int32) (*DoltgresType, error) { + typmod, err := GetTypModFromCharLength("char", length) if err != nil { return nil, err } - str := converted.(string) - writer := utils.NewWriter(uint64(len(str) + 4)) - writer.String(str) - return writer.Data(), nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b CharType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - reader := utils.NewReader(val) - return reader.String(), nil + newType := *BpChar.WithAttTypMod(typmod) + return &newType, nil } diff --git a/server/types/char_array.go b/server/types/char_array.go index 2f58598ad6..c101f796d6 100644 --- a/server/types/char_array.go +++ b/server/types/char_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // BpCharArray is the array variant of BpChar. -var BpCharArray = createArrayType(BpChar, SerializationID_CharArray, oid.T__bpchar) +var BpCharArray = CreateArrayTypeFromBaseType(BpChar) diff --git a/server/types/cstring.go b/server/types/cstring.go new file mode 100644 index 0000000000..d3aaa98263 --- /dev/null +++ b/server/types/cstring.go @@ -0,0 +1,57 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package types + +import ( + "github.com/lib/pq/oid" +) + +// Cstring is the cstring type. +var Cstring = &DoltgresType{ + OID: uint32(oid.T_cstring), + Name: "cstring", + Schema: "pg_catalog", + TypLength: int16(-2), + PassedByVal: false, + TypType: TypeType_Pseudo, + TypCategory: TypeCategory_PseudoTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__cstring), + InputFunc: toFuncID("cstring_in", oid.T_cstring), + OutputFunc: toFuncID("cstring_out", oid.T_cstring), + ReceiveFunc: toFuncID("cstring_recv", oid.T_internal), + SendFunc: toFuncID("cstring_send", oid.T_cstring), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Char, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), +} diff --git a/server/types/cstring_array.go b/server/types/cstring_array.go new file mode 100644 index 0000000000..a40b12f1d2 --- /dev/null +++ b/server/types/cstring_array.go @@ -0,0 +1,18 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package types + +// CstringArray is the cstring type. +var CstringArray = CreateArrayTypeFromBaseType(Cstring) diff --git a/server/types/date.go b/server/types/date.go index 2e11b62f76..2a0c8a3210 100644 --- a/server/types/date.go +++ b/server/types/date.go @@ -15,248 +15,43 @@ package types import ( - "bytes" - "fmt" - "reflect" - "time" - - "github.com/dolthub/doltgresql/postgres/parser/pgdate" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Date is the day, month, and year. -var Date = DateType{} - -// DateType is the extended type implementation of the PostgreSQL date. -type DateType struct{} - -var _ DoltgresType = DateType{} - -// Alignment implements the DoltgresType interface. -func (b DateType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b DateType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Date -} - -// BaseName implements the DoltgresType interface. -func (b DateType) BaseName() string { - return "date" -} - -// Category implements the DoltgresType interface. -func (b DateType) Category() TypeCategory { - return TypeCategory_DateTimeTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b DateType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b DateType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(time.Time) - bb := bc.(time.Time) - return ab.Compare(bb), nil -} - -// Convert implements the DoltgresType interface. -func (b DateType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case time.Time: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b DateType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b DateType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b DateType) GetSerializationID() SerializationID { - return SerializationID_Date -} - -// IoInput implements the DoltgresType interface. -func (b DateType) IoInput(ctx *sql.Context, input string) (any, error) { - if date, _, err := pgdate.ParseDate(time.Now(), pgdate.ParseModeYMD, input); err == nil { - return date.ToTime() - } else if date, _, err = pgdate.ParseDate(time.Now(), pgdate.ParseModeDMY, input); err == nil { - return date.ToTime() - } else if date, _, err = pgdate.ParseDate(time.Now(), pgdate.ParseModeMDY, input); err == nil { - return date.ToTime() - } else { - return nil, err - } -} - -// IoOutput implements the DoltgresType interface. -func (b DateType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return converted.(time.Time).Format("2006-01-02"), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b DateType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b DateType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b DateType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b DateType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 4 -} - -// OID implements the DoltgresType interface. -func (b DateType) OID() uint32 { - return uint32(oid.T_date) -} - -// Promote implements the DoltgresType interface. -func (b DateType) Promote() sql.Type { - return Date -} - -// SerializedCompare implements the DoltgresType interface. -func (b DateType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - // The marshalled time format is byte-comparable - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b DateType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b DateType) String() string { - return "date" -} - -// ToArrayType implements the DoltgresType interface. -func (b DateType) ToArrayType() DoltgresArrayType { - return DateArray -} - -// Type implements the DoltgresType interface. -func (b DateType) Type() query.Type { - return sqltypes.Date -} - -// ValueType implements the DoltgresType interface. -func (b DateType) ValueType() reflect.Type { - return reflect.TypeOf(time.Time{}) -} - -// Zero implements the DoltgresType interface. -func (b DateType) Zero() any { - return time.Time{} -} - -// SerializeType implements the DoltgresType interface. -func (b DateType) SerializeType() ([]byte, error) { - return SerializationID_Date.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b DateType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Date, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b DateType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - return converted.(time.Time).MarshalBinary() -} - -// DeserializeValue implements the DoltgresType interface. -func (b DateType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - t := time.Time{} - if err := t.UnmarshalBinary(val); err != nil { - return nil, err - } - return t, nil +var Date = &DoltgresType{ + OID: uint32(oid.T_date), + Name: "date", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_DateTimeTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__date), + InputFunc: toFuncID("date_in", oid.T_cstring), + OutputFunc: toFuncID("date_out", oid.T_date), + ReceiveFunc: toFuncID("date_recv", oid.T_internal), + SendFunc: toFuncID("date_send", oid.T_date), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("date_cmp", oid.T_date, oid.T_date), } diff --git a/server/types/date_array.go b/server/types/date_array.go index f601885502..5f7ceb1436 100644 --- a/server/types/date_array.go +++ b/server/types/date_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - -// DateArray is the array variant of Date. -var DateArray = createArrayType(Date, SerializationID_DateArray, oid.T__date) +// DateArray is the day, month, and year array. +var DateArray = CreateArrayTypeFromBaseType(Date) diff --git a/server/types/doltgrestypebaseid_string.go b/server/types/doltgrestypebaseid_string.go deleted file mode 100755 index 6f89088ee4..0000000000 --- a/server/types/doltgrestypebaseid_string.go +++ /dev/null @@ -1,153 +0,0 @@ -// Code generated by "stringer -type=DoltgresTypeBaseID"; DO NOT EDIT. - -package types - -import "strconv" - -func _() { - // An "invalid array index" compiler error signifies that the constant values have changed. - // Re-run the stringer command to generate them again. - var x [1]struct{} - _ = x[DoltgresTypeBaseID_Any-8192] - _ = x[DoltgresTypeBaseID_AnyElement-8193] - _ = x[DoltgresTypeBaseID_AnyArray-8194] - _ = x[DoltgresTypeBaseID_AnyNonArray-8195] - _ = x[DoltgresTypeBaseID_AnyEnum-8196] - _ = x[DoltgresTypeBaseID_AnyRange-8197] - _ = x[DoltgresTypeBaseID_AnyMultirange-8198] - _ = x[DoltgresTypeBaseID_AnyCompatible-8199] - _ = x[DoltgresTypeBaseID_AnyCompatibleArray-8200] - _ = x[DoltgresTypeBaseID_AnyCompatibleNonArray-8201] - _ = x[DoltgresTypeBaseID_AnyCompatibleRange-8202] - _ = x[DoltgresTypeBaseID_AnyCompatibleMultirange-8203] - _ = x[DoltgresTypeBaseID_CString-8204] - _ = x[DoltgresTypeBaseID_Internal-8205] - _ = x[DoltgresTypeBaseID_Language_Handler-8206] - _ = x[DoltgresTypeBaseID_FDW_Handler-8207] - _ = x[DoltgresTypeBaseID_Table_AM_Handler-8208] - _ = x[DoltgresTypeBaseID_Index_AM_Handler-8209] - _ = x[DoltgresTypeBaseID_TSM_Handler-8210] - _ = x[DoltgresTypeBaseID_Record-8211] - _ = x[DoltgresTypeBaseID_Trigger-8212] - _ = x[DoltgresTypeBaseID_Event_Trigger-8213] - _ = x[DoltgresTypeBaseID_PG_DDL_Command-8214] - _ = x[DoltgresTypeBaseID_Void-8215] - _ = x[DoltgresTypeBaseID_Unknown-8216] - _ = x[DoltgresTypeBaseID_Int16Serial-8217] - _ = x[DoltgresTypeBaseID_Int32Serial-8218] - _ = x[DoltgresTypeBaseID_Int64Serial-8219] - _ = x[DoltgresTypeBaseID_Regclass-8220] - _ = x[DoltgresTypeBaseID_Regcollation-8221] - _ = x[DoltgresTypeBaseID_Regconfig-8222] - _ = x[DoltgresTypeBaseID_Regdictionary-8223] - _ = x[DoltgresTypeBaseID_Regnamespace-8224] - _ = x[DoltgresTypeBaseID_Regoper-8225] - _ = x[DoltgresTypeBaseID_Regoperator-8226] - _ = x[DoltgresTypeBaseID_Regproc-8227] - _ = x[DoltgresTypeBaseID_Regprocedure-8228] - _ = x[DoltgresTypeBaseID_Regrole-8229] - _ = x[DoltgresTypeBaseID_Regtype-8230] - _ = x[DoltgresTypeBaseID_Bool-3] - _ = x[DoltgresTypeBaseID_Bytea-7] - _ = x[DoltgresTypeBaseID_Char-9] - _ = x[DoltgresTypeBaseID_Date-15] - _ = x[DoltgresTypeBaseID_Float32-21] - _ = x[DoltgresTypeBaseID_Float64-23] - _ = x[DoltgresTypeBaseID_Int16-27] - _ = x[DoltgresTypeBaseID_Int32-29] - _ = x[DoltgresTypeBaseID_Int64-33] - _ = x[DoltgresTypeBaseID_InternalChar-96] - _ = x[DoltgresTypeBaseID_Interval-37] - _ = x[DoltgresTypeBaseID_Json-39] - _ = x[DoltgresTypeBaseID_JsonB-41] - _ = x[DoltgresTypeBaseID_Name-90] - _ = x[DoltgresTypeBaseID_Null-53] - _ = x[DoltgresTypeBaseID_Numeric-54] - _ = x[DoltgresTypeBaseID_Oid-92] - _ = x[DoltgresTypeBaseID_Text-64] - _ = x[DoltgresTypeBaseID_Time-66] - _ = x[DoltgresTypeBaseID_Timestamp-70] - _ = x[DoltgresTypeBaseID_TimestampTZ-74] - _ = x[DoltgresTypeBaseID_TimeTZ-68] - _ = x[DoltgresTypeBaseID_Uuid-82] - _ = x[DoltgresTypeBaseID_VarChar-86] - _ = x[DoltgresTypeBaseID_Xid-94] - _ = x[DoltgresTypeBaseId_Domain-98] -} - -const _DoltgresTypeBaseID_name = "DoltgresTypeBaseID_BoolDoltgresTypeBaseID_ByteaDoltgresTypeBaseID_CharDoltgresTypeBaseID_DateDoltgresTypeBaseID_Float32DoltgresTypeBaseID_Float64DoltgresTypeBaseID_Int16DoltgresTypeBaseID_Int32DoltgresTypeBaseID_Int64DoltgresTypeBaseID_IntervalDoltgresTypeBaseID_JsonDoltgresTypeBaseID_JsonBDoltgresTypeBaseID_NullDoltgresTypeBaseID_NumericDoltgresTypeBaseID_TextDoltgresTypeBaseID_TimeDoltgresTypeBaseID_TimeTZDoltgresTypeBaseID_TimestampDoltgresTypeBaseID_TimestampTZDoltgresTypeBaseID_UuidDoltgresTypeBaseID_VarCharDoltgresTypeBaseID_NameDoltgresTypeBaseID_OidDoltgresTypeBaseID_XidDoltgresTypeBaseID_InternalCharDoltgresTypeBaseId_DomainDoltgresTypeBaseID_AnyDoltgresTypeBaseID_AnyElementDoltgresTypeBaseID_AnyArrayDoltgresTypeBaseID_AnyNonArrayDoltgresTypeBaseID_AnyEnumDoltgresTypeBaseID_AnyRangeDoltgresTypeBaseID_AnyMultirangeDoltgresTypeBaseID_AnyCompatibleDoltgresTypeBaseID_AnyCompatibleArrayDoltgresTypeBaseID_AnyCompatibleNonArrayDoltgresTypeBaseID_AnyCompatibleRangeDoltgresTypeBaseID_AnyCompatibleMultirangeDoltgresTypeBaseID_CStringDoltgresTypeBaseID_InternalDoltgresTypeBaseID_Language_HandlerDoltgresTypeBaseID_FDW_HandlerDoltgresTypeBaseID_Table_AM_HandlerDoltgresTypeBaseID_Index_AM_HandlerDoltgresTypeBaseID_TSM_HandlerDoltgresTypeBaseID_RecordDoltgresTypeBaseID_TriggerDoltgresTypeBaseID_Event_TriggerDoltgresTypeBaseID_PG_DDL_CommandDoltgresTypeBaseID_VoidDoltgresTypeBaseID_UnknownDoltgresTypeBaseID_Int16SerialDoltgresTypeBaseID_Int32SerialDoltgresTypeBaseID_Int64SerialDoltgresTypeBaseID_RegclassDoltgresTypeBaseID_RegcollationDoltgresTypeBaseID_RegconfigDoltgresTypeBaseID_RegdictionaryDoltgresTypeBaseID_RegnamespaceDoltgresTypeBaseID_RegoperDoltgresTypeBaseID_RegoperatorDoltgresTypeBaseID_RegprocDoltgresTypeBaseID_RegprocedureDoltgresTypeBaseID_RegroleDoltgresTypeBaseID_Regtype" - -var _DoltgresTypeBaseID_map = map[DoltgresTypeBaseID]string{ - 3: _DoltgresTypeBaseID_name[0:23], - 7: _DoltgresTypeBaseID_name[23:47], - 9: _DoltgresTypeBaseID_name[47:70], - 15: _DoltgresTypeBaseID_name[70:93], - 21: _DoltgresTypeBaseID_name[93:119], - 23: _DoltgresTypeBaseID_name[119:145], - 27: _DoltgresTypeBaseID_name[145:169], - 29: _DoltgresTypeBaseID_name[169:193], - 33: _DoltgresTypeBaseID_name[193:217], - 37: _DoltgresTypeBaseID_name[217:244], - 39: _DoltgresTypeBaseID_name[244:267], - 41: _DoltgresTypeBaseID_name[267:291], - 53: _DoltgresTypeBaseID_name[291:314], - 54: _DoltgresTypeBaseID_name[314:340], - 64: _DoltgresTypeBaseID_name[340:363], - 66: _DoltgresTypeBaseID_name[363:386], - 68: _DoltgresTypeBaseID_name[386:411], - 70: _DoltgresTypeBaseID_name[411:439], - 74: _DoltgresTypeBaseID_name[439:469], - 82: _DoltgresTypeBaseID_name[469:492], - 86: _DoltgresTypeBaseID_name[492:518], - 90: _DoltgresTypeBaseID_name[518:541], - 92: _DoltgresTypeBaseID_name[541:563], - 94: _DoltgresTypeBaseID_name[563:585], - 96: _DoltgresTypeBaseID_name[585:616], - 98: _DoltgresTypeBaseID_name[616:641], - 8192: _DoltgresTypeBaseID_name[641:663], - 8193: _DoltgresTypeBaseID_name[663:692], - 8194: _DoltgresTypeBaseID_name[692:719], - 8195: _DoltgresTypeBaseID_name[719:749], - 8196: _DoltgresTypeBaseID_name[749:775], - 8197: _DoltgresTypeBaseID_name[775:802], - 8198: _DoltgresTypeBaseID_name[802:834], - 8199: _DoltgresTypeBaseID_name[834:866], - 8200: _DoltgresTypeBaseID_name[866:903], - 8201: _DoltgresTypeBaseID_name[903:943], - 8202: _DoltgresTypeBaseID_name[943:980], - 8203: _DoltgresTypeBaseID_name[980:1022], - 8204: _DoltgresTypeBaseID_name[1022:1048], - 8205: _DoltgresTypeBaseID_name[1048:1075], - 8206: _DoltgresTypeBaseID_name[1075:1110], - 8207: _DoltgresTypeBaseID_name[1110:1140], - 8208: _DoltgresTypeBaseID_name[1140:1175], - 8209: _DoltgresTypeBaseID_name[1175:1210], - 8210: _DoltgresTypeBaseID_name[1210:1240], - 8211: _DoltgresTypeBaseID_name[1240:1265], - 8212: _DoltgresTypeBaseID_name[1265:1291], - 8213: _DoltgresTypeBaseID_name[1291:1323], - 8214: _DoltgresTypeBaseID_name[1323:1356], - 8215: _DoltgresTypeBaseID_name[1356:1379], - 8216: _DoltgresTypeBaseID_name[1379:1405], - 8217: _DoltgresTypeBaseID_name[1405:1435], - 8218: _DoltgresTypeBaseID_name[1435:1465], - 8219: _DoltgresTypeBaseID_name[1465:1495], - 8220: _DoltgresTypeBaseID_name[1495:1522], - 8221: _DoltgresTypeBaseID_name[1522:1553], - 8222: _DoltgresTypeBaseID_name[1553:1581], - 8223: _DoltgresTypeBaseID_name[1581:1613], - 8224: _DoltgresTypeBaseID_name[1613:1644], - 8225: _DoltgresTypeBaseID_name[1644:1670], - 8226: _DoltgresTypeBaseID_name[1670:1700], - 8227: _DoltgresTypeBaseID_name[1700:1726], - 8228: _DoltgresTypeBaseID_name[1726:1757], - 8229: _DoltgresTypeBaseID_name[1757:1783], - 8230: _DoltgresTypeBaseID_name[1783:1809], -} - -func (i DoltgresTypeBaseID) String() string { - if str, ok := _DoltgresTypeBaseID_map[i]; ok { - return str - } - return "DoltgresTypeBaseID(" + strconv.FormatInt(int64(i), 10) + ")" -} diff --git a/server/types/domain.go b/server/types/domain.go index 7e069ec919..55736fdcde 100644 --- a/server/types/domain.go +++ b/server/types/domain.go @@ -15,274 +15,56 @@ package types import ( - "fmt" - "reflect" - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" - - "github.com/dolthub/doltgresql/utils" + "github.com/lib/pq/oid" ) -type DomainType struct { - Schema string - Name string - AsType DoltgresType - DefaultExpr string - NotNull bool - Checks []*sql.CheckDefinition -} - -// NewDomainType creates new instance of domain Type. +// NewDomainType creates new instance of domain DoltgresType. func NewDomainType( ctx *sql.Context, schema string, name string, - asType DoltgresType, + asType *DoltgresType, defaultExpr string, notNull bool, checks []*sql.CheckDefinition, owner string, // TODO -) (*Type, error) { - passedByVal := false - l := asType.MaxTextResponseByteLength(ctx) - if l&1 == 0 && l < 9 { - passedByVal = true - } - return &Type{ - Oid: 0, // TODO: generate unique OID +) *DoltgresType { + return &DoltgresType{ + OID: asType.OID, // TODO: generate unique OID, using underlying type OID for now Name: name, Schema: schema, Owner: owner, - Length: int16(l), - PassedByVal: passedByVal, + TypLength: asType.TypLength, + PassedByVal: asType.PassedByVal, TypType: TypeType_Domain, - TypCategory: asType.Category(), - IsPreferred: asType.IsPreferredType(), + TypCategory: asType.TypCategory, + IsPreferred: asType.IsPreferred, IsDefined: true, Delimiter: ",", RelID: 0, - SubscriptFunc: "", + SubscriptFunc: toFuncID("-"), Elem: 0, Array: 0, // TODO: refers to array type of this type - InputFunc: "domain_in", - OutputFunc: "", // TODO: base type's out function - ReceiveFunc: "domain_recv", - SendFunc: "", // TODO: base type's send function - ModInFunc: "-", - ModOutFunc: "-", - AnalyzeFunc: "-", - Align: asType.Alignment(), - Storage: TypeStorage_Plain, // TODO: base type's storage + InputFunc: toFuncID("domain_in", oid.T_cstring, oid.T_oid, oid.T_int4), + OutputFunc: asType.OutputFunc, + ReceiveFunc: toFuncID("domain_recv", oid.T_internal, oid.T_oid, oid.T_int4), + SendFunc: asType.SendFunc, + ModInFunc: asType.ModInFunc, + ModOutFunc: asType.ModOutFunc, + AnalyzeFunc: toFuncID("-"), + Align: asType.Align, + Storage: asType.Storage, NotNull: notNull, - BaseTypeOID: asType.OID(), + BaseTypeOID: asType.OID, TypMod: -1, NDims: 0, - Collation: 0, + TypCollation: 0, DefaulBin: "", Default: defaultExpr, - Acl: "", + Acl: nil, Checks: checks, - }, nil -} - -var _ DoltgresType = DomainType{} - -// Alignment implements the DoltgresType interface. -func (d DomainType) Alignment() TypeAlignment { - return d.AsType.Alignment() -} - -// BaseID implements the DoltgresType interface. -func (d DomainType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseId_Domain -} - -// BaseName implements the DoltgresType interface. -func (d DomainType) BaseName() string { - return d.Name -} - -// Category implements the DoltgresType interface. -func (d DomainType) Category() TypeCategory { - return d.AsType.Category() -} - -// CollationCoercibility implements the DoltgresType interface. -func (d DomainType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return d.AsType.CollationCoercibility(ctx) -} - -// Compare implements the DoltgresType interface. -func (d DomainType) Compare(i interface{}, i2 interface{}) (int, error) { - return d.AsType.Compare(i, i2) -} - -// Convert implements the DoltgresType interface. -func (d DomainType) Convert(i interface{}) (interface{}, sql.ConvertInRange, error) { - return d.AsType.Convert(i) -} - -// Equals implements the DoltgresType interface. -func (d DomainType) Equals(otherType sql.Type) bool { - return d.AsType.Equals(otherType) -} - -// FormatValue implements the types.ExtendedType interface. -func (d DomainType) FormatValue(val any) (string, error) { - return d.AsType.FormatValue(val) -} - -// GetSerializationID implements the DoltgresType interface. -func (d DomainType) GetSerializationID() SerializationID { - return SerializationId_Domain -} - -// IoInput implements the DoltgresType interface. -func (d DomainType) IoInput(ctx *sql.Context, input string) (any, error) { - return d.AsType.IoInput(ctx, input) -} - -// IoOutput implements the DoltgresType interface. -func (d DomainType) IoOutput(ctx *sql.Context, output any) (string, error) { - return d.AsType.IoOutput(ctx, output) -} - -// IsPreferredType implements the DoltgresType interface. -func (d DomainType) IsPreferredType() bool { - return d.AsType.IsPreferredType() -} - -// IsUnbounded implements the DoltgresType interface. -func (d DomainType) IsUnbounded() bool { - return d.AsType.IsUnbounded() -} - -// MaxSerializedWidth implements the types.ExtendedType interface. -func (d DomainType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return d.AsType.MaxSerializedWidth() -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (d DomainType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return d.AsType.MaxTextResponseByteLength(ctx) -} - -// OID implements the DoltgresType interface. -func (d DomainType) OID() uint32 { - //TODO: generate unique oid - return d.AsType.OID() -} - -// Promote implements the DoltgresType interface. -func (d DomainType) Promote() sql.Type { - return d.AsType.Promote() -} - -// SerializedCompare implements the DoltgresType interface. -func (d DomainType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - return d.AsType.SerializedCompare(v1, v2) -} - -// SQL implements the DoltgresType interface. -func (d DomainType) SQL(ctx *sql.Context, dest []byte, v interface{}) (sqltypes.Value, error) { - return d.AsType.SQL(ctx, dest, v) -} - -// String implements the DoltgresType interface. -func (d DomainType) String() string { - return d.Name -} - -// ToArrayType implements the DoltgresType interface. -func (d DomainType) ToArrayType() DoltgresArrayType { - return d.AsType.ToArrayType() -} - -// Type implements the DoltgresType interface. -func (d DomainType) Type() query.Type { - return d.AsType.Type() -} - -// ValueType implements the DoltgresType interface. -func (d DomainType) ValueType() reflect.Type { - return d.AsType.ValueType() -} - -// Zero implements the DoltgresType interface. -func (d DomainType) Zero() interface{} { - return d.AsType.Zero() -} - -// SerializeType implements the DoltgresType interface. -func (d DomainType) SerializeType() ([]byte, error) { - b := SerializationId_Domain.ToByteSlice(0) - writer := utils.NewWriter(256) - writer.String(d.Schema) - writer.String(d.Name) - writer.String(d.DefaultExpr) - writer.Bool(d.NotNull) - writer.VariableUint(uint64(len(d.Checks))) - for _, check := range d.Checks { - writer.String(check.Name) - writer.String(check.CheckExpression) - } - asTyp, err := d.AsType.SerializeType() - if err != nil { - return nil, err - } - b = append(b, writer.Data()...) - return append(b, asTyp...), nil -} - -// deserializeType implements the DoltgresType interface. -func (d DomainType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - reader := utils.NewReader(metadata) - d.Schema = reader.String() - d.Name = reader.String() - d.DefaultExpr = reader.String() - d.NotNull = reader.Bool() - numOfChecks := reader.VariableUint() - for k := uint64(0); k < numOfChecks; k++ { - checkName := reader.String() - checkExpr := reader.String() - d.Checks = append(d.Checks, &sql.CheckDefinition{ - Name: checkName, - CheckExpression: checkExpr, - Enforced: true, - }) - } - t, err := DeserializeType(metadata[reader.BytesRead():]) - if err != nil { - return nil, err - } - d.AsType = t.(DoltgresType) - return d, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, d.String()) - } -} - -// SerializeValue implements the types.ExtendedType interface. -func (d DomainType) SerializeValue(val any) ([]byte, error) { - return d.AsType.SerializeValue(val) -} - -// DeserializeValue implements the types.ExtendedType interface. -func (d DomainType) DeserializeValue(val []byte) (any, error) { - return d.AsType.DeserializeValue(val) -} - -// UnderlyingBaseType returns underlying type of the domain type that is a base type. -func (d DomainType) UnderlyingBaseType() DoltgresType { - switch t := d.AsType.(type) { - case DomainType: - return t.UnderlyingBaseType() - default: - return t + attTypMod: -1, + CompareFunc: asType.CompareFunc, } } diff --git a/server/types/float32.go b/server/types/float32.go index a0be2bd834..3c267287c0 100644 --- a/server/types/float32.go +++ b/server/types/float32.go @@ -15,266 +15,44 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "math" - "reflect" - "strconv" - "strings" - "github.com/lib/pq/oid" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" ) // Float32 is an float32. -var Float32 = Float32Type{} - -// Float32Type is the extended type implementation of the PostgreSQL real. -type Float32Type struct{} - -var _ DoltgresType = Float32Type{} - -// Alignment implements the DoltgresType interface. -func (b Float32Type) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b Float32Type) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Float32 -} - -// BaseName implements the DoltgresType interface. -func (b Float32Type) BaseName() string { - return "float4" -} - -// Category implements the DoltgresType interface. -func (b Float32Type) Category() TypeCategory { - return TypeCategory_NumericTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b Float32Type) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b Float32Type) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(float32) - bb := bc.(float32) - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b Float32Type) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case float32: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b Float32Type) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b Float32Type) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - converted, _, err := b.Convert(val) - if err != nil { - return "", err - } - return strconv.FormatFloat(float64(converted.(float32)), 'g', -1, 32), nil -} - -// GetSerializationID implements the DoltgresType interface. -func (b Float32Type) GetSerializationID() SerializationID { - return SerializationID_Float32 -} - -// IoInput implements the DoltgresType interface. -func (b Float32Type) IoInput(ctx *sql.Context, input string) (any, error) { - val, err := strconv.ParseFloat(strings.TrimSpace(input), 32) - if err != nil { - return nil, fmt.Errorf("invalid input syntax for type %s: %q", b.String(), input) - } - return float32(val), nil -} - -// IoOutput implements the DoltgresType interface. -func (b Float32Type) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return strconv.FormatFloat(float64(converted.(float32)), 'f', -1, 32), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b Float32Type) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b Float32Type) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b Float32Type) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b Float32Type) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 4 -} - -// OID implements the DoltgresType interface. -func (b Float32Type) OID() uint32 { - return uint32(oid.T_float4) -} - -// Promote implements the DoltgresType interface. -func (b Float32Type) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b Float32Type) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b Float32Type) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.FormatValue(v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b Float32Type) String() string { - return "real" -} - -// ToArrayType implements the DoltgresType interface. -func (b Float32Type) ToArrayType() DoltgresArrayType { - return Float32Array -} - -// Type implements the DoltgresType interface. -func (b Float32Type) Type() query.Type { - return sqltypes.Float32 -} - -// ValueType implements the DoltgresType interface. -func (b Float32Type) ValueType() reflect.Type { - return reflect.TypeOf(float32(0)) -} - -// Zero implements the DoltgresType interface. -func (b Float32Type) Zero() any { - return float32(0) -} - -// SerializeType implements the DoltgresType interface. -func (b Float32Type) SerializeType() ([]byte, error) { - return SerializationID_Float32.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b Float32Type) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Float32, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b Float32Type) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - retVal := make([]byte, 4) - // Make the serialized form trivially comparable using bytes.Compare: https://stackoverflow.com/a/54557561 - unsignedBits := math.Float32bits(converted.(float32)) - if converted.(float32) >= 0 { - unsignedBits ^= 1 << 31 - } else { - unsignedBits = ^unsignedBits - } - binary.BigEndian.PutUint32(retVal, unsignedBits) - return retVal, nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b Float32Type) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - unsignedBits := binary.BigEndian.Uint32(val) - if unsignedBits&(1<<31) != 0 { - unsignedBits ^= 1 << 31 - } else { - unsignedBits = ^unsignedBits - } - return math.Float32frombits(unsignedBits), nil +var Float32 = &DoltgresType{ + OID: uint32(oid.T_float4), + Name: "float4", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__float4), + InputFunc: toFuncID("float4in", oid.T_cstring), + OutputFunc: toFuncID("float4out", oid.T_float4), + ReceiveFunc: toFuncID("float4recv", oid.T_internal), + SendFunc: toFuncID("float4send", oid.T_float4), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btfloat4cmp", oid.T_float4, oid.T_float4), + InternalName: "real", } diff --git a/server/types/float32_array.go b/server/types/float32_array.go index 612252514c..fc1afeba4c 100644 --- a/server/types/float32_array.go +++ b/server/types/float32_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // Float32Array is the array variant of Float32. -var Float32Array = createArrayType(Float32, SerializationID_Float32Array, oid.T__float4) +var Float32Array = CreateArrayTypeFromBaseType(Float32) diff --git a/server/types/float64.go b/server/types/float64.go index cf30aa4322..35ebd78f48 100644 --- a/server/types/float64.go +++ b/server/types/float64.go @@ -15,265 +15,44 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "math" - "reflect" - "strconv" - "strings" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Float64 is an float64. -var Float64 = Float64Type{} - -// Float64Type is the extended type implementation of the PostgreSQL double precision. -type Float64Type struct{} - -var _ DoltgresType = Float64Type{} - -// Alignment implements the DoltgresType interface. -func (b Float64Type) Alignment() TypeAlignment { - return TypeAlignment_Double -} - -// BaseID implements the DoltgresType interface. -func (b Float64Type) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Float64 -} - -// BaseName implements the DoltgresType interface. -func (b Float64Type) BaseName() string { - return "float8" -} - -// Category implements the DoltgresType interface. -func (b Float64Type) Category() TypeCategory { - return TypeCategory_NumericTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b Float64Type) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b Float64Type) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(float64) - bb := bc.(float64) - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b Float64Type) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case float64: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b Float64Type) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b Float64Type) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - converted, _, err := b.Convert(val) - if err != nil { - return "", err - } - return strconv.FormatFloat(converted.(float64), 'g', -1, 64), nil -} - -// GetSerializationID implements the DoltgresType interface. -func (b Float64Type) GetSerializationID() SerializationID { - return SerializationID_Float64 -} - -// IoInput implements the DoltgresType interface. -func (b Float64Type) IoInput(ctx *sql.Context, input string) (any, error) { - val, err := strconv.ParseFloat(strings.TrimSpace(input), 64) - if err != nil { - return nil, fmt.Errorf("invalid input syntax for type %s: %q", b.String(), input) - } - return val, nil -} - -// IoOutput implements the DoltgresType interface. -func (b Float64Type) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return strconv.FormatFloat(converted.(float64), 'f', -1, 64), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b Float64Type) IsPreferredType() bool { - return true -} - -// IsUnbounded implements the DoltgresType interface. -func (b Float64Type) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b Float64Type) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b Float64Type) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 8 -} - -// OID implements the DoltgresType interface. -func (b Float64Type) OID() uint32 { - return uint32(oid.T_float8) -} - -// Promote implements the DoltgresType interface. -func (b Float64Type) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b Float64Type) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b Float64Type) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.FormatValue(v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b Float64Type) String() string { - return "double precision" -} - -// ToArrayType implements the DoltgresType interface. -func (b Float64Type) ToArrayType() DoltgresArrayType { - return Float64Array -} - -// Type implements the DoltgresType interface. -func (b Float64Type) Type() query.Type { - return sqltypes.Float64 -} - -// ValueType implements the DoltgresType interface. -func (b Float64Type) ValueType() reflect.Type { - return reflect.TypeOf(float64(0)) -} - -// Zero implements the DoltgresType interface. -func (b Float64Type) Zero() any { - return float64(0) -} - -// SerializeType implements the DoltgresType interface. -func (b Float64Type) SerializeType() ([]byte, error) { - return SerializationID_Float64.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b Float64Type) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Float64, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b Float64Type) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - retVal := make([]byte, 8) - // Make the serialized form trivially comparable using bytes.Compare: https://stackoverflow.com/a/54557561 - unsignedBits := math.Float64bits(converted.(float64)) - if converted.(float64) >= 0 { - unsignedBits ^= 1 << 63 - } else { - unsignedBits = ^unsignedBits - } - binary.BigEndian.PutUint64(retVal, unsignedBits) - return retVal, nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b Float64Type) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - unsignedBits := binary.BigEndian.Uint64(val) - if unsignedBits&(1<<63) != 0 { - unsignedBits ^= 1 << 63 - } else { - unsignedBits = ^unsignedBits - } - return math.Float64frombits(unsignedBits), nil +var Float64 = &DoltgresType{ + OID: uint32(oid.T_float8), + Name: "float8", + Schema: "pg_catalog", + TypLength: int16(8), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: true, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__float8), + InputFunc: toFuncID("float8in", oid.T_cstring), + OutputFunc: toFuncID("float8out", oid.T_float8), + ReceiveFunc: toFuncID("float8recv", oid.T_internal), + SendFunc: toFuncID("float8send", oid.T_float8), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Double, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btfloat8cmp", oid.T_float8, oid.T_float8), + InternalName: "double precision", } diff --git a/server/types/float64_array.go b/server/types/float64_array.go index f487206550..fd971ba486 100644 --- a/server/types/float64_array.go +++ b/server/types/float64_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // Float64Array is the array variant of Float64. -var Float64Array = createArrayType(Float64, SerializationID_Float64Array, oid.T__float8) +var Float64Array = CreateArrayTypeFromBaseType(Float64) diff --git a/server/types/function_registry.go b/server/types/function_registry.go new file mode 100644 index 0000000000..61e109b35f --- /dev/null +++ b/server/types/function_registry.go @@ -0,0 +1,180 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package types + +import ( + "fmt" + "strconv" + "strings" + "sync" + + "github.com/dolthub/go-mysql-server/sql" + "github.com/lib/pq/oid" +) + +// functionNameSplitter is used to delineate the parameter OIDs in a function string. +const functionNameSplitter = ";" + +// QuickFunction is an interface redefinition of the one defined in the `server/functions/framework` package to avoid cycles. +type QuickFunction interface { + CallVariadic(ctx *sql.Context, args ...any) (interface{}, error) + ResolvedTypes() []*DoltgresType + WithResolvedTypes(newTypes []*DoltgresType) any +} + +// LoadFunctionFromCatalog returns the function matching the given name and parameter types. This is intended solely for +// functions that are used for types, as the returned functions are not valid using the Eval function. +var LoadFunctionFromCatalog func(funcName string, parameterTypes []*DoltgresType) any + +// functionRegistry is a local registry that holds a mapping from ID to QuickFunction. This is done as types are now +// passed by struct, meaning that we need to cache the loading of functions somewhere. In addition, we don't yet support +// deleting built-in functions, so we can make a global cache. This makes a hard assumption that all functions being +// referenced actually exist, which should be true until built-in function deletion is implemented. +// +// In a way, one can view this as associated an OID to a function. With a proper OID system, this would not need to +// exist. It should be removed once OIDs are figured out. +type functionRegistry struct { + mutex *sync.Mutex + counter uint32 + mapping map[string]uint32 + revMapping map[uint32]string + functions [256]QuickFunction // Arbitrary number, big enough for now to fit every function in it +} + +// globalFunctionRegistry is the global functionRegistry. Only one needs to exist since we do not yet allow deleting +// built-in functions. +var globalFunctionRegistry = functionRegistry{ + mutex: &sync.Mutex{}, + counter: 1, + mapping: map[string]uint32{"-": 0}, + revMapping: map[uint32]string{0: "-"}, +} + +// StringToID returns an ID for the given function string. +func (r *functionRegistry) StringToID(functionString string) uint32 { + r.mutex.Lock() + defer r.mutex.Unlock() + if id, ok := r.mapping[functionString]; ok { + return id + } + if r.counter >= uint32(len(r.functions)) { + panic("max function count reached in static array") + } + r.mapping[functionString] = r.counter + r.revMapping[r.counter] = functionString + r.counter++ + return r.counter - 1 +} + +// GetFunction returns the associated function for the given ID. This will always return a valid function. +func (r *functionRegistry) GetFunction(id uint32) QuickFunction { + f := r.functions[id] + if f != nil { + return f + } + if id == 0 { + return nil + } + r.mutex.Lock() + defer r.mutex.Unlock() + f = r.loadFunction(id) + if f == nil { + // If we hit this panic, then we're missing a test that uses this function (and we should add that test) + panic(fmt.Errorf("cannot find function: `%s`", r.revMapping[id])) + } + return f +} + +// GetFullString returns the function string associated with the given ID. +func (r *functionRegistry) GetFullString(id uint32) string { + r.mutex.Lock() + defer r.mutex.Unlock() + return r.revMapping[id] +} + +// GetString returns the extracted function name from the function string associated with the given ID. +func (r *functionRegistry) GetString(id uint32) string { + r.mutex.Lock() + defer r.mutex.Unlock() + return r.nameWithoutParams(r.revMapping[id]) +} + +// loadFunction loads the given function +func (r *functionRegistry) loadFunction(id uint32) QuickFunction { + // We make this check a second time (first in GetFunction) since the function may have been added while another + // function acquired the lock. + f := r.functions[id] + if f != nil { + return f + } + if LoadFunctionFromCatalog == nil { + return nil + } + functionString := r.revMapping[id] + name, params, ok := r.nameWithParams(functionString) + if !ok { + return nil + } + potentialFunction := LoadFunctionFromCatalog(name, params) + if potentialFunction == nil { + return nil + } + f = potentialFunction.(QuickFunction) + r.functions[id] = f + return f +} + +// nameWithoutParams returns the name only from the given function string. +func (*functionRegistry) nameWithoutParams(funcString string) string { + return strings.Split(funcString, functionNameSplitter)[0] +} + +// nameWithParams returns the name and parameter types from the given function string. Return false if there were issues +// with the OID parameters. +func (*functionRegistry) nameWithParams(funcString string) (string, []*DoltgresType, bool) { + parts := strings.Split(funcString, functionNameSplitter) + if len(parts) == 1 { + return parts[0], nil, true + } + dTypes := make([]*DoltgresType, len(parts)-1) + for i := 1; i < len(parts); i++ { + oidVal, err := strconv.Atoi(parts[i]) + if err != nil { + return parts[0], nil, false + } + typ, ok := OidToBuiltInDoltgresType[uint32(oidVal)] + if !ok { + return parts[0], nil, false + } + dTypes[i-1] = typ + } + return parts[0], dTypes, true +} + +// toFuncID creates a valid function string for the given name and parameters, then registers the name with the +// global functionRegistry. The ID from the registry is returned. +func toFuncID(functionName string, params ...oid.Oid) uint32 { + if functionName == "-" { + return 0 + } + if len(params) > 0 { + paramStrs := make([]string, len(params)) + for i := range params { + paramStrs[i] = strconv.Itoa(int(params[i])) + } + functionName = fmt.Sprintf("%s%s%s", functionName, functionNameSplitter, strings.Join(paramStrs, functionNameSplitter)) + } + return globalFunctionRegistry.StringToID(functionName) +} diff --git a/server/types/globals.go b/server/types/globals.go index 12b0d36ddf..3bc6e1adeb 100644 --- a/server/types/globals.go +++ b/server/types/globals.go @@ -14,86 +14,10 @@ package types -import "fmt" +import ( + "sort" -// DoltgresTypeBaseID is an ID that is common between all variations of a DoltgresType. For example, VARCHAR(3) and -// VARCHAR(6) are different types, however they will return the same DoltgresTypeBaseID. This ID is not suitable for -// serialization, as it may change over time. Many types use their SerializationID as their base ID, so for types that -// are not serializable (such as the "any" types), it is recommended that they start way after the largest -// SerializationID to prevent base ID conflicts. -type DoltgresTypeBaseID uint32 - -//go:generate go run golang.org/x/tools/cmd/stringer -type=DoltgresTypeBaseID - -const ( - DoltgresTypeBaseID_Any DoltgresTypeBaseID = iota + 8192 - DoltgresTypeBaseID_AnyElement - DoltgresTypeBaseID_AnyArray - DoltgresTypeBaseID_AnyNonArray - DoltgresTypeBaseID_AnyEnum - DoltgresTypeBaseID_AnyRange - DoltgresTypeBaseID_AnyMultirange - DoltgresTypeBaseID_AnyCompatible - DoltgresTypeBaseID_AnyCompatibleArray - DoltgresTypeBaseID_AnyCompatibleNonArray - DoltgresTypeBaseID_AnyCompatibleRange - DoltgresTypeBaseID_AnyCompatibleMultirange - DoltgresTypeBaseID_CString - DoltgresTypeBaseID_Internal - DoltgresTypeBaseID_Language_Handler - DoltgresTypeBaseID_FDW_Handler - DoltgresTypeBaseID_Table_AM_Handler - DoltgresTypeBaseID_Index_AM_Handler - DoltgresTypeBaseID_TSM_Handler - DoltgresTypeBaseID_Record - DoltgresTypeBaseID_Trigger - DoltgresTypeBaseID_Event_Trigger - DoltgresTypeBaseID_PG_DDL_Command - DoltgresTypeBaseID_Void - DoltgresTypeBaseID_Unknown - DoltgresTypeBaseID_Int16Serial - DoltgresTypeBaseID_Int32Serial - DoltgresTypeBaseID_Int64Serial - DoltgresTypeBaseID_Regclass - DoltgresTypeBaseID_Regcollation - DoltgresTypeBaseID_Regconfig - DoltgresTypeBaseID_Regdictionary - DoltgresTypeBaseID_Regnamespace - DoltgresTypeBaseID_Regoper - DoltgresTypeBaseID_Regoperator - DoltgresTypeBaseID_Regproc - DoltgresTypeBaseID_Regprocedure - DoltgresTypeBaseID_Regrole - DoltgresTypeBaseID_Regtype -) - -const ( - DoltgresTypeBaseID_Bool = DoltgresTypeBaseID(SerializationID_Bool) - DoltgresTypeBaseID_Bytea = DoltgresTypeBaseID(SerializationID_Bytea) - DoltgresTypeBaseID_Char = DoltgresTypeBaseID(SerializationID_Char) - DoltgresTypeBaseID_Date = DoltgresTypeBaseID(SerializationID_Date) - DoltgresTypeBaseID_Float32 = DoltgresTypeBaseID(SerializationID_Float32) - DoltgresTypeBaseID_Float64 = DoltgresTypeBaseID(SerializationID_Float64) - DoltgresTypeBaseID_Int16 = DoltgresTypeBaseID(SerializationID_Int16) - DoltgresTypeBaseID_Int32 = DoltgresTypeBaseID(SerializationID_Int32) - DoltgresTypeBaseID_Int64 = DoltgresTypeBaseID(SerializationID_Int64) - DoltgresTypeBaseID_InternalChar = DoltgresTypeBaseID(SerializationID_InternalChar) - DoltgresTypeBaseID_Interval = DoltgresTypeBaseID(SerializationID_Interval) - DoltgresTypeBaseID_Json = DoltgresTypeBaseID(SerializationID_Json) - DoltgresTypeBaseID_JsonB = DoltgresTypeBaseID(SerializationID_JsonB) - DoltgresTypeBaseID_Name = DoltgresTypeBaseID(SerializationID_Name) - DoltgresTypeBaseID_Null = DoltgresTypeBaseID(SerializationID_Null) - DoltgresTypeBaseID_Numeric = DoltgresTypeBaseID(SerializationID_Numeric) - DoltgresTypeBaseID_Oid = DoltgresTypeBaseID(SerializationID_Oid) - DoltgresTypeBaseID_Text = DoltgresTypeBaseID(SerializationID_Text) - DoltgresTypeBaseID_Time = DoltgresTypeBaseID(SerializationID_Time) - DoltgresTypeBaseID_Timestamp = DoltgresTypeBaseID(SerializationID_Timestamp) - DoltgresTypeBaseID_TimestampTZ = DoltgresTypeBaseID(SerializationID_TimestampTZ) - DoltgresTypeBaseID_TimeTZ = DoltgresTypeBaseID(SerializationID_TimeTZ) - DoltgresTypeBaseID_Uuid = DoltgresTypeBaseID(SerializationID_Uuid) - DoltgresTypeBaseID_VarChar = DoltgresTypeBaseID(SerializationID_VarChar) - DoltgresTypeBaseID_Xid = DoltgresTypeBaseID(SerializationID_Xid) - DoltgresTypeBaseId_Domain = DoltgresTypeBaseID(SerializationId_Domain) + "github.com/lib/pq/oid" ) // TypeAlignment represents the alignment required when storing a value of this type. @@ -153,95 +77,262 @@ const ( TypeType_MultiRange TypeType = "m" ) -// baseIDArrayTypes contains a map of all base IDs that represent array variants. -var baseIDArrayTypes = map[DoltgresTypeBaseID]DoltgresArrayType{} - -// baseIDCategories contains a map from all base IDs to their respective categories -// TODO: add all of the types to each category -var baseIDCategories = map[DoltgresTypeBaseID]TypeCategory{} - -// preferredTypeInCategory contains a map from each type category to that category's preferred type. -// TODO: add all of the preferred types -var preferredTypeInCategory = map[TypeCategory][]DoltgresTypeBaseID{} - -// oidToType holds a reference from a given OID to its type. -var oidToType = map[uint32]DoltgresType{} - -// Init reads the list of all types and creates mappings that will be used by various functions. -func Init() { - for baseID, t := range typesFromBaseID { - if dat, ok := t.(DoltgresArrayType); ok { - baseIDArrayTypes[t.BaseID()] = dat - } - if t.IsPreferredType() { - preferredTypeInCategory[t.Category()] = append(preferredTypeInCategory[t.Category()], t.BaseID()) - } - // Add the types to the OID map - if baseID.HasUniqueOID() { - if existingType, ok := oidToType[t.OID()]; ok { - panic(fmt.Errorf("OID (%d) type conflict: `%s` and `%s`", t.OID(), existingType.String(), t.String())) - } - oidToType[t.OID()] = t - baseIDCategories[t.BaseID()] = t.Category() - } - } -} - -// IsBaseIDArrayType returns whether the base ID is an array type. If it is, it also returns the type. -func (id DoltgresTypeBaseID) IsBaseIDArrayType() (DoltgresArrayType, bool) { - dat, ok := baseIDArrayTypes[id] - return dat, ok +// typesFromOID contains a map from a OID to its originating type. +var typesFromOID = map[uint32]*DoltgresType{ + AnyArray.OID: AnyArray, + AnyElement.OID: AnyElement, + AnyNonArray.OID: AnyNonArray, + Bool.OID: Bool, + BoolArray.OID: BoolArray, + BpChar.OID: BpChar, + BpCharArray.OID: BpCharArray, + Bytea.OID: Bytea, + ByteaArray.OID: ByteaArray, + Cstring.OID: Cstring, + CstringArray.OID: CstringArray, + Date.OID: Date, + DateArray.OID: DateArray, + Float32.OID: Float32, + Float32Array.OID: Float32Array, + Float64.OID: Float64, + Float64Array.OID: Float64Array, + Int16.OID: Int16, + Int16Array.OID: Int16Array, + Int32.OID: Int32, + Int32Array.OID: Int32Array, + Int64.OID: Int64, + Int64Array.OID: Int64Array, + Internal.OID: Internal, + InternalChar.OID: InternalChar, + InternalCharArray.OID: InternalCharArray, + Interval.OID: Interval, + IntervalArray.OID: IntervalArray, + Json.OID: Json, + JsonArray.OID: JsonArray, + JsonB.OID: JsonB, + JsonBArray.OID: JsonBArray, + Name.OID: Name, + NameArray.OID: NameArray, + Numeric.OID: Numeric, + NumericArray.OID: NumericArray, + Oid.OID: Oid, + OidArray.OID: OidArray, + Regclass.OID: Regclass, + RegclassArray.OID: RegclassArray, + Regproc.OID: Regproc, + RegprocArray.OID: RegprocArray, + Regtype.OID: Regtype, + RegtypeArray.OID: RegtypeArray, + Text.OID: Text, + TextArray.OID: TextArray, + Time.OID: Time, + TimeArray.OID: TimeArray, + Timestamp.OID: Timestamp, + TimestampArray.OID: TimestampArray, + TimestampTZ.OID: TimestampTZ, + TimestampTZArray.OID: TimestampTZArray, + TimeTZ.OID: TimeTZ, + TimeTZArray.OID: TimeTZArray, + Unknown.OID: Unknown, + Uuid.OID: Uuid, + UuidArray.OID: UuidArray, + VarChar.OID: VarChar, + VarCharArray.OID: VarCharArray, + Xid.OID: Xid, + XidArray.OID: XidArray, } -// GetTypeCategory returns the TypeCategory that this base ID belongs to. Returns Unknown if the ID does not belong to a -// category. -func (id DoltgresTypeBaseID) GetTypeCategory() TypeCategory { - if tc, ok := baseIDCategories[id]; ok { - return tc - } - return TypeCategory_UnknownTypes -} - -// GetRepresentativeType returns the representative type of the base ID. This is usually the unbounded version or -// equivalent. -func (id DoltgresTypeBaseID) GetRepresentativeType() DoltgresType { - if t, ok := typesFromBaseID[id]; ok { - return t - } - return Unknown -} - -// HasUniqueOID returns whether the type belonging to the base ID has a unique OID. This will be true for most types. -// Examples of types that do not have unique OIDs are the serial types, since they're not actual types. -func (id DoltgresTypeBaseID) HasUniqueOID() bool { - switch id { - case DoltgresTypeBaseID_Null, - DoltgresTypeBaseID_Int16Serial, - DoltgresTypeBaseID_Int32Serial, - DoltgresTypeBaseID_Int64Serial: - return false - default: - return true +// GetTypeByOID returns the DoltgresType matching the given OID. +// If the OID does not match a type, then nil is returned. +func GetTypeByOID(oid uint32) *DoltgresType { + t, ok := typesFromOID[oid] + if !ok { + // TODO: return UNKNOWN? + return nil } + return t } -// IsPreferredType returns whether the type passed is a preferred type for this TypeCategory. -func (cat TypeCategory) IsPreferredType(p DoltgresTypeBaseID) bool { - if pts, ok := preferredTypeInCategory[cat]; ok { - for _, pt := range pts { - if pt == p { - return true - } - } +// GetAllTypes returns a slice containing all registered types. +// The slice is sorted by each type's OID. +func GetAllTypes() []*DoltgresType { + pgTypes := make([]*DoltgresType, 0, len(typesFromOID)) + for _, typ := range typesFromOID { + pgTypes = append(pgTypes, typ) } - return false + sort.Slice(pgTypes, func(i, j int) bool { + return pgTypes[i].OID < pgTypes[j].OID + }) + return pgTypes } -// GetTypeByOID returns the DoltgresType matching the given OID. If the OID does not match a type, then nil is returned. -func GetTypeByOID(oid uint32) DoltgresType { - t, ok := oidToType[oid] - if !ok { - return nil - } - return t +// OidToBuiltInDoltgresType is a map of oid to built-in Doltgres type. +var OidToBuiltInDoltgresType = map[uint32]*DoltgresType{ + uint32(oid.T_bool): Bool, + uint32(oid.T_bytea): Bytea, + uint32(oid.T_char): InternalChar, + uint32(oid.T_name): Name, + uint32(oid.T_int8): Int64, + uint32(oid.T_int2): Int16, + uint32(oid.T_int2vector): Unknown, + uint32(oid.T_int4): Int32, + uint32(oid.T_regproc): Regproc, + uint32(oid.T_text): Text, + uint32(oid.T_oid): Oid, + uint32(oid.T_tid): Unknown, + uint32(oid.T_xid): Xid, + uint32(oid.T_cid): Unknown, + uint32(oid.T_oidvector): Unknown, + uint32(oid.T_pg_ddl_command): Unknown, + uint32(oid.T_pg_type): Unknown, + uint32(oid.T_pg_attribute): Unknown, + uint32(oid.T_pg_proc): Unknown, + uint32(oid.T_pg_class): Unknown, + uint32(oid.T_json): Json, + uint32(oid.T_xml): Unknown, + uint32(oid.T__xml): Unknown, + uint32(oid.T_pg_node_tree): Unknown, + uint32(oid.T__json): JsonArray, + uint32(oid.T_smgr): Unknown, + uint32(oid.T_index_am_handler): Unknown, + uint32(oid.T_point): Unknown, + uint32(oid.T_lseg): Unknown, + uint32(oid.T_path): Unknown, + uint32(oid.T_box): Unknown, + uint32(oid.T_polygon): Unknown, + uint32(oid.T_line): Unknown, + uint32(oid.T__line): Unknown, + uint32(oid.T_cidr): Unknown, + uint32(oid.T__cidr): Unknown, + uint32(oid.T_float4): Float32, + uint32(oid.T_float8): Float64, + uint32(oid.T_abstime): Unknown, + uint32(oid.T_reltime): Unknown, + uint32(oid.T_tinterval): Unknown, + uint32(oid.T_unknown): Unknown, + uint32(oid.T_circle): Unknown, + uint32(oid.T__circle): Unknown, + uint32(oid.T_money): Unknown, + uint32(oid.T__money): Unknown, + uint32(oid.T_macaddr): Unknown, + uint32(oid.T_inet): Unknown, + uint32(oid.T__bool): BoolArray, + uint32(oid.T__bytea): ByteaArray, + uint32(oid.T__char): InternalCharArray, + uint32(oid.T__name): NameArray, + uint32(oid.T__int2): Int16Array, + uint32(oid.T__int2vector): Unknown, + uint32(oid.T__int4): Int32Array, + uint32(oid.T__regproc): RegprocArray, + uint32(oid.T__text): TextArray, + uint32(oid.T__tid): Unknown, + uint32(oid.T__xid): XidArray, + uint32(oid.T__cid): Unknown, + uint32(oid.T__oidvector): Unknown, + uint32(oid.T__bpchar): BpCharArray, + uint32(oid.T__varchar): VarCharArray, + uint32(oid.T__int8): Int64Array, + uint32(oid.T__point): Unknown, + uint32(oid.T__lseg): Unknown, + uint32(oid.T__path): Unknown, + uint32(oid.T__box): Unknown, + uint32(oid.T__float4): Float32Array, + uint32(oid.T__float8): Float64Array, + uint32(oid.T__abstime): Unknown, + uint32(oid.T__reltime): Unknown, + uint32(oid.T__tinterval): Unknown, + uint32(oid.T__polygon): Unknown, + uint32(oid.T__oid): OidArray, + uint32(oid.T_aclitem): Unknown, + uint32(oid.T__aclitem): Unknown, + uint32(oid.T__macaddr): Unknown, + uint32(oid.T__inet): Unknown, + uint32(oid.T_bpchar): BpChar, + uint32(oid.T_varchar): VarChar, + uint32(oid.T_date): Date, + uint32(oid.T_time): Time, + uint32(oid.T_timestamp): Timestamp, + uint32(oid.T__timestamp): TimestampArray, + uint32(oid.T__date): DateArray, + uint32(oid.T__time): TimeArray, + uint32(oid.T_timestamptz): TimestampTZ, + uint32(oid.T__timestamptz): TimestampTZArray, + uint32(oid.T_interval): Interval, + uint32(oid.T__interval): IntervalArray, + uint32(oid.T__numeric): NumericArray, + uint32(oid.T_pg_database): Unknown, + uint32(oid.T__cstring): CstringArray, + uint32(oid.T_timetz): TimeTZ, + uint32(oid.T__timetz): TimeTZArray, + uint32(oid.T_bit): Unknown, + uint32(oid.T__bit): Unknown, + uint32(oid.T_varbit): Unknown, + uint32(oid.T__varbit): Unknown, + uint32(oid.T_numeric): Numeric, + uint32(oid.T_refcursor): Unknown, + uint32(oid.T__refcursor): Unknown, + uint32(oid.T_regprocedure): Unknown, + uint32(oid.T_regoper): Unknown, + uint32(oid.T_regoperator): Unknown, + uint32(oid.T_regclass): Regclass, + uint32(oid.T_regtype): Regtype, + uint32(oid.T__regprocedure): Unknown, + uint32(oid.T__regoper): Unknown, + uint32(oid.T__regoperator): Unknown, + uint32(oid.T__regclass): RegclassArray, + uint32(oid.T__regtype): RegtypeArray, + uint32(oid.T_record): Unknown, + uint32(oid.T_cstring): Cstring, + uint32(oid.T_any): Unknown, + uint32(oid.T_anyarray): AnyArray, + uint32(oid.T_void): Unknown, + uint32(oid.T_trigger): Unknown, + uint32(oid.T_language_handler): Unknown, + uint32(oid.T_internal): Internal, + uint32(oid.T_opaque): Unknown, + uint32(oid.T_anyelement): AnyElement, + uint32(oid.T__record): Unknown, + uint32(oid.T_anynonarray): AnyNonArray, + uint32(oid.T_pg_authid): Unknown, + uint32(oid.T_pg_auth_members): Unknown, + uint32(oid.T__txid_snapshot): Unknown, + uint32(oid.T_uuid): Uuid, + uint32(oid.T__uuid): UuidArray, + uint32(oid.T_txid_snapshot): Unknown, + uint32(oid.T_fdw_handler): Unknown, + uint32(oid.T_pg_lsn): Unknown, + uint32(oid.T__pg_lsn): Unknown, + uint32(oid.T_tsm_handler): Unknown, + uint32(oid.T_anyenum): Unknown, + uint32(oid.T_tsvector): Unknown, + uint32(oid.T_tsquery): Unknown, + uint32(oid.T_gtsvector): Unknown, + uint32(oid.T__tsvector): Unknown, + uint32(oid.T__gtsvector): Unknown, + uint32(oid.T__tsquery): Unknown, + uint32(oid.T_regconfig): Unknown, + uint32(oid.T__regconfig): Unknown, + uint32(oid.T_regdictionary): Unknown, + uint32(oid.T__regdictionary): Unknown, + uint32(oid.T_jsonb): JsonB, + uint32(oid.T__jsonb): JsonBArray, + uint32(oid.T_anyrange): Unknown, + uint32(oid.T_event_trigger): Unknown, + uint32(oid.T_int4range): Unknown, + uint32(oid.T__int4range): Unknown, + uint32(oid.T_numrange): Unknown, + uint32(oid.T__numrange): Unknown, + uint32(oid.T_tsrange): Unknown, + uint32(oid.T__tsrange): Unknown, + uint32(oid.T_tstzrange): Unknown, + uint32(oid.T__tstzrange): Unknown, + uint32(oid.T_daterange): Unknown, + uint32(oid.T__daterange): Unknown, + uint32(oid.T_int8range): Unknown, + uint32(oid.T__int8range): Unknown, + uint32(oid.T_pg_shseclabel): Unknown, + uint32(oid.T_regnamespace): Unknown, + uint32(oid.T__regnamespace): Unknown, + uint32(oid.T_regrole): Unknown, + uint32(oid.T__regrole): Unknown, } diff --git a/server/types/int16.go b/server/types/int16.go index d6abca57c0..4e22dd9c00 100644 --- a/server/types/int16.go +++ b/server/types/int16.go @@ -15,250 +15,44 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "reflect" - "strconv" - "strings" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Int16 is an int16. -var Int16 = Int16Type{} - -// Int16Type is the extended type implementation of the PostgreSQL smallint. -type Int16Type struct{} - -var _ DoltgresType = Int16Type{} - -// Alignment implements the DoltgresType interface. -func (b Int16Type) Alignment() TypeAlignment { - return TypeAlignment_Short -} - -// BaseID implements the DoltgresType interface. -func (b Int16Type) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Int16 -} - -// BaseName implements the DoltgresType interface. -func (b Int16Type) BaseName() string { - return "int2" -} - -// Category implements the DoltgresType interface. -func (b Int16Type) Category() TypeCategory { - return TypeCategory_NumericTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b Int16Type) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b Int16Type) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(int16) - bb := bc.(int16) - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b Int16Type) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case int16: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b Int16Type) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b Int16Type) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b Int16Type) GetSerializationID() SerializationID { - return SerializationID_Int16 -} - -// IoInput implements the DoltgresType interface. -func (b Int16Type) IoInput(ctx *sql.Context, input string) (any, error) { - val, err := strconv.ParseInt(strings.TrimSpace(input), 10, 16) - if err != nil { - return nil, fmt.Errorf("invalid input syntax for type %s: %q", b.String(), input) - } - if val > 32767 || val < -32768 { - return nil, fmt.Errorf("value %q is out of range for type %s", input, b.String()) - } - return int16(val), nil -} - -// IoOutput implements the DoltgresType interface. -func (b Int16Type) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return strconv.FormatInt(int64(converted.(int16)), 10), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b Int16Type) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b Int16Type) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b Int16Type) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b Int16Type) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 2 -} - -// OID implements the DoltgresType interface. -func (b Int16Type) OID() uint32 { - return uint32(oid.T_int2) -} - -// Promote implements the DoltgresType interface. -func (b Int16Type) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b Int16Type) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b Int16Type) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b Int16Type) String() string { - return "smallint" -} - -// ToArrayType implements the DoltgresType interface. -func (b Int16Type) ToArrayType() DoltgresArrayType { - return Int16Array -} - -// Type implements the DoltgresType interface. -func (b Int16Type) Type() query.Type { - return sqltypes.Int16 -} - -// ValueType implements the DoltgresType interface. -func (b Int16Type) ValueType() reflect.Type { - return reflect.TypeOf(int16(0)) -} - -// Zero implements the DoltgresType interface. -func (b Int16Type) Zero() any { - return int16(0) -} - -// SerializeType implements the DoltgresType interface. -func (b Int16Type) SerializeType() ([]byte, error) { - return SerializationID_Int16.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b Int16Type) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Int16, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b Int16Type) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - retVal := make([]byte, 2) - binary.BigEndian.PutUint16(retVal, uint16(converted.(int16))+(1<<15)) - return retVal, nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b Int16Type) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - return int16(binary.BigEndian.Uint16(val) - (1 << 15)), nil +var Int16 = &DoltgresType{ + OID: uint32(oid.T_int2), + Name: "int2", + Schema: "pg_catalog", + TypLength: int16(2), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__int2), + InputFunc: toFuncID("int2in", oid.T_cstring), + OutputFunc: toFuncID("int2out", oid.T_int2), + ReceiveFunc: toFuncID("int2recv", oid.T_internal), + SendFunc: toFuncID("int2send", oid.T_int2), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Short, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btint2cmp", oid.T_int2, oid.T_int2), + InternalName: "smallint", } diff --git a/server/types/int16_array.go b/server/types/int16_array.go index c48577f579..9be1d8ac99 100644 --- a/server/types/int16_array.go +++ b/server/types/int16_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // Int16Array is the array variant of Int16. -var Int16Array = createArrayType(Int16, SerializationID_Int16Array, oid.T__int2) +var Int16Array = CreateArrayTypeFromBaseType(Int16) diff --git a/server/types/int16_serial.go b/server/types/int16_serial.go index 90e08f3801..44e1f90cb6 100644 --- a/server/types/int16_serial.go +++ b/server/types/int16_serial.go @@ -14,167 +14,43 @@ package types -import ( - "fmt" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" - "github.com/lib/pq/oid" -) +import "github.com/lib/pq/oid" // Int16Serial is an int16 serial type. -var Int16Serial = Int16TypeSerial{} - -// Int16TypeSerial is the extended type implementation of the PostgreSQL smallserial. -type Int16TypeSerial struct{} - -var _ DoltgresType = Int16TypeSerial{} - -// Alignment implements the DoltgresType interface. -func (b Int16TypeSerial) Alignment() TypeAlignment { - return TypeAlignment_Short -} - -// BaseID implements the DoltgresType interface. -func (b Int16TypeSerial) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Int16Serial -} - -// BaseName implements the DoltgresType interface. -func (b Int16TypeSerial) BaseName() string { - return "smallserial" -} - -// Category implements the DoltgresType interface. -func (b Int16TypeSerial) Category() TypeCategory { - return TypeCategory_UnknownTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b Int16TypeSerial) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b Int16TypeSerial) Compare(v1 any, v2 any) (int, error) { - return 0, fmt.Errorf("SERIAL types are not comparable") -} - -// Convert implements the DoltgresType interface. -func (b Int16TypeSerial) Convert(val any) (any, sql.ConvertInRange, error) { - return nil, sql.OutOfRange, fmt.Errorf("SERIAL types are not convertable") -} - -// Equals implements the DoltgresType interface. -func (b Int16TypeSerial) Equals(otherType sql.Type) bool { - _, ok := otherType.(Int16TypeSerial) - return ok -} - -// FormatValue implements the DoltgresType interface. -func (b Int16TypeSerial) FormatValue(val any) (string, error) { - return "", fmt.Errorf("SERIAL types are not formattable") -} - -// GetSerializationID implements the DoltgresType interface. -func (b Int16TypeSerial) GetSerializationID() SerializationID { - return SerializationID_Invalid -} - -// IoInput implements the DoltgresType interface. -func (b Int16TypeSerial) IoInput(ctx *sql.Context, input string) (any, error) { - return "", fmt.Errorf("SERIAL types cannot receive I/O input") -} - -// IoOutput implements the DoltgresType interface. -func (b Int16TypeSerial) IoOutput(ctx *sql.Context, output any) (string, error) { - return "", fmt.Errorf("SERIAL types cannot produce I/O output") -} - -// IsPreferredType implements the DoltgresType interface. -func (b Int16TypeSerial) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b Int16TypeSerial) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b Int16TypeSerial) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b Int16TypeSerial) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 2 -} - -// OID implements the DoltgresType interface. -func (b Int16TypeSerial) OID() uint32 { - return uint32(oid.T_int2) -} - -// Promote implements the DoltgresType interface. -func (b Int16TypeSerial) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b Int16TypeSerial) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - return 0, fmt.Errorf("SERIAL types are not comparable") -} - -// SQL implements the DoltgresType interface. -func (b Int16TypeSerial) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - return sqltypes.Value{}, fmt.Errorf("SERIAL types may not be passed over the wire") -} - -// String implements the DoltgresType interface. -func (b Int16TypeSerial) String() string { - return "smallserial" -} - -// ToArrayType implements the DoltgresType interface. -func (b Int16TypeSerial) ToArrayType() DoltgresArrayType { - return Unknown -} - -// Type implements the DoltgresType interface. -func (b Int16TypeSerial) Type() query.Type { - return sqltypes.Int16 -} - -// ValueType implements the DoltgresType interface. -func (b Int16TypeSerial) ValueType() reflect.Type { - return reflect.TypeOf(int16(0)) -} - -// Zero implements the DoltgresType interface. -func (b Int16TypeSerial) Zero() any { - return int16(0) -} - -// SerializeType implements the DoltgresType interface. -func (b Int16TypeSerial) SerializeType() ([]byte, error) { - return nil, fmt.Errorf("SERIAL types are not serializable") -} - -// deserializeType implements the DoltgresType interface. -func (b Int16TypeSerial) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - return nil, fmt.Errorf("SERIAL types are not deserializable") -} - -// SerializeValue implements the DoltgresType interface. -func (b Int16TypeSerial) SerializeValue(val any) ([]byte, error) { - return nil, fmt.Errorf("SERIAL types are not serializable") -} - -// DeserializeValue implements the DoltgresType interface. -func (b Int16TypeSerial) DeserializeValue(val []byte) (any, error) { - return nil, fmt.Errorf("SERIAL types are not deserializable") +var Int16Serial = &DoltgresType{ + OID: 0, // doesn't have unique OID + Name: "smallserial", + Schema: "pg_catalog", + TypLength: int16(2), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__int2), + InputFunc: toFuncID("int2in", oid.T_cstring), + OutputFunc: toFuncID("int2out", oid.T_int2), + ReceiveFunc: toFuncID("int2recv"), + SendFunc: toFuncID("int2send"), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Short, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btint2cmp", oid.T_int2, oid.T_int2), + IsSerial: true, } diff --git a/server/types/int32.go b/server/types/int32.go index 78ccaa734f..4b0dd1682c 100644 --- a/server/types/int32.go +++ b/server/types/int32.go @@ -15,250 +15,44 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "reflect" - "strconv" - "strings" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Int32 is an int32. -var Int32 = Int32Type{} - -// Int32Type is the extended type implementation of the PostgreSQL integer. -type Int32Type struct{} - -var _ DoltgresType = Int32Type{} - -// Alignment implements the DoltgresType interface. -func (b Int32Type) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b Int32Type) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Int32 -} - -// BaseName implements the DoltgresType interface. -func (b Int32Type) BaseName() string { - return "int4" -} - -// Category implements the DoltgresType interface. -func (b Int32Type) Category() TypeCategory { - return TypeCategory_NumericTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b Int32Type) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b Int32Type) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(int32) - bb := bc.(int32) - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b Int32Type) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case int32: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b Int32Type) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b Int32Type) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b Int32Type) GetSerializationID() SerializationID { - return SerializationID_Int32 -} - -// IoInput implements the DoltgresType interface. -func (b Int32Type) IoInput(ctx *sql.Context, input string) (any, error) { - val, err := strconv.ParseInt(strings.TrimSpace(input), 10, 32) - if err != nil { - return nil, fmt.Errorf("invalid input syntax for type %s: %q", b.String(), input) - } - if val > 2147483647 || val < -2147483648 { - return nil, fmt.Errorf("value %q is out of range for type %s", input, b.String()) - } - return int32(val), nil -} - -// IoOutput implements the DoltgresType interface. -func (b Int32Type) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return strconv.FormatInt(int64(converted.(int32)), 10), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b Int32Type) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b Int32Type) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b Int32Type) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b Int32Type) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 4 -} - -// OID implements the DoltgresType interface. -func (b Int32Type) OID() uint32 { - return uint32(oid.T_int4) -} - -// Promote implements the DoltgresType interface. -func (b Int32Type) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b Int32Type) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b Int32Type) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b Int32Type) String() string { - return "integer" -} - -// ToArrayType implements the DoltgresType interface. -func (b Int32Type) ToArrayType() DoltgresArrayType { - return Int32Array -} - -// Type implements the DoltgresType interface. -func (b Int32Type) Type() query.Type { - return sqltypes.Int32 -} - -// ValueType implements the DoltgresType interface. -func (b Int32Type) ValueType() reflect.Type { - return reflect.TypeOf(int32(0)) -} - -// Zero implements the DoltgresType interface. -func (b Int32Type) Zero() any { - return int32(0) -} - -// SerializeType implements the DoltgresType interface. -func (b Int32Type) SerializeType() ([]byte, error) { - return SerializationID_Int32.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b Int32Type) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Int32, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b Int32Type) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - retVal := make([]byte, 4) - binary.BigEndian.PutUint32(retVal, uint32(converted.(int32))+(1<<31)) - return retVal, nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b Int32Type) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - return int32(binary.BigEndian.Uint32(val) - (1 << 31)), nil +var Int32 = &DoltgresType{ + OID: uint32(oid.T_int4), + Name: "int4", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__int4), + InputFunc: toFuncID("int4in", oid.T_cstring), + OutputFunc: toFuncID("int4out", oid.T_int4), + ReceiveFunc: toFuncID("int4recv", oid.T_internal), + SendFunc: toFuncID("int4send", oid.T_int4), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btint4cmp", oid.T_int4, oid.T_int4), + InternalName: "integer", } diff --git a/server/types/int32_array.go b/server/types/int32_array.go index de3ef85861..e9d3fa0a2a 100644 --- a/server/types/int32_array.go +++ b/server/types/int32_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // Int32Array is the array variant of Int32. -var Int32Array = createArrayType(Int32, SerializationID_Int32Array, oid.T__int4) +var Int32Array = CreateArrayTypeFromBaseType(Int32) diff --git a/server/types/int32_serial.go b/server/types/int32_serial.go index 980b850406..3216284e5c 100644 --- a/server/types/int32_serial.go +++ b/server/types/int32_serial.go @@ -14,167 +14,43 @@ package types -import ( - "fmt" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" - "github.com/lib/pq/oid" -) - -// Int32Serial is an int16 serial type. -var Int32Serial = Int32TypeSerial{} - -// Int32TypeSerial is the extended type implementation of the PostgreSQL serial. -type Int32TypeSerial struct{} - -var _ DoltgresType = Int32TypeSerial{} - -// Alignment implements the DoltgresType interface. -func (b Int32TypeSerial) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b Int32TypeSerial) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Int32Serial -} - -// BaseName implements the DoltgresType interface. -func (b Int32TypeSerial) BaseName() string { - return "serial" -} - -// Category implements the DoltgresType interface. -func (b Int32TypeSerial) Category() TypeCategory { - return TypeCategory_UnknownTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b Int32TypeSerial) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b Int32TypeSerial) Compare(v1 any, v2 any) (int, error) { - return 0, fmt.Errorf("SERIAL types are not comparable") -} - -// Convert implements the DoltgresType interface. -func (b Int32TypeSerial) Convert(val any) (any, sql.ConvertInRange, error) { - return nil, sql.OutOfRange, fmt.Errorf("SERIAL types are not convertable") -} - -// Equals implements the DoltgresType interface. -func (b Int32TypeSerial) Equals(otherType sql.Type) bool { - _, ok := otherType.(Int32TypeSerial) - return ok -} - -// FormatValue implements the DoltgresType interface. -func (b Int32TypeSerial) FormatValue(val any) (string, error) { - return "", fmt.Errorf("SERIAL types are not formattable") -} - -// GetSerializationID implements the DoltgresType interface. -func (b Int32TypeSerial) GetSerializationID() SerializationID { - return SerializationID_Invalid -} - -// IoInput implements the DoltgresType interface. -func (b Int32TypeSerial) IoInput(ctx *sql.Context, input string) (any, error) { - return "", fmt.Errorf("SERIAL types cannot receive I/O input") -} - -// IoOutput implements the DoltgresType interface. -func (b Int32TypeSerial) IoOutput(ctx *sql.Context, output any) (string, error) { - return "", fmt.Errorf("SERIAL types cannot produce I/O output") -} - -// IsPreferredType implements the DoltgresType interface. -func (b Int32TypeSerial) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b Int32TypeSerial) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b Int32TypeSerial) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b Int32TypeSerial) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 4 -} - -// OID implements the DoltgresType interface. -func (b Int32TypeSerial) OID() uint32 { - return uint32(oid.T_int4) -} - -// Promote implements the DoltgresType interface. -func (b Int32TypeSerial) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b Int32TypeSerial) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - return 0, fmt.Errorf("SERIAL types are not comparable") -} - -// SQL implements the DoltgresType interface. -func (b Int32TypeSerial) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - return sqltypes.Value{}, fmt.Errorf("SERIAL types may not be passed over the wire") -} - -// String implements the DoltgresType interface. -func (b Int32TypeSerial) String() string { - return "serial" -} - -// ToArrayType implements the DoltgresType interface. -func (b Int32TypeSerial) ToArrayType() DoltgresArrayType { - return Unknown -} - -// Type implements the DoltgresType interface. -func (b Int32TypeSerial) Type() query.Type { - return sqltypes.Int32 -} - -// ValueType implements the DoltgresType interface. -func (b Int32TypeSerial) ValueType() reflect.Type { - return reflect.TypeOf(int32(0)) -} - -// Zero implements the DoltgresType interface. -func (b Int32TypeSerial) Zero() any { - return int32(0) -} - -// SerializeType implements the DoltgresType interface. -func (b Int32TypeSerial) SerializeType() ([]byte, error) { - return nil, fmt.Errorf("SERIAL types are not serializable") -} - -// deserializeType implements the DoltgresType interface. -func (b Int32TypeSerial) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - return nil, fmt.Errorf("SERIAL types are not deserializable") -} - -// SerializeValue implements the DoltgresType interface. -func (b Int32TypeSerial) SerializeValue(val any) ([]byte, error) { - return nil, fmt.Errorf("SERIAL types are not serializable") -} - -// DeserializeValue implements the DoltgresType interface. -func (b Int32TypeSerial) DeserializeValue(val []byte) (any, error) { - return nil, fmt.Errorf("SERIAL types are not deserializable") +import "github.com/lib/pq/oid" + +// Int32Serial is an int32 serial type. +var Int32Serial = &DoltgresType{ + OID: 0, // doesn't have unique OID + Name: "serial", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__int4), + InputFunc: toFuncID("int4in", oid.T_cstring), + OutputFunc: toFuncID("int4out", oid.T_int4), + ReceiveFunc: toFuncID("int4recv"), + SendFunc: toFuncID("int4send"), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btint4cmp", oid.T_int4, oid.T_int4), + IsSerial: true, } diff --git a/server/types/int64.go b/server/types/int64.go index b08de193c3..e89dc5b0a6 100644 --- a/server/types/int64.go +++ b/server/types/int64.go @@ -15,247 +15,44 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "reflect" - "strconv" - "strings" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Int64 is an int64. -var Int64 = Int64Type{} - -// Int64Type is the extended type implementation of the PostgreSQL bigint. -type Int64Type struct{} - -var _ DoltgresType = Int64Type{} - -// Alignment implements the DoltgresType interface. -func (b Int64Type) Alignment() TypeAlignment { - return TypeAlignment_Double -} - -// BaseID implements the DoltgresType interface. -func (b Int64Type) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Int64 -} - -// BaseName implements the DoltgresType interface. -func (b Int64Type) BaseName() string { - return "int8" -} - -// Category implements the DoltgresType interface. -func (b Int64Type) Category() TypeCategory { - return TypeCategory_NumericTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b Int64Type) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b Int64Type) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(int64) - bb := bc.(int64) - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b Int64Type) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case int64: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b Int64Type) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b Int64Type) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b Int64Type) GetSerializationID() SerializationID { - return SerializationID_Int64 -} - -// IoInput implements the DoltgresType interface. -func (b Int64Type) IoInput(ctx *sql.Context, input string) (any, error) { - val, err := strconv.ParseInt(strings.TrimSpace(input), 10, 64) - if err != nil { - return nil, fmt.Errorf("invalid input syntax for type %s: %q", b.String(), input) - } - return val, nil -} - -// IoOutput implements the DoltgresType interface. -func (b Int64Type) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return strconv.FormatInt(converted.(int64), 10), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b Int64Type) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b Int64Type) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b Int64Type) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b Int64Type) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 8 -} - -// OID implements the DoltgresType interface. -func (b Int64Type) OID() uint32 { - return uint32(oid.T_int8) -} - -// Promote implements the DoltgresType interface. -func (b Int64Type) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b Int64Type) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b Int64Type) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b Int64Type) String() string { - return "bigint" -} - -// ToArrayType implements the DoltgresType interface. -func (b Int64Type) ToArrayType() DoltgresArrayType { - return Int64Array -} - -// Type implements the DoltgresType interface. -func (b Int64Type) Type() query.Type { - return sqltypes.Int64 -} - -// ValueType implements the DoltgresType interface. -func (b Int64Type) ValueType() reflect.Type { - return reflect.TypeOf(int64(0)) -} - -// Zero implements the DoltgresType interface. -func (b Int64Type) Zero() any { - return int64(0) -} - -// SerializeType implements the DoltgresType interface. -func (b Int64Type) SerializeType() ([]byte, error) { - return SerializationID_Int64.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b Int64Type) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Int64, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b Int64Type) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - retVal := make([]byte, 8) - binary.BigEndian.PutUint64(retVal, uint64(converted.(int64))+(1<<63)) - return retVal, nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b Int64Type) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - return int64(binary.BigEndian.Uint64(val) - (1 << 63)), nil +var Int64 = &DoltgresType{ + OID: uint32(oid.T_int8), + Name: "int8", + Schema: "pg_catalog", + TypLength: int16(8), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__int8), + InputFunc: toFuncID("int8in", oid.T_cstring), + OutputFunc: toFuncID("int8out", oid.T_int8), + ReceiveFunc: toFuncID("int8recv", oid.T_internal), + SendFunc: toFuncID("int8send", oid.T_int8), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Double, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btint8cmp", oid.T_int8, oid.T_int8), + InternalName: "bigint", } diff --git a/server/types/int64_array.go b/server/types/int64_array.go index 8ee4ea966d..62308261f0 100644 --- a/server/types/int64_array.go +++ b/server/types/int64_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // Int64Array is the array variant of Int64. -var Int64Array = createArrayType(Int64, SerializationID_Int64Array, oid.T__int8) +var Int64Array = CreateArrayTypeFromBaseType(Int64) diff --git a/server/types/int64_serial.go b/server/types/int64_serial.go index d92681b342..790951f082 100644 --- a/server/types/int64_serial.go +++ b/server/types/int64_serial.go @@ -14,167 +14,43 @@ package types -import ( - "fmt" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" - "github.com/lib/pq/oid" -) - -// Int64Serial is an int16 serial type. -var Int64Serial = Int64TypeSerial{} - -// Int64TypeSerial is the extended type implementation of the PostgreSQL bigserial. -type Int64TypeSerial struct{} - -var _ DoltgresType = Int64TypeSerial{} - -// Alignment implements the DoltgresType interface. -func (b Int64TypeSerial) Alignment() TypeAlignment { - return TypeAlignment_Double -} - -// BaseID implements the DoltgresType interface. -func (b Int64TypeSerial) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Int64Serial -} - -// BaseName implements the DoltgresType interface. -func (b Int64TypeSerial) BaseName() string { - return "bigserial" -} - -// Category implements the DoltgresType interface. -func (b Int64TypeSerial) Category() TypeCategory { - return TypeCategory_UnknownTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b Int64TypeSerial) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b Int64TypeSerial) Compare(v1 any, v2 any) (int, error) { - return 0, fmt.Errorf("SERIAL types are not comparable") -} - -// Convert implements the DoltgresType interface. -func (b Int64TypeSerial) Convert(val any) (any, sql.ConvertInRange, error) { - return nil, sql.OutOfRange, fmt.Errorf("SERIAL types are not convertable") -} - -// Equals implements the DoltgresType interface. -func (b Int64TypeSerial) Equals(otherType sql.Type) bool { - _, ok := otherType.(Int64TypeSerial) - return ok -} - -// FormatValue implements the DoltgresType interface. -func (b Int64TypeSerial) FormatValue(val any) (string, error) { - return "", fmt.Errorf("SERIAL types are not formattable") -} - -// GetSerializationID implements the DoltgresType interface. -func (b Int64TypeSerial) GetSerializationID() SerializationID { - return SerializationID_Invalid -} - -// IoInput implements the DoltgresType interface. -func (b Int64TypeSerial) IoInput(ctx *sql.Context, input string) (any, error) { - return "", fmt.Errorf("SERIAL types cannot receive I/O input") -} - -// IoOutput implements the DoltgresType interface. -func (b Int64TypeSerial) IoOutput(ctx *sql.Context, output any) (string, error) { - return "", fmt.Errorf("SERIAL types cannot produce I/O output") -} - -// IsPreferredType implements the DoltgresType interface. -func (b Int64TypeSerial) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b Int64TypeSerial) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b Int64TypeSerial) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b Int64TypeSerial) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 8 -} - -// OID implements the DoltgresType interface. -func (b Int64TypeSerial) OID() uint32 { - return uint32(oid.T_int8) -} - -// Promote implements the DoltgresType interface. -func (b Int64TypeSerial) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b Int64TypeSerial) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - return 0, fmt.Errorf("SERIAL types are not comparable") -} - -// SQL implements the DoltgresType interface. -func (b Int64TypeSerial) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - return sqltypes.Value{}, fmt.Errorf("SERIAL types may not be passed over the wire") -} - -// String implements the DoltgresType interface. -func (b Int64TypeSerial) String() string { - return "bigserial" -} - -// ToArrayType implements the DoltgresType interface. -func (b Int64TypeSerial) ToArrayType() DoltgresArrayType { - return Unknown -} - -// Type implements the DoltgresType interface. -func (b Int64TypeSerial) Type() query.Type { - return sqltypes.Int64 -} - -// ValueType implements the DoltgresType interface. -func (b Int64TypeSerial) ValueType() reflect.Type { - return reflect.TypeOf(int64(0)) -} - -// Zero implements the DoltgresType interface. -func (b Int64TypeSerial) Zero() any { - return int64(0) -} - -// SerializeType implements the DoltgresType interface. -func (b Int64TypeSerial) SerializeType() ([]byte, error) { - return nil, fmt.Errorf("SERIAL types are not serializable") -} - -// deserializeType implements the DoltgresType interface. -func (b Int64TypeSerial) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - return nil, fmt.Errorf("SERIAL types are not deserializable") -} - -// SerializeValue implements the DoltgresType interface. -func (b Int64TypeSerial) SerializeValue(val any) ([]byte, error) { - return nil, fmt.Errorf("SERIAL types are not serializable") -} - -// DeserializeValue implements the DoltgresType interface. -func (b Int64TypeSerial) DeserializeValue(val []byte) (any, error) { - return nil, fmt.Errorf("SERIAL types are not deserializable") +import "github.com/lib/pq/oid" + +// Int64Serial is an int64 serial type. +var Int64Serial = &DoltgresType{ + OID: 0, // doesn't have unique OID + Name: "bigserial", + Schema: "pg_catalog", + TypLength: int16(8), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__int8), + InputFunc: toFuncID("int8in", oid.T_cstring), + OutputFunc: toFuncID("int8out", oid.T_int8), + ReceiveFunc: toFuncID("int8recv"), + SendFunc: toFuncID("int8send"), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Double, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btint8cmp", oid.T_int8, oid.T_int8), + IsSerial: true, } diff --git a/server/types/interface.go b/server/types/interface.go deleted file mode 100644 index 10566978fb..0000000000 --- a/server/types/interface.go +++ /dev/null @@ -1,374 +0,0 @@ -// Copyright 2024 Dolthub, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package types - -import ( - "sort" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/lib/pq/oid" - "gopkg.in/src-d/go-errors.v1" -) - -var ErrTypeAlreadyExists = errors.NewKind(`type "%s" already exists`) -var ErrTypeDoesNotExist = errors.NewKind(`type "%s" does not exist`) - -// Type represents a single type. -type Type struct { - Oid uint32 - Name string - Schema string // TODO: should be `uint32`. - Owner string // TODO: should be `uint32`. - Length int16 - PassedByVal bool - TypType TypeType - TypCategory TypeCategory - IsPreferred bool - IsDefined bool - Delimiter string - RelID uint32 // for Composite types - SubscriptFunc string - Elem uint32 - Array uint32 - InputFunc string - OutputFunc string - ReceiveFunc string - SendFunc string - ModInFunc string - ModOutFunc string - AnalyzeFunc string - Align TypeAlignment - Storage TypeStorage - NotNull bool // for Domain types - BaseTypeOID uint32 // for Domain types - TypMod int32 // for Domain types - NDims int32 // for Domain types - Collation uint32 - DefaulBin string // for Domain types - Default string - Acl string // TODO: list of privileges - Checks []*sql.CheckDefinition // TODO: this is not part of `pg_type` instead `pg_constraint` for Domain types. -} - -// DoltgresType is a type that is distinct from the MySQL types in GMS. -type DoltgresType interface { - types.ExtendedType - // Alignment returns a char representing the alignment required when storing a value of this type. - Alignment() TypeAlignment - // BaseID returns the DoltgresTypeBaseID for this type. - BaseID() DoltgresTypeBaseID - // BaseName returns the name of the type displayed in pg_catalog tables. - BaseName() string - // Category returns a char representing an arbitrary classification of data types that is used by the parser to determine which implicit casts should be “preferred”. - Category() TypeCategory - // GetSerializationID returns the SerializationID for this type. - GetSerializationID() SerializationID - // IoInput returns a value from the given input string. This function mirrors Postgres' I/O input function. Such - // strings are intended for serialization and automatic cross-type conversion. An input string will never represent - // NULL. - IoInput(ctx *sql.Context, input string) (any, error) - // IoOutput returns a string from the given output value. This function mirrors Postgres' I/O output function. These - // strings are not intended for output, but are instead intended for serialization and cross-type conversion. Output - // values will always be non-NULL. - IoOutput(ctx *sql.Context, output any) (string, error) - // IsPreferredType returns true if the type is preferred type. - IsPreferredType() bool - // IsUnbounded returns whether the type is unbounded. Unbounded types do not enforce a length, precision, etc. on - // values. All values are still bound by the field size limit, but that differs from any type-enforced limits. - IsUnbounded() bool - // OID returns an OID that we are associating with this type. OIDs are not unique, and are not guaranteed to be the - // same between versions of Postgres. However, they've so far appeared relatively stable, and many libraries rely on - // them for type identification, so we return them here. These should not be used for any sort of identification on - // our side. For that, we should use DoltgresTypeBaseID, which we can guarantee will be unique and non-changing once - // we've stabilized development. - OID() uint32 - // SerializeType returns a byte slice representing the serialized form of the type. All serialized types MUST start - // with their SerializationID. Deserialization is done through the DeserializeType function. - SerializeType() ([]byte, error) - // deserializeType returns a new type based on the given version and metadata. The metadata is all data after the - // serialization header. This is called from within the types package. To deserialize types normally, use - // DeserializeType, which will call this as needed. - deserializeType(version uint16, metadata []byte) (DoltgresType, error) - // ToArrayType converts the calling DoltgresType into its corresponding array type. When called on a - // DoltgresArrayType, then it simply returns itself, as a multidimensional or nested array is equivalent to a - // standard array. - ToArrayType() DoltgresArrayType -} - -// DoltgresArrayType is a DoltgresType that represents an array variant of a non-array type. -type DoltgresArrayType interface { - DoltgresType - // BaseType is the inner type of the array. This will always be a non-array type. - BaseType() DoltgresType -} - -// DoltgresPolymorphicType is a DoltgresType that represents one of the polymorphic types. These types are special -// built-in pseudo-types that are used during function resolution to allow a function to handle multiple types from a -// single definition. All polymorphic types have "any" as a prefix. The exception is the "any" type, which is not a -// polymorphic type. -type DoltgresPolymorphicType interface { - DoltgresType - // IsValid returns whether the given type is valid for the calling polymorphic type. - IsValid(target DoltgresType) bool -} - -// typesFromBaseID contains a map from a DoltgresTypeBaseID to its originating type. -var typesFromBaseID = map[DoltgresTypeBaseID]DoltgresType{ - AnyArray.BaseID(): AnyArray, - AnyElement.BaseID(): AnyElement, - AnyNonArray.BaseID(): AnyNonArray, - BpChar.BaseID(): BpChar, - BpCharArray.BaseID(): BpCharArray, - Bool.BaseID(): Bool, - BoolArray.BaseID(): BoolArray, - Bytea.BaseID(): Bytea, - ByteaArray.BaseID(): ByteaArray, - Date.BaseID(): Date, - DateArray.BaseID(): DateArray, - Float32.BaseID(): Float32, - Float32Array.BaseID(): Float32Array, - Float64.BaseID(): Float64, - Float64Array.BaseID(): Float64Array, - Int16.BaseID(): Int16, - Int16Array.BaseID(): Int16Array, - Int16Serial.BaseID(): Int16Serial, - Int32.BaseID(): Int32, - Int32Array.BaseID(): Int32Array, - Int32Serial.BaseID(): Int32Serial, - Int64.BaseID(): Int64, - Int64Array.BaseID(): Int64Array, - Int64Serial.BaseID(): Int64Serial, - InternalChar.BaseID(): InternalChar, - InternalCharArray.BaseID(): InternalCharArray, - Interval.BaseID(): Interval, - IntervalArray.BaseID(): IntervalArray, - Json.BaseID(): Json, - JsonArray.BaseID(): JsonArray, - JsonB.BaseID(): JsonB, - JsonBArray.BaseID(): JsonBArray, - Name.BaseID(): Name, - NameArray.BaseID(): NameArray, - Numeric.BaseID(): Numeric, - NumericArray.BaseID(): NumericArray, - Oid.BaseID(): Oid, - OidArray.BaseID(): OidArray, - Regclass.BaseID(): Regclass, - RegclassArray.BaseID(): RegclassArray, - Regproc.BaseID(): Regproc, - RegprocArray.BaseID(): RegprocArray, - Regtype.BaseID(): Regtype, - RegtypeArray.BaseID(): RegtypeArray, - Text.BaseID(): Text, - TextArray.BaseID(): TextArray, - Time.BaseID(): Time, - TimeArray.BaseID(): TimeArray, - Timestamp.BaseID(): Timestamp, - TimestampArray.BaseID(): TimestampArray, - TimestampTZ.BaseID(): TimestampTZ, - TimestampTZArray.BaseID(): TimestampTZArray, - TimeTZ.BaseID(): TimeTZ, - TimeTZArray.BaseID(): TimeTZArray, - Uuid.BaseID(): Uuid, - UuidArray.BaseID(): UuidArray, - Unknown.BaseID(): Unknown, - VarChar.BaseID(): VarChar, - VarCharArray.BaseID(): VarCharArray, - Xid.BaseID(): Xid, - XidArray.BaseID(): XidArray, -} - -// GetAllTypes returns a slice containing all registered types. The slice is sorted by each type's base ID. -func GetAllTypes() []DoltgresType { - pgTypes := make([]DoltgresType, 0, len(typesFromBaseID)) - for _, typ := range typesFromBaseID { - pgTypes = append(pgTypes, typ) - } - sort.Slice(pgTypes, func(i, j int) bool { - return pgTypes[i].BaseID() < pgTypes[j].BaseID() - }) - return pgTypes -} - -// OidToBuildInDoltgresType is map of oid to built-in Doltgres type. -var OidToBuildInDoltgresType = map[uint32]DoltgresType{ - uint32(oid.T_bool): Bool, - uint32(oid.T_bytea): Bytea, - uint32(oid.T_char): InternalChar, - uint32(oid.T_name): Name, - uint32(oid.T_int8): Int64, - uint32(oid.T_int2): Int16, - uint32(oid.T_int2vector): Unknown, - uint32(oid.T_int4): Int32, - uint32(oid.T_regproc): Regproc, - uint32(oid.T_text): Text, - uint32(oid.T_oid): Oid, - uint32(oid.T_tid): Unknown, - uint32(oid.T_xid): Xid, - uint32(oid.T_cid): Unknown, - uint32(oid.T_oidvector): Unknown, - uint32(oid.T_pg_ddl_command): Unknown, - uint32(oid.T_pg_type): Unknown, - uint32(oid.T_pg_attribute): Unknown, - uint32(oid.T_pg_proc): Unknown, - uint32(oid.T_pg_class): Unknown, - uint32(oid.T_json): Json, - uint32(oid.T_xml): Unknown, - uint32(oid.T__xml): Unknown, - uint32(oid.T_pg_node_tree): Unknown, - uint32(oid.T__json): JsonArray, - uint32(oid.T_smgr): Unknown, - uint32(oid.T_index_am_handler): Unknown, - uint32(oid.T_point): Unknown, - uint32(oid.T_lseg): Unknown, - uint32(oid.T_path): Unknown, - uint32(oid.T_box): Unknown, - uint32(oid.T_polygon): Unknown, - uint32(oid.T_line): Unknown, - uint32(oid.T__line): Unknown, - uint32(oid.T_cidr): Unknown, - uint32(oid.T__cidr): Unknown, - uint32(oid.T_float4): Float32, - uint32(oid.T_float8): Float64, - uint32(oid.T_abstime): Unknown, - uint32(oid.T_reltime): Unknown, - uint32(oid.T_tinterval): Unknown, - uint32(oid.T_unknown): Unknown, - uint32(oid.T_circle): Unknown, - uint32(oid.T__circle): Unknown, - uint32(oid.T_money): Unknown, - uint32(oid.T__money): Unknown, - uint32(oid.T_macaddr): Unknown, - uint32(oid.T_inet): Unknown, - uint32(oid.T__bool): BoolArray, - uint32(oid.T__bytea): ByteaArray, - uint32(oid.T__char): InternalCharArray, - uint32(oid.T__name): NameArray, - uint32(oid.T__int2): Int16Array, - uint32(oid.T__int2vector): Unknown, - uint32(oid.T__int4): Int32Array, - uint32(oid.T__regproc): RegprocArray, - uint32(oid.T__text): TextArray, - uint32(oid.T__tid): Unknown, - uint32(oid.T__xid): XidArray, - uint32(oid.T__cid): Unknown, - uint32(oid.T__oidvector): Unknown, - uint32(oid.T__bpchar): BpCharArray, - uint32(oid.T__varchar): VarCharArray, - uint32(oid.T__int8): Int64Array, - uint32(oid.T__point): Unknown, - uint32(oid.T__lseg): Unknown, - uint32(oid.T__path): Unknown, - uint32(oid.T__box): Unknown, - uint32(oid.T__float4): Float32Array, - uint32(oid.T__float8): Float64Array, - uint32(oid.T__abstime): Unknown, - uint32(oid.T__reltime): Unknown, - uint32(oid.T__tinterval): Unknown, - uint32(oid.T__polygon): Unknown, - uint32(oid.T__oid): OidArray, - uint32(oid.T_aclitem): Unknown, - uint32(oid.T__aclitem): Unknown, - uint32(oid.T__macaddr): Unknown, - uint32(oid.T__inet): Unknown, - uint32(oid.T_bpchar): BpChar, - uint32(oid.T_varchar): VarChar, - uint32(oid.T_date): Date, - uint32(oid.T_time): Time, - uint32(oid.T_timestamp): Timestamp, - uint32(oid.T__timestamp): TimestampArray, - uint32(oid.T__date): DateArray, - uint32(oid.T__time): TimeArray, - uint32(oid.T_timestamptz): TimestampTZ, - uint32(oid.T__timestamptz): TimestampTZArray, - uint32(oid.T_interval): Interval, - uint32(oid.T__interval): IntervalArray, - uint32(oid.T__numeric): NumericArray, - uint32(oid.T_pg_database): Unknown, - uint32(oid.T__cstring): Unknown, - uint32(oid.T_timetz): TimeTZ, - uint32(oid.T__timetz): TimeTZArray, - uint32(oid.T_bit): Unknown, - uint32(oid.T__bit): Unknown, - uint32(oid.T_varbit): Unknown, - uint32(oid.T__varbit): Unknown, - uint32(oid.T_numeric): Numeric, - uint32(oid.T_refcursor): Unknown, - uint32(oid.T__refcursor): Unknown, - uint32(oid.T_regprocedure): Unknown, - uint32(oid.T_regoper): Unknown, - uint32(oid.T_regoperator): Unknown, - uint32(oid.T_regclass): Regclass, - uint32(oid.T_regtype): Regtype, - uint32(oid.T__regprocedure): Unknown, - uint32(oid.T__regoper): Unknown, - uint32(oid.T__regoperator): Unknown, - uint32(oid.T__regclass): RegclassArray, - uint32(oid.T__regtype): RegtypeArray, - uint32(oid.T_record): Unknown, - uint32(oid.T_cstring): Unknown, - uint32(oid.T_any): Unknown, - uint32(oid.T_anyarray): AnyArray, - uint32(oid.T_void): Unknown, - uint32(oid.T_trigger): Unknown, - uint32(oid.T_language_handler): Unknown, - uint32(oid.T_internal): Unknown, - uint32(oid.T_opaque): Unknown, - uint32(oid.T_anyelement): AnyElement, - uint32(oid.T__record): Unknown, - uint32(oid.T_anynonarray): AnyNonArray, - uint32(oid.T_pg_authid): Unknown, - uint32(oid.T_pg_auth_members): Unknown, - uint32(oid.T__txid_snapshot): Unknown, - uint32(oid.T_uuid): Uuid, - uint32(oid.T__uuid): UuidArray, - uint32(oid.T_txid_snapshot): Unknown, - uint32(oid.T_fdw_handler): Unknown, - uint32(oid.T_pg_lsn): Unknown, - uint32(oid.T__pg_lsn): Unknown, - uint32(oid.T_tsm_handler): Unknown, - uint32(oid.T_anyenum): Unknown, - uint32(oid.T_tsvector): Unknown, - uint32(oid.T_tsquery): Unknown, - uint32(oid.T_gtsvector): Unknown, - uint32(oid.T__tsvector): Unknown, - uint32(oid.T__gtsvector): Unknown, - uint32(oid.T__tsquery): Unknown, - uint32(oid.T_regconfig): Unknown, - uint32(oid.T__regconfig): Unknown, - uint32(oid.T_regdictionary): Unknown, - uint32(oid.T__regdictionary): Unknown, - uint32(oid.T_jsonb): JsonB, - uint32(oid.T__jsonb): JsonBArray, - uint32(oid.T_anyrange): Unknown, - uint32(oid.T_event_trigger): Unknown, - uint32(oid.T_int4range): Unknown, - uint32(oid.T__int4range): Unknown, - uint32(oid.T_numrange): Unknown, - uint32(oid.T__numrange): Unknown, - uint32(oid.T_tsrange): Unknown, - uint32(oid.T__tsrange): Unknown, - uint32(oid.T_tstzrange): Unknown, - uint32(oid.T__tstzrange): Unknown, - uint32(oid.T_daterange): Unknown, - uint32(oid.T__daterange): Unknown, - uint32(oid.T_int8range): Unknown, - uint32(oid.T__int8range): Unknown, - uint32(oid.T_pg_shseclabel): Unknown, - uint32(oid.T_regnamespace): Unknown, - uint32(oid.T__regnamespace): Unknown, - uint32(oid.T_regrole): Unknown, - uint32(oid.T__regrole): Unknown, -} diff --git a/server/types/internal.go b/server/types/internal.go new file mode 100644 index 0000000000..2a45a26c7e --- /dev/null +++ b/server/types/internal.go @@ -0,0 +1,63 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package types + +import "github.com/lib/pq/oid" + +// Internal is an internal type, which means `external binary` type. +var Internal = &DoltgresType{ + OID: uint32(oid.T_internal), + Name: "internal", + Schema: "pg_catalog", + TypLength: int16(8), + PassedByVal: true, + TypType: TypeType_Pseudo, + TypCategory: TypeCategory_PseudoTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: 0, + InputFunc: toFuncID("internal_in", oid.T_cstring), + OutputFunc: toFuncID("internal_out", oid.T_internal), + ReceiveFunc: toFuncID("-"), + SendFunc: toFuncID("-"), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Double, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), +} + +// NewInternalTypeWithBaseType returns Internal type with +// internal base type set with given type. +func NewInternalTypeWithBaseType(t uint32) *DoltgresType { + it := Internal + it.BaseTypeForInternal = t + return it +} diff --git a/server/types/internal_char.go b/server/types/internal_char.go index 57d662add4..7020ab7717 100644 --- a/server/types/internal_char.go +++ b/server/types/internal_char.go @@ -15,259 +15,47 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "reflect" - "strings" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" - - "github.com/dolthub/doltgresql/utils" ) // InternalCharLength will always be 1. const InternalCharLength = 1 // InternalChar is a single-byte internal type. In Postgres, it's displayed as "char". -var InternalChar = InternalCharType{} - -// InternalCharType is the type implementation of the internal PostgreSQL "char" type. -type InternalCharType struct{} - -var _ DoltgresType = InternalCharType{} - -// Alignment implements the DoltgresType interface. -func (b InternalCharType) Alignment() TypeAlignment { - return TypeAlignment_Char -} - -// BaseID implements the DoltgresType interface. -func (b InternalCharType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_InternalChar -} - -// BaseName implements the DoltgresType interface. -func (b InternalCharType) BaseName() string { - return `"char"` -} - -// Category implements the DoltgresType interface. -func (b InternalCharType) Category() TypeCategory { - return TypeCategory_InternalUseTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b InternalCharType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b InternalCharType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := strings.TrimRight(ac.(string), " ") - bb := strings.TrimRight(bc.(string), " ") - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b InternalCharType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case string: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b InternalCharType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b InternalCharType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b InternalCharType) GetSerializationID() SerializationID { - return SerializationID_InternalChar -} - -// IoInput implements the DoltgresType interface. -func (b InternalCharType) IoInput(ctx *sql.Context, input string) (any, error) { - c := []byte(input) - if uint32(len(c)) > InternalCharLength { - return input[:InternalCharLength], nil - } - return input, nil -} - -// IoOutput implements the DoltgresType interface. -func (b InternalCharType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - str := converted.(string) - if uint32(len(str)) > InternalCharLength { - return str[:InternalCharLength], nil - } - return str, nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b InternalCharType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b InternalCharType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b InternalCharType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b InternalCharType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return InternalCharLength -} - -// OID implements the DoltgresType interface. -func (b InternalCharType) OID() uint32 { - return uint32(oid.T_char) -} - -// Promote implements the DoltgresType interface. -func (b InternalCharType) Promote() sql.Type { - return InternalChar -} - -// SerializedCompare implements the DoltgresType interface. -func (b InternalCharType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - return serializedStringCompare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b InternalCharType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b InternalCharType) String() string { - return `"char"` -} - -// ToArrayType implements the DoltgresType interface. -func (b InternalCharType) ToArrayType() DoltgresArrayType { - return InternalCharArray -} - -// Type implements the DoltgresType interface. -func (b InternalCharType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b InternalCharType) ValueType() reflect.Type { - return reflect.TypeOf("") -} - -// Zero implements the DoltgresType interface. -func (b InternalCharType) Zero() any { - return "" -} - -// SerializeType implements the DoltgresType interface. -func (b InternalCharType) SerializeType() ([]byte, error) { - t := make([]byte, serializationIDHeaderSize+4) - copy(t, SerializationID_InternalChar.ToByteSlice(0)) - binary.LittleEndian.PutUint32(t[serializationIDHeaderSize:], InternalCharLength) - return t, nil -} - -// deserializeType implements the DoltgresType interface. -func (b InternalCharType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return InternalCharType{}, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b InternalCharType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - str := converted.(string) - writer := utils.NewWriter(uint64(len(str) + 4)) - writer.String(str) - return writer.Data(), nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b InternalCharType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - reader := utils.NewReader(val) - return reader.String(), nil +var InternalChar = &DoltgresType{ + OID: uint32(oid.T_char), + Name: "char", + Schema: "pg_catalog", + TypLength: int16(InternalCharLength), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_InternalUseTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__char), + InputFunc: toFuncID("charin", oid.T_cstring), + OutputFunc: toFuncID("charout", oid.T_char), + ReceiveFunc: toFuncID("charrecv", oid.T_internal), + SendFunc: toFuncID("charsend", oid.T_char), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Char, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btcharcmp", oid.T_char, oid.T_char), + InternalName: `"char"`, } diff --git a/server/types/internal_char_array.go b/server/types/internal_char_array.go index 25f045eef0..96da9aaad1 100644 --- a/server/types/internal_char_array.go +++ b/server/types/internal_char_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // InternalCharArray is the array variant of InternalChar. -var InternalCharArray = createArrayType(InternalChar, SerializationID_InternalCharArray, oid.T__char) +var InternalCharArray = CreateArrayTypeFromBaseType(InternalChar) diff --git a/server/types/interval.go b/server/types/interval.go index b942b8e718..ad12bc7e5f 100644 --- a/server/types/interval.go +++ b/server/types/interval.go @@ -15,254 +15,43 @@ package types import ( - "bytes" - "fmt" - "reflect" - - "github.com/dolthub/doltgresql/postgres/parser/duration" - "github.com/dolthub/doltgresql/postgres/parser/sem/tree" - "github.com/dolthub/doltgresql/utils" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Interval is the interval type. -var Interval = IntervalType{} - -// IntervalType is the extended type implementation of the PostgreSQL interval. -type IntervalType struct{} - -var _ DoltgresType = IntervalType{} - -// Alignment implements the DoltgresType interface. -func (b IntervalType) Alignment() TypeAlignment { - return TypeAlignment_Double -} - -// BaseID implements the DoltgresType interface. -func (b IntervalType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Interval -} - -// BaseName implements the DoltgresType interface. -func (b IntervalType) BaseName() string { - return "interval" -} - -// Category implements the DoltgresType interface. -func (b IntervalType) Category() TypeCategory { - return TypeCategory_TimespanTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b IntervalType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b IntervalType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(duration.Duration) - bb := bc.(duration.Duration) - return ab.Compare(bb), nil -} - -// Convert implements the DoltgresType interface. -func (b IntervalType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case duration.Duration: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b IntervalType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b IntervalType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b IntervalType) GetSerializationID() SerializationID { - return SerializationID_Interval -} - -// IoInput implements the DoltgresType interface. -func (b IntervalType) IoInput(ctx *sql.Context, input string) (any, error) { - dInterval, err := tree.ParseDInterval(input) - if err != nil { - return nil, err - } - return dInterval.Duration, nil -} - -// IoOutput implements the DoltgresType interface. -func (b IntervalType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - // TODO: depends on `intervalStyle` configuration variable. Defaults to `postgres`. - d := converted.(duration.Duration) - return d.String(), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b IntervalType) IsPreferredType() bool { - return true -} - -// IsUnbounded implements the DoltgresType interface. -func (b IntervalType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b IntervalType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b IntervalType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 16 -} - -// OID implements the DoltgresType interface. -func (b IntervalType) OID() uint32 { - return uint32(oid.T_interval) -} - -// Promote implements the DoltgresType interface. -func (b IntervalType) Promote() sql.Type { - return Interval -} - -// SerializedCompare implements the DoltgresType interface. -func (b IntervalType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b IntervalType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b IntervalType) String() string { - return "interval" -} - -// ToArrayType implements the DoltgresType interface. -func (b IntervalType) ToArrayType() DoltgresArrayType { - return IntervalArray -} - -// Type implements the DoltgresType interface. -func (b IntervalType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b IntervalType) ValueType() reflect.Type { - return reflect.TypeOf(duration.MakeDuration(0, 0, 0)) -} - -// Zero implements the DoltgresType interface. -func (b IntervalType) Zero() any { - return duration.MakeDuration(0, 0, 0) -} - -// SerializeType implements the DoltgresType interface. -func (b IntervalType) SerializeType() ([]byte, error) { - return SerializationID_Interval.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b IntervalType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Interval, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b IntervalType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - sortNanos, months, days, err := converted.(duration.Duration).Encode() - if err != nil { - return nil, err - } - writer := utils.NewWriter(0) - writer.Int64(sortNanos) - writer.Int32(int32(months)) - writer.Int32(int32(days)) - return writer.Data(), nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b IntervalType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - reader := utils.NewReader(val) - sortNanos := reader.Int64() - months := reader.Int32() - days := reader.Int32() - return duration.Decode(sortNanos, int64(months), int64(days)) +var Interval = &DoltgresType{ + OID: uint32(oid.T_interval), + Name: "interval", + Schema: "pg_catalog", + TypLength: int16(16), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_TimespanTypes, + IsPreferred: true, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__interval), + InputFunc: toFuncID("interval_in", oid.T_cstring, oid.T_oid, oid.T_int4), + OutputFunc: toFuncID("interval_out", oid.T_interval), + ReceiveFunc: toFuncID("interval_recv", oid.T_internal, oid.T_oid, oid.T_int4), + SendFunc: toFuncID("interval_send", oid.T_interval), + ModInFunc: toFuncID("intervaltypmodin", oid.T__cstring), + ModOutFunc: toFuncID("intervaltypmodout", oid.T_int4), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Double, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("interval_cmp", oid.T_interval, oid.T_interval), } diff --git a/server/types/interval_array.go b/server/types/interval_array.go index 77e26ba9f6..b4a7e80adc 100644 --- a/server/types/interval_array.go +++ b/server/types/interval_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // IntervalArray is the array variant of Interval. -var IntervalArray = createArrayType(Interval, SerializationID_IntervalArray, oid.T__interval) +var IntervalArray = CreateArrayTypeFromBaseType(Interval) diff --git a/server/types/json.go b/server/types/json.go index ec3ec78fe9..25ad676c2f 100644 --- a/server/types/json.go +++ b/server/types/json.go @@ -15,245 +15,43 @@ package types import ( - "bytes" - "fmt" - "math" - "reflect" - "unsafe" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" - "github.com/goccy/go-json" "github.com/lib/pq/oid" ) // Json is the standard JSON type. -var Json = JsonType{} - -// JsonType is the extended type implementation of the PostgreSQL json. -type JsonType struct{} - -var _ DoltgresType = JsonType{} - -// Alignment implements the DoltgresType interface. -func (b JsonType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b JsonType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Json -} - -// BaseName implements the DoltgresType interface. -func (b JsonType) BaseName() string { - return "json" -} - -// Category implements the DoltgresType interface. -func (b JsonType) Category() TypeCategory { - return TypeCategory_UserDefinedTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b JsonType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b JsonType) Compare(v1 any, v2 any) (int, error) { - // JSON does not have any default ordering operators (ORDER BY does not work, etc.), so this is strictly for GMS/Dolt - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(string) - bb := bc.(string) - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b JsonType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case string: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b JsonType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b JsonType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b JsonType) GetSerializationID() SerializationID { - return SerializationID_Json -} - -// IoInput implements the DoltgresType interface. -func (b JsonType) IoInput(ctx *sql.Context, input string) (any, error) { - if json.Valid(unsafe.Slice(unsafe.StringData(input), len(input))) { - return input, nil - } - return nil, fmt.Errorf("invalid input syntax for type json") -} - -// IoOutput implements the DoltgresType interface. -func (b JsonType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return converted.(string), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b JsonType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b JsonType) IsUnbounded() bool { - return true -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b JsonType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_Unbounded -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b JsonType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return math.MaxUint32 -} - -// OID implements the DoltgresType interface. -func (b JsonType) OID() uint32 { - return uint32(oid.T_json) -} - -// Promote implements the DoltgresType interface. -func (b JsonType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b JsonType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b JsonType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b JsonType) String() string { - return "json" -} - -// ToArrayType implements the DoltgresType interface. -func (b JsonType) ToArrayType() DoltgresArrayType { - return JsonArray -} - -// Type implements the DoltgresType interface. -func (b JsonType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b JsonType) ValueType() reflect.Type { - return reflect.TypeOf("") -} - -// Zero implements the DoltgresType interface. -func (b JsonType) Zero() any { - return "" -} - -// SerializeType implements the DoltgresType interface. -func (b JsonType) SerializeType() ([]byte, error) { - return SerializationID_Json.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b JsonType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Json, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b JsonType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - return []byte(converted.(string)), nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b JsonType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - return string(val), nil +var Json = &DoltgresType{ + OID: uint32(oid.T_json), + Name: "json", + Schema: "pg_catalog", + TypLength: int16(-1), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_UserDefinedTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__json), + InputFunc: toFuncID("json_in", oid.T_cstring), + OutputFunc: toFuncID("json_out", oid.T_json), + ReceiveFunc: toFuncID("json_recv", oid.T_internal), + SendFunc: toFuncID("json_send", oid.T_json), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Extended, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), } diff --git a/server/types/json_array.go b/server/types/json_array.go index 1b0e261d10..d9f06c0386 100644 --- a/server/types/json_array.go +++ b/server/types/json_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // JsonArray is the array variant of Json. -var JsonArray = createArrayType(Json, SerializationID_JsonArray, oid.T__json) +var JsonArray = CreateArrayTypeFromBaseType(Json) diff --git a/server/types/json_document.go b/server/types/json_document.go index 71c3dc1139..22e62a1503 100644 --- a/server/types/json_document.go +++ b/server/types/json_document.go @@ -16,14 +16,16 @@ package types import ( "fmt" + "sort" "strings" + "github.com/goccy/go-json" "github.com/shopspring/decimal" "github.com/dolthub/doltgresql/utils" ) -// JsonValueType represents the type of a JSON value. These values are serialized, and therefore should never be modified. +// JsonValueType represents a JSON value type. These values are serialized, and therefore should never be modified. type JsonValueType byte const ( @@ -124,8 +126,8 @@ func JsonValueCopy(value JsonValue) JsonValue { } } -// jsonValueCompare compares two values. -func jsonValueCompare(v1 JsonValue, v2 JsonValue) int { +// JsonValueCompare compares two values. +func JsonValueCompare(v1 JsonValue, v2 JsonValue) int { // Some types sort before others, so we'll check those first v1TypeSortOrder := jsonValueTypeSortOrder(v1) v2TypeSortOrder := jsonValueTypeSortOrder(v2) @@ -151,7 +153,7 @@ func jsonValueCompare(v1 JsonValue, v2 JsonValue) int { } else if v1.Items[i].Key > v2.Items[i].Key { return 1 } else { - innerCmp := jsonValueCompare(v1.Items[i].Value, v2.Items[i].Value) + innerCmp := JsonValueCompare(v1.Items[i].Value, v2.Items[i].Value) if innerCmp != 0 { return innerCmp } @@ -166,7 +168,7 @@ func jsonValueCompare(v1 JsonValue, v2 JsonValue) int { return 1 } for i := 0; i < len(v1); i++ { - innerCmp := jsonValueCompare(v1[i], v2[i]) + innerCmp := JsonValueCompare(v1[i], v2[i]) if innerCmp != 0 { return innerCmp } @@ -220,21 +222,21 @@ func jsonValueTypeSortOrder(value JsonValue) int { } } -// jsonValueSerialize is the recursive serializer for JSON values. -func jsonValueSerialize(writer *utils.Writer, value JsonValue) { +// JsonValueSerialize is the recursive serializer for JSON values. +func JsonValueSerialize(writer *utils.Writer, value JsonValue) { switch value := value.(type) { case JsonValueObject: writer.Byte(byte(JsonValueType_Object)) writer.VariableUint(uint64(len(value.Items))) for _, item := range value.Items { writer.String(item.Key) - jsonValueSerialize(writer, item.Value) + JsonValueSerialize(writer, item.Value) } case JsonValueArray: writer.Byte(byte(JsonValueType_Array)) writer.VariableUint(uint64(len(value))) for _, item := range value { - jsonValueSerialize(writer, item) + JsonValueSerialize(writer, item) } case JsonValueString: writer.Byte(byte(JsonValueType_String)) @@ -252,15 +254,15 @@ func jsonValueSerialize(writer *utils.Writer, value JsonValue) { } } -// jsonValueDeserialize is the recursive deserializer for JSON values. -func jsonValueDeserialize(reader *utils.Reader) (_ JsonValue, err error) { +// JsonValueDeserialize is the recursive deserializer for JSON values. +func JsonValueDeserialize(reader *utils.Reader) (_ JsonValue, err error) { switch JsonValueType(reader.Byte()) { case JsonValueType_Object: items := make([]JsonValueObjectItem, reader.VariableUint()) index := make(map[string]int) for i := range items { items[i].Key = reader.String() - items[i].Value, err = jsonValueDeserialize(reader) + items[i].Value, err = JsonValueDeserialize(reader) if err != nil { return nil, err } @@ -273,7 +275,7 @@ func jsonValueDeserialize(reader *utils.Reader) (_ JsonValue, err error) { case JsonValueType_Array: values := make(JsonValueArray, reader.VariableUint()) for i := range values { - values[i], err = jsonValueDeserialize(reader) + values[i], err = JsonValueDeserialize(reader) if err != nil { return nil, err } @@ -294,8 +296,8 @@ func jsonValueDeserialize(reader *utils.Reader) (_ JsonValue, err error) { } } -// jsonValueFormatter is the recursive formatter for JSON values. -func jsonValueFormatter(sb *strings.Builder, value JsonValue) { +// JsonValueFormatter is the recursive formatter for JSON values. +func JsonValueFormatter(sb *strings.Builder, value JsonValue) { switch value := value.(type) { case JsonValueObject: sb.WriteRune('{') @@ -306,7 +308,7 @@ func jsonValueFormatter(sb *strings.Builder, value JsonValue) { sb.WriteRune('"') sb.WriteString(strings.ReplaceAll(item.Key, `"`, `\"`)) sb.WriteString(`": `) - jsonValueFormatter(sb, item.Value) + JsonValueFormatter(sb, item.Value) } sb.WriteRune('}') case JsonValueArray: @@ -315,7 +317,7 @@ func jsonValueFormatter(sb *strings.Builder, value JsonValue) { if i > 0 { sb.WriteString(", ") } - jsonValueFormatter(sb, item) + JsonValueFormatter(sb, item) } sb.WriteRune(']') case JsonValueString: @@ -334,3 +336,69 @@ func jsonValueFormatter(sb *strings.Builder, value JsonValue) { sb.WriteString(`null`) } } + +// UnmarshalToJsonDocument converts a JSON document byte slice into the actual JSON document. +func UnmarshalToJsonDocument(val []byte) (JsonDocument, error) { + var decoded interface{} + if err := json.Unmarshal(val, &decoded); err != nil { + return JsonDocument{}, err + } + jsonValue, err := ConvertToJsonDocument(decoded) + if err != nil { + return JsonDocument{}, err + } + return JsonDocument{Value: jsonValue}, nil +} + +// ConvertToJsonDocument recursively constructs a valid JsonDocument based on the structures returned by the decoder. +func ConvertToJsonDocument(val interface{}) (JsonValue, error) { + var err error + switch val := val.(type) { + case map[string]interface{}: + keys := utils.GetMapKeys(val) + sort.Slice(keys, func(i, j int) bool { + // Key length is sorted before key contents + if len(keys[i]) < len(keys[j]) { + return true + } else if len(keys[i]) > len(keys[j]) { + return false + } else { + return keys[i] < keys[j] + } + }) + items := make([]JsonValueObjectItem, len(val)) + index := make(map[string]int) + for i, key := range keys { + items[i].Key = key + items[i].Value, err = ConvertToJsonDocument(val[key]) + if err != nil { + return nil, err + } + index[key] = i + } + return JsonValueObject{ + Items: items, + Index: index, + }, nil + case []interface{}: + values := make(JsonValueArray, len(val)) + for i, item := range val { + values[i], err = ConvertToJsonDocument(item) + if err != nil { + return nil, err + } + } + return values, nil + case string: + return JsonValueString(val), nil + case float64: + // TODO: handle this as a proper numeric as float64 is not precise enough + return JsonValueNumber(decimal.NewFromFloat(val)), nil + case bool: + return JsonValueBoolean(val), nil + case nil: + return JsonValueNull(0), nil + default: + return nil, fmt.Errorf("unexpected type while constructing JsonDocument: %T", val) + } +} diff --git a/server/types/jsonb.go b/server/types/jsonb.go index de49f769b5..ed0290b945 100644 --- a/server/types/jsonb.go +++ b/server/types/jsonb.go @@ -15,326 +15,43 @@ package types import ( - "bytes" - "fmt" - "math" - "reflect" - "sort" - "strings" - "unsafe" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" - "github.com/goccy/go-json" "github.com/lib/pq/oid" - "github.com/shopspring/decimal" - - "github.com/dolthub/doltgresql/utils" ) // JsonB is the deserialized and structured version of JSON that deals with JsonDocument. -var JsonB = JsonBType{} - -// JsonBType is the extended type implementation of the PostgreSQL jsonb. -type JsonBType struct{} - -var _ DoltgresType = JsonBType{} - -// Alignment implements the DoltgresType interface. -func (b JsonBType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b JsonBType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_JsonB -} - -// BaseName implements the DoltgresType interface. -func (b JsonBType) BaseName() string { - return "jsonb" -} - -// Category implements the DoltgresType interface. -func (b JsonBType) Category() TypeCategory { - return TypeCategory_UserDefinedTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b JsonBType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b JsonBType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - ab := ac.(JsonDocument) - bb := bc.(JsonDocument) - - return jsonValueCompare(ab.Value, bb.Value), nil -} - -// Convert implements the DoltgresType interface. -func (b JsonBType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case JsonDocument: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b JsonBType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b JsonBType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b JsonBType) GetSerializationID() SerializationID { - return SerializationID_JsonB -} - -// IoInput implements the DoltgresType interface. -func (b JsonBType) IoInput(ctx *sql.Context, input string) (any, error) { - inputBytes := unsafe.Slice(unsafe.StringData(input), len(input)) - if json.Valid(inputBytes) { - doc, err := b.unmarshalToJsonDocument(inputBytes) - return doc, err - } - return nil, fmt.Errorf("invalid input syntax for type json") -} - -// IoOutput implements the DoltgresType interface. -func (b JsonBType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - sb := strings.Builder{} - sb.Grow(256) - jsonValueFormatter(&sb, converted.(JsonDocument).Value) - return sb.String(), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b JsonBType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b JsonBType) IsUnbounded() bool { - return true -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b JsonBType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_Unbounded -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b JsonBType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return math.MaxUint32 -} - -// OID implements the DoltgresType interface. -func (b JsonBType) OID() uint32 { - return uint32(oid.T_jsonb) -} - -// Promote implements the DoltgresType interface. -func (b JsonBType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b JsonBType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - v1Doc, err := b.DeserializeValue(v1) - if err != nil { - return 0, err - } - v2Doc, err := b.DeserializeValue(v2) - if err != nil { - return 0, err - } - return b.Compare(v1Doc, v2Doc) -} - -// SQL implements the DoltgresType interface. -func (b JsonBType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b JsonBType) String() string { - return "jsonb" -} - -// ToArrayType implements the DoltgresType interface. -func (b JsonBType) ToArrayType() DoltgresArrayType { - return JsonBArray -} - -// Type implements the DoltgresType interface. -func (b JsonBType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b JsonBType) ValueType() reflect.Type { - return reflect.TypeOf(JsonDocument{}) -} - -// Zero implements the DoltgresType interface. -func (b JsonBType) Zero() any { - return JsonDocument{Value: JsonValueNull(0)} -} - -// SerializeType implements the DoltgresType interface. -func (b JsonBType) SerializeType() ([]byte, error) { - return SerializationID_JsonB.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b JsonBType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return JsonB, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b JsonBType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - writer := utils.NewWriter(256) - jsonValueSerialize(writer, converted.(JsonDocument).Value) - return writer.Data(), nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b JsonBType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - reader := utils.NewReader(val) - jsonValue, err := jsonValueDeserialize(reader) - return JsonDocument{Value: jsonValue}, err -} - -// unmarshalToJsonDocument converts a JSON document byte slice into the actual JSON document. -func (b JsonBType) unmarshalToJsonDocument(val []byte) (JsonDocument, error) { - var decoded interface{} - if err := json.Unmarshal(val, &decoded); err != nil { - return JsonDocument{}, err - } - jsonValue, err := b.ConvertToJsonDocument(decoded) - if err != nil { - return JsonDocument{}, err - } - return JsonDocument{Value: jsonValue}, nil -} - -// ConvertToJsonDocument recursively constructs a valid JsonDocument based on the structures returned by the decoder. -func (b JsonBType) ConvertToJsonDocument(val interface{}) (JsonValue, error) { - var err error - switch val := val.(type) { - case map[string]interface{}: - keys := utils.GetMapKeys(val) - sort.Slice(keys, func(i, j int) bool { - // Key length is sorted before key contents - if len(keys[i]) < len(keys[j]) { - return true - } else if len(keys[i]) > len(keys[j]) { - return false - } else { - return keys[i] < keys[j] - } - }) - items := make([]JsonValueObjectItem, len(val)) - index := make(map[string]int) - for i, key := range keys { - items[i].Key = key - items[i].Value, err = b.ConvertToJsonDocument(val[key]) - if err != nil { - return nil, err - } - index[key] = i - } - return JsonValueObject{ - Items: items, - Index: index, - }, nil - case []interface{}: - values := make(JsonValueArray, len(val)) - for i, item := range val { - values[i], err = b.ConvertToJsonDocument(item) - if err != nil { - return nil, err - } - } - return values, nil - case string: - return JsonValueString(val), nil - case float64: - // TODO: handle this as a proper numeric as float64 is not precise enough - return JsonValueNumber(decimal.NewFromFloat(val)), nil - case bool: - return JsonValueBoolean(val), nil - case nil: - return JsonValueNull(0), nil - default: - return nil, fmt.Errorf("unexpected type while constructing JsonDocument: %T", val) - } +var JsonB = &DoltgresType{ + OID: uint32(oid.T_jsonb), + Name: "jsonb", + Schema: "pg_catalog", + TypLength: int16(-1), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_UserDefinedTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("jsonb_subscript_handler", oid.T_internal), + Elem: 0, + Array: uint32(oid.T__jsonb), + InputFunc: toFuncID("jsonb_in", oid.T_cstring), + OutputFunc: toFuncID("jsonb_out", oid.T_jsonb), + ReceiveFunc: toFuncID("jsonb_recv", oid.T_internal), + SendFunc: toFuncID("jsonb_send", oid.T_jsonb), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Extended, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("jsonb_cmp", oid.T_jsonb, oid.T_jsonb), } diff --git a/server/types/jsonb_array.go b/server/types/jsonb_array.go index e86734cc72..96ef8ff8ea 100644 --- a/server/types/jsonb_array.go +++ b/server/types/jsonb_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // JsonBArray is the array variant of JsonB. -var JsonBArray = createArrayType(JsonB, SerializationID_JsonBArray, oid.T__jsonb) +var JsonBArray = CreateArrayTypeFromBaseType(JsonB) diff --git a/server/types/name.go b/server/types/name.go index dd85c25921..bdeddda7ff 100644 --- a/server/types/name.go +++ b/server/types/name.go @@ -15,233 +15,46 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" - - "github.com/dolthub/doltgresql/utils" ) -// NameLength is the constant length of Name in Postgres 15. +// NameLength is the constant length of Name in Postgres 15. Represents (NAMEDATALEN-1) const NameLength = 63 // Name is a 63-byte internal type for object names. -var Name = NameType{Length: NameLength} - -// NameType is the extended type implementation of the PostgreSQL name. -type NameType struct { - // Length represents the maximum number of characters that the type may hold. - Length uint32 -} - -var _ DoltgresType = NameType{} - -// Alignment implements the DoltgresType interface. -func (b NameType) Alignment() TypeAlignment { - return TypeAlignment_Char -} - -// BaseID implements the DoltgresType interface. -func (b NameType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Name -} - -// BaseName implements the DoltgresType interface. -func (b NameType) BaseName() string { - return "name" -} - -// Category implements the DoltgresType interface. -func (b NameType) Category() TypeCategory { - return TypeCategory_StringTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b NameType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b NameType) Compare(v1 any, v2 any) (int, error) { - return compareVarChar(b, v1, v2) -} - -// Convert implements the DoltgresType interface. -func (b NameType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case string: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b NameType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b NameType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b NameType) GetSerializationID() SerializationID { - return SerializationID_Name -} - -// IoInput implements the DoltgresType interface. -func (b NameType) IoInput(ctx *sql.Context, input string) (any, error) { - // Name seems to never throw an error, regardless of the context or how long the input is - input, _ = truncateString(input, b.Length) - return input, nil -} - -// IoOutput implements the DoltgresType interface. -func (b NameType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - str, _ := truncateString(converted.(string), b.Length) - return str, nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b NameType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b NameType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b NameType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b NameType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return b.Length * 4 -} - -// OID implements the DoltgresType interface. -func (b NameType) OID() uint32 { - return uint32(oid.T_name) -} - -// Promote implements the DoltgresType interface. -func (b NameType) Promote() sql.Type { - return Name -} - -// SerializedCompare implements the DoltgresType interface. -func (b NameType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - return serializedStringCompare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b NameType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b NameType) String() string { - return "name" -} - -// ToArrayType implements the DoltgresType interface. -func (b NameType) ToArrayType() DoltgresArrayType { - return NameArray -} - -// Type implements the DoltgresType interface. -func (b NameType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b NameType) ValueType() reflect.Type { - return reflect.TypeOf("") -} - -// Zero implements the DoltgresType interface. -func (b NameType) Zero() any { - return "" -} - -// SerializeType implements the DoltgresType interface. -func (b NameType) SerializeType() ([]byte, error) { - t := make([]byte, serializationIDHeaderSize+4) - copy(t, SerializationID_Name.ToByteSlice(0)) - binary.LittleEndian.PutUint32(t[serializationIDHeaderSize:], b.Length) - return t, nil -} - -// deserializeType implements the DoltgresType interface. -func (b NameType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return NameType{ - Length: binary.LittleEndian.Uint32(metadata), - }, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b NameType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - str := converted.(string) - writer := utils.NewWriter(uint64(len(str) + 1)) - writer.String(str) - return writer.Data(), nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b NameType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - reader := utils.NewReader(val) - return reader.String(), nil +var Name = &DoltgresType{ + OID: uint32(oid.T_name), + Name: "name", + Schema: "pg_catalog", + TypLength: int16(64), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_StringTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("raw_array_subscript_handler", oid.T_internal), + Elem: uint32(oid.T_char), + Array: uint32(oid.T__name), + InputFunc: toFuncID("namein", oid.T_cstring), + OutputFunc: toFuncID("nameout", oid.T_name), + ReceiveFunc: toFuncID("namerecv", oid.T_internal), + SendFunc: toFuncID("namesend", oid.T_name), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Char, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 950, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btnamecmp", oid.T_name, oid.T_name), } diff --git a/server/types/name_array.go b/server/types/name_array.go index 15f1d88d42..c46f32901d 100644 --- a/server/types/name_array.go +++ b/server/types/name_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // NameArray is the array variant of Name. -var NameArray = createArrayType(Name, SerializationID_NameArray, oid.T__name) +var NameArray = CreateArrayTypeFromBaseType(Name) diff --git a/server/types/numeric.go b/server/types/numeric.go index 75b8dc4941..defef5b801 100644 --- a/server/types/numeric.go +++ b/server/types/numeric.go @@ -15,18 +15,10 @@ package types import ( - "bytes" - "encoding/binary" "fmt" - "reflect" "strings" "github.com/lib/pq/oid" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/shopspring/decimal" ) @@ -46,252 +38,83 @@ var ( ) // Numeric is a precise and unbounded decimal value. -var Numeric = NumericType{-1, -1} - -// NumericType is the extended type implementation of the PostgreSQL numeric. -type NumericType struct { - // TODO: implement precision and scale - Precision int32 - Scale int32 -} - -var _ DoltgresType = NumericType{} - -// Alignment implements the DoltgresType interface. -func (b NumericType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b NumericType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Numeric -} - -// BaseName implements the DoltgresType interface. -func (b NumericType) BaseName() string { - return "numeric" -} - -// Category implements the DoltgresType interface. -func (b NumericType) Category() TypeCategory { - return TypeCategory_NumericTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b NumericType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b NumericType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(decimal.Decimal) - bb := bc.(decimal.Decimal) - return ab.Cmp(bb), nil -} - -// Convert implements the DoltgresType interface. -func (b NumericType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case decimal.Decimal: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b NumericType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b NumericType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b NumericType) GetSerializationID() SerializationID { - return SerializationID_Numeric -} - -// IoInput implements the DoltgresType interface. -func (b NumericType) IoInput(ctx *sql.Context, input string) (any, error) { - val, err := decimal.NewFromString(strings.TrimSpace(input)) - if err != nil { - return nil, fmt.Errorf("invalid input syntax for type %s: %q", b.String(), input) - } - return val, nil -} - -// IoOutput implements the DoltgresType interface. -func (b NumericType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - dec := converted.(decimal.Decimal) - scale := b.Scale - if scale == -1 { - scale = dec.Exponent() * -1 - } - return dec.StringFixed(scale), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b NumericType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b NumericType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b NumericType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_Unbounded -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b NumericType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 65535 -} - -// OID implements the DoltgresType interface. -func (b NumericType) OID() uint32 { - return uint32(oid.T_numeric) -} - -// Promote implements the DoltgresType interface. -func (b NumericType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b NumericType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - ac, err := b.DeserializeValue(v1) - if err != nil { - return 0, err - } - bc, err := b.DeserializeValue(v2) +var Numeric = &DoltgresType{ + OID: uint32(oid.T_numeric), + Name: "numeric", + Schema: "pg_catalog", + TypLength: int16(-1), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__numeric), + InputFunc: toFuncID("numeric_in", oid.T_cstring, oid.T_oid, oid.T_int4), + OutputFunc: toFuncID("numeric_out", oid.T_numeric), + ReceiveFunc: toFuncID("numeric_recv", oid.T_internal, oid.T_oid, oid.T_int4), + SendFunc: toFuncID("numeric_send", oid.T_numeric), + ModInFunc: toFuncID("numerictypmodin", oid.T__cstring), + ModOutFunc: toFuncID("numerictypmodout", oid.T_int4), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Main, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("numeric_cmp", oid.T_numeric, oid.T_numeric), +} + +// NewNumericTypeWithPrecisionAndScale returns Numeric type with typmod set. +func NewNumericTypeWithPrecisionAndScale(precision, scale int32) (*DoltgresType, error) { + typmod, err := GetTypmodFromNumericPrecisionAndScale(precision, scale) if err != nil { - return 0, err + return nil, err } - ab := ac.(decimal.Decimal) - bb := bc.(decimal.Decimal) - return ab.Cmp(bb), nil + newType := *Numeric.WithAttTypMod(typmod) + return &newType, nil } -// SQL implements the DoltgresType interface. -func (b NumericType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil +// GetTypmodFromNumericPrecisionAndScale takes Numeric type precision and scale and returns the type modifier value. +func GetTypmodFromNumericPrecisionAndScale(precision, scale int32) (int32, error) { + if precision < 1 || precision > 1000 { + return 0, fmt.Errorf("NUMERIC precision %v must be between 1 and 1000", precision) } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err + if scale < -1000 || scale > 1000 { + return 0, fmt.Errorf("NUMERIC scale 20000 must be between -1000 and 1000") } - return sqltypes.MakeTrusted(sqltypes.VarChar, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b NumericType) String() string { - return "numeric" -} - -// ToArrayType implements the DoltgresType interface. -func (b NumericType) ToArrayType() DoltgresArrayType { - return NumericArray -} - -// Type implements the DoltgresType interface. -func (b NumericType) Type() query.Type { - return sqltypes.Decimal -} - -// ValueType implements the DoltgresType interface. -func (b NumericType) ValueType() reflect.Type { - return reflect.TypeOf(decimal.Zero) + return (precision << 16) | scale, nil } -// Zero implements the DoltgresType interface. -func (b NumericType) Zero() any { - return decimal.Zero +// GetPrecisionAndScaleFromTypmod takes Numeric type modifier and returns precision and scale values. +func GetPrecisionAndScaleFromTypmod(typmod int32) (int32, int32) { + scale := typmod & 0xFFFF + precision := (typmod >> 16) & 0xFFFF + return precision, scale } -// SerializeType implements the DoltgresType interface. -func (b NumericType) SerializeType() ([]byte, error) { - t := make([]byte, serializationIDHeaderSize+8) - copy(t, SerializationID_Numeric.ToByteSlice(0)) - binary.LittleEndian.PutUint32(t[serializationIDHeaderSize:], uint32(b.Precision)) - binary.LittleEndian.PutUint32(t[serializationIDHeaderSize+4:], uint32(b.Scale)) - return t, nil -} - -// deserializeType implements the DoltgresType interface. -func (b NumericType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return NumericType{ - Precision: int32(binary.LittleEndian.Uint32(metadata)), - Scale: int32(binary.LittleEndian.Uint32(metadata[4:])), - }, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b NumericType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil +// GetNumericValueWithTypmod returns either given numeric value or truncated or error +// depending on the precision and scale decoded from given type modifier value. +func GetNumericValueWithTypmod(val decimal.Decimal, typmod int32) (decimal.Decimal, error) { + if typmod == -1 { + return val, nil } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - return converted.(decimal.Decimal).MarshalBinary() -} - -// DeserializeValue implements the DoltgresType interface. -func (b NumericType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil + precision, scale := GetPrecisionAndScaleFromTypmod(typmod) + str := val.StringFixed(scale) + parts := strings.Split(str, ".") + if int32(len(parts[0])) > precision-scale && val.IntPart() != 0 { + // TODO: split error message to ERROR and DETAIL + return decimal.Decimal{}, fmt.Errorf("numeric field overflow - A field with precision %v, scale %v must round to an absolute value less than 10^%v", precision, scale, precision-scale) } - retVal := decimal.NewFromInt(0) - err := retVal.UnmarshalBinary(val) - return retVal, err + return decimal.NewFromString(str) } diff --git a/server/types/numeric_array.go b/server/types/numeric_array.go index 6f365b88d8..26dea32deb 100644 --- a/server/types/numeric_array.go +++ b/server/types/numeric_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // NumericArray is the array variant of Numeric. -var NumericArray = createArrayType(Numeric, SerializationID_NumericArray, oid.T__numeric) +var NumericArray = CreateArrayTypeFromBaseType(Numeric) diff --git a/server/types/oid.go b/server/types/oid.go index d8b6e98759..39003ad8b3 100644 --- a/server/types/oid.go +++ b/server/types/oid.go @@ -15,255 +15,43 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "reflect" - "strconv" - "strings" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) -// Oid is a data type used for identifying internal objects. It is implemented as an unsigned 32 bit integer. -var Oid = OidType{} - -// OidType is the extended type implementation of the PostgreSQL oid. -type OidType struct{} - -var _ DoltgresType = OidType{} - -// Alignment implements the DoltgresType interface. -func (b OidType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b OidType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Oid -} - -// BaseName implements the DoltgresType interface. -func (b OidType) BaseName() string { - return "oid" -} - -// Category implements the DoltgresType interface. -func (b OidType) Category() TypeCategory { - return TypeCategory_NumericTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b OidType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b OidType) Compare(v1 any, v2 any) (int, error) { - return compareUint32(b, v1, v2) -} - -// Convert implements the DoltgresType interface. -func (b OidType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case uint32: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b OidType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b OidType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b OidType) GetSerializationID() SerializationID { - return SerializationID_Oid -} - -// IoInput implements the DoltgresType interface. -func (b OidType) IoInput(ctx *sql.Context, input string) (any, error) { - val, err := strconv.ParseInt(strings.TrimSpace(input), 10, 64) - if err != nil { - return nil, fmt.Errorf("invalid input syntax for type %s: %q", b.String(), input) - } - // Note: This minimum is different (-4294967295) for Postgres 15.4 compiled by Visual C++ - if val > MaxUint32 || val < MinInt32 { - return nil, fmt.Errorf("value %q is out of range for type %s", input, b.String()) - } - return uint32(val), nil -} - -// IoOutput implements the DoltgresType interface. -func (b OidType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return strconv.FormatUint(uint64(converted.(uint32)), 10), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b OidType) IsPreferredType() bool { - return true -} - -// IsUnbounded implements the DoltgresType interface. -func (b OidType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b OidType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b OidType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 4 -} - -// OID implements the DoltgresType interface. -func (b OidType) OID() uint32 { - return uint32(oid.T_oid) -} - -// Promote implements the DoltgresType interface. -func (b OidType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b OidType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b OidType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b OidType) String() string { - return "oid" -} - -// ToArrayType implements the DoltgresType interface. -func (b OidType) ToArrayType() DoltgresArrayType { - return OidArray -} - -// Type implements the DoltgresType interface. -func (b OidType) Type() query.Type { - return sqltypes.Uint32 -} - -// ValueType implements the DoltgresType interface. -func (b OidType) ValueType() reflect.Type { - return reflect.TypeOf(uint32(0)) -} - -// Zero implements the DoltgresType interface. -func (b OidType) Zero() any { - return uint32(0) -} - -// SerializeType implements the DoltgresType interface. -func (b OidType) SerializeType() ([]byte, error) { - return SerializationID_Oid.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b OidType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Oid, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b OidType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - retVal := make([]byte, 4) - binary.BigEndian.PutUint32(retVal, converted.(uint32)) - return retVal, nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b OidType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - return binary.BigEndian.Uint32(val), nil -} - -func compareUint32(b DoltgresType, v1, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(uint32) - bb := bc.(uint32) - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } +// Oid is a data type used for identifying internal objects. It is implemented as an unsigned 32-bit integer. +var Oid = &DoltgresType{ + OID: uint32(oid.T_oid), + Name: "oid", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: true, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__oid), + InputFunc: toFuncID("oidin", oid.T_cstring), + OutputFunc: toFuncID("oidout", oid.T_oid), + ReceiveFunc: toFuncID("oidrecv", oid.T_internal), + SendFunc: toFuncID("oidsend", oid.T_oid), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("btoidcmp", oid.T_oid, oid.T_oid), } diff --git a/server/types/oid/iterate.go b/server/types/oid/iterate.go index 4916bb20a8..25a8500ad3 100644 --- a/server/types/oid/iterate.go +++ b/server/types/oid/iterate.go @@ -123,11 +123,11 @@ type ItemTable struct { Item sql.Table } -// ItemType contains the relevant information to pass to the Type callback. +// ItemType contains the relevant information to pass to the DoltgresType callback. type ItemType struct { // TODO: add Index when we add custom types OID uint32 - Item pgtypes.DoltgresType + Item *pgtypes.DoltgresType } // ItemView contains the relevant information to pass to the View callback. @@ -161,7 +161,7 @@ func IterateDatabase(ctx *sql.Context, database string, callbacks Callbacks) err // Then we'll iterate over everything that is contained within a schema if currentSchemaDatabase, ok := currentDatabase.(sql.SchemaDatabase); ok && callbacks.iteratesOverSchemas() { - // Load and sort all of the schemas by name ascending + // Load and sort all schemas by name ascending schemas, err := currentSchemaDatabase.AllSchemas(ctx) if err != nil { return err @@ -214,7 +214,7 @@ func iterateFunctions(ctx *sql.Context, callbacks Callbacks) error { // iterateTypes is called by IterateCurrentDatabase to handle types func iterateTypes(ctx *sql.Context, callbacks Callbacks) error { // We only iterate over the types that are present in the pg_type table. - // This means that we ignore the schema if one is given and it's not equal to "pg_catalog". + // This means that we ignore the schema if one is given and not equal to "pg_catalog". // If no schemas were given, then we'll automatically look for the types in "pg_catalog". if len(callbacks.SearchSchemas) > 0 { containsPgCatalog := false @@ -230,17 +230,15 @@ func iterateTypes(ctx *sql.Context, callbacks Callbacks) error { } // this gets all built-in types for _, t := range pgtypes.GetAllTypes() { - if t.BaseID().HasUniqueOID() { - cont, err := callbacks.Type(ctx, ItemType{ - OID: t.OID(), - Item: t, - }) - if err != nil { - return err - } - if !cont { - return nil - } + cont, err := callbacks.Type(ctx, ItemType{ + OID: t.OID, + Item: t, + }) + if err != nil { + return err + } + if !cont { + return nil } } // TODO: add domain and custom types when supported @@ -789,7 +787,7 @@ func runTable(ctx *sql.Context, oid uint32, callbacks Callbacks, itemSchema Item // runType is called by RunCallback to handle types within Section_BuiltIn. func runType(ctx *sql.Context, toid uint32, callbacks Callbacks) error { - if t := pgtypes.GetTypeByOID(toid); t != nil { + if t := pgtypes.GetTypeByOID(toid); !t.IsEmptyType() { itemType := ItemType{ OID: toid, Item: t, diff --git a/server/types/oid/regtype.go b/server/types/oid/regtype.go index a2f8dac55d..1a0eead3db 100644 --- a/server/types/oid/regtype.go +++ b/server/types/oid/regtype.go @@ -60,7 +60,11 @@ func regtype_IoInput(ctx *sql.Context, input string) (uint32, error) { resultOid := uint32(0) err = IterateCurrentDatabase(ctx, Callbacks{ Type: func(ctx *sql.Context, typ ItemType) (cont bool, err error) { - if typeName == typ.Item.String() || typeName == typ.Item.BaseName() || (typeName == "char" && typ.Item.BaseName() == "bpchar") { + tin := typ.Item.Name + if tin == "char" { + tin = `"char"` + } + if typeName == typ.Item.String() || typeName == tin || (typeName == "char" && tin == "bpchar") { resultOid = typ.OID return false, nil } else if t, ok := types.OidToType[oid.Oid(typ.OID)]; ok && typeName == t.SQLStandardName() { diff --git a/server/types/oid_array.go b/server/types/oid_array.go index 2df88452f8..e62c7ba497 100644 --- a/server/types/oid_array.go +++ b/server/types/oid_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // OidArray is the array variant of Oid. -var OidArray = createArrayType(Oid, SerializationID_OidArray, oid.T__oid) +var OidArray = CreateArrayTypeFromBaseType(Oid) diff --git a/server/types/regclass.go b/server/types/regclass.go index 3766701d86..60b28145f3 100644 --- a/server/types/regclass.go +++ b/server/types/regclass.go @@ -15,204 +15,50 @@ package types import ( - "bytes" - "fmt" - "reflect" - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Regclass is the OID type for finding items in pg_class. -var Regclass = RegclassType{} - -// RegclassType is the extended type implementation of the PostgreSQL regclass. -type RegclassType struct{} - -var _ DoltgresType = RegclassType{} - -// Alignment implements the DoltgresType interface. -func (b RegclassType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b RegclassType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Regclass -} - -// BaseName implements the DoltgresType interface. -func (b RegclassType) BaseName() string { - return "regclass" -} - -// Category implements the DoltgresType interface. -func (b RegclassType) Category() TypeCategory { - return TypeCategory_NumericTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b RegclassType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b RegclassType) Compare(v1 any, v2 any) (int, error) { - return OidType{}.Compare(v1, v2) -} - -// Convert implements the DoltgresType interface. -func (b RegclassType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case uint32: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b RegclassType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b RegclassType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b RegclassType) GetSerializationID() SerializationID { - return SerializationID_Invalid +var Regclass = &DoltgresType{ + OID: uint32(oid.T_regclass), + Name: "regclass", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__regclass), + InputFunc: toFuncID("regclassin", oid.T_cstring), + OutputFunc: toFuncID("regclassout", oid.T_regclass), + ReceiveFunc: toFuncID("regclassrecv", oid.T_internal), + SendFunc: toFuncID("regclasssend", oid.T_regclass), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), } // Regclass_IoInput is the implementation for IoInput that is being set from another package to avoid circular dependencies. var Regclass_IoInput func(ctx *sql.Context, input string) (uint32, error) -// IoInput implements the DoltgresType interface. -func (b RegclassType) IoInput(ctx *sql.Context, input string) (any, error) { - return Regclass_IoInput(ctx, input) -} - // Regclass_IoOutput is the implementation for IoOutput that is being set from another package to avoid circular dependencies. var Regclass_IoOutput func(ctx *sql.Context, oid uint32) (string, error) - -// IoOutput implements the DoltgresType interface. -func (b RegclassType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return Regclass_IoOutput(ctx, converted.(uint32)) -} - -// IsPreferredType implements the DoltgresType interface. -func (b RegclassType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b RegclassType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b RegclassType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b RegclassType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 4 -} - -// OID implements the DoltgresType interface. -func (b RegclassType) OID() uint32 { - return uint32(oid.T_regclass) -} - -// Promote implements the DoltgresType interface. -func (b RegclassType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b RegclassType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b RegclassType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b RegclassType) String() string { - return "regclass" -} - -// ToArrayType implements the DoltgresType interface. -func (b RegclassType) ToArrayType() DoltgresArrayType { - return RegclassArray -} - -// Type implements the DoltgresType interface. -func (b RegclassType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b RegclassType) ValueType() reflect.Type { - return reflect.TypeOf(uint32(0)) -} - -// Zero implements the DoltgresType interface. -func (b RegclassType) Zero() any { - return uint32(0) -} - -// SerializeType implements the DoltgresType interface. -func (b RegclassType) SerializeType() ([]byte, error) { - return nil, fmt.Errorf("%s cannot be serialized", b.String()) -} - -// deserializeType implements the DoltgresType interface. -func (b RegclassType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - return nil, fmt.Errorf("%s cannot be deserialized", b.String()) -} - -// SerializeValue implements the DoltgresType interface. -func (b RegclassType) SerializeValue(val any) ([]byte, error) { - return nil, fmt.Errorf("%s cannot serialize values", b.String()) -} - -// DeserializeValue implements the DoltgresType interface. -func (b RegclassType) DeserializeValue(val []byte) (any, error) { - return nil, fmt.Errorf("%s cannot deserialize values", b.String()) -} diff --git a/server/types/regclass_array.go b/server/types/regclass_array.go index 8b9520fc9a..02ac6e2b77 100644 --- a/server/types/regclass_array.go +++ b/server/types/regclass_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // RegclassArray is the array variant of Regclass. -var RegclassArray = createArrayType(Regclass, SerializationID_Invalid, oid.T__regclass) +var RegclassArray = CreateArrayTypeFromBaseType(Regclass) diff --git a/server/types/regproc.go b/server/types/regproc.go index 8d1f1656fe..e528bee4ba 100644 --- a/server/types/regproc.go +++ b/server/types/regproc.go @@ -15,204 +15,50 @@ package types import ( - "bytes" - "fmt" - "reflect" - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Regproc is the OID type for finding function names. -var Regproc = RegprocType{} - -// RegprocType is the extended type implementation of the PostgreSQL regproc. -type RegprocType struct{} - -var _ DoltgresType = RegprocType{} - -// Alignment implements the DoltgresType interface. -func (b RegprocType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b RegprocType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Regproc -} - -// BaseName implements the DoltgresType interface. -func (b RegprocType) BaseName() string { - return "regproc" -} - -// Category implements the DoltgresType interface. -func (b RegprocType) Category() TypeCategory { - return TypeCategory_NumericTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b RegprocType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b RegprocType) Compare(v1 any, v2 any) (int, error) { - return OidType{}.Compare(v1, v2) -} - -// Convert implements the DoltgresType interface. -func (b RegprocType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case uint32: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b RegprocType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b RegprocType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b RegprocType) GetSerializationID() SerializationID { - return SerializationID_Invalid +var Regproc = &DoltgresType{ + OID: uint32(oid.T_regproc), + Name: "regproc", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__regproc), + InputFunc: toFuncID("regprocin", oid.T_cstring), + OutputFunc: toFuncID("regprocout", oid.T_regproc), + ReceiveFunc: toFuncID("regprocrecv", oid.T_internal), + SendFunc: toFuncID("regprocsend", oid.T_regproc), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), } // Regproc_IoInput is the implementation for IoInput that is being set from another package to avoid circular dependencies. var Regproc_IoInput func(ctx *sql.Context, input string) (uint32, error) -// IoInput implements the DoltgresType interface. -func (b RegprocType) IoInput(ctx *sql.Context, input string) (any, error) { - return Regproc_IoInput(ctx, input) -} - // Regproc_IoOutput is the implementation for IoOutput that is being set from another package to avoid circular dependencies. var Regproc_IoOutput func(ctx *sql.Context, oid uint32) (string, error) - -// IoOutput implements the DoltgresType interface. -func (b RegprocType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return Regproc_IoOutput(ctx, converted.(uint32)) -} - -// IsPreferredType implements the DoltgresType interface. -func (b RegprocType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b RegprocType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b RegprocType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b RegprocType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 4 -} - -// OID implements the DoltgresType interface. -func (b RegprocType) OID() uint32 { - return uint32(oid.T_regproc) -} - -// Promote implements the DoltgresType interface. -func (b RegprocType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b RegprocType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b RegprocType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b RegprocType) String() string { - return "regproc" -} - -// ToArrayType implements the DoltgresType interface. -func (b RegprocType) ToArrayType() DoltgresArrayType { - return RegprocArray -} - -// Type implements the DoltgresType interface. -func (b RegprocType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b RegprocType) ValueType() reflect.Type { - return reflect.TypeOf(uint32(0)) -} - -// Zero implements the DoltgresType interface. -func (b RegprocType) Zero() any { - return uint32(0) -} - -// SerializeType implements the DoltgresType interface. -func (b RegprocType) SerializeType() ([]byte, error) { - return nil, fmt.Errorf("%s cannot be serialized", b.String()) -} - -// deserializeType implements the DoltgresType interface. -func (b RegprocType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - return nil, fmt.Errorf("%s cannot be deserialized", b.String()) -} - -// SerializeValue implements the DoltgresType interface. -func (b RegprocType) SerializeValue(val any) ([]byte, error) { - return nil, fmt.Errorf("%s cannot serialize values", b.String()) -} - -// DeserializeValue implements the DoltgresType interface. -func (b RegprocType) DeserializeValue(val []byte) (any, error) { - return nil, fmt.Errorf("%s cannot deserialize values", b.String()) -} diff --git a/server/types/regproc_array.go b/server/types/regproc_array.go index e2a45b88dd..b2973e2e3b 100644 --- a/server/types/regproc_array.go +++ b/server/types/regproc_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // RegprocArray is the array variant of Regproc. -var RegprocArray = createArrayType(Regproc, SerializationID_Invalid, oid.T__regproc) +var RegprocArray = CreateArrayTypeFromBaseType(Regproc) diff --git a/server/types/regtype.go b/server/types/regtype.go index d3e8e11d16..9846bca99f 100644 --- a/server/types/regtype.go +++ b/server/types/regtype.go @@ -15,204 +15,50 @@ package types import ( - "bytes" - "fmt" - "reflect" - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Regtype is the OID type for finding items in pg_type. -var Regtype = RegtypeType{} - -// RegtypeType is the extended type implementation of the PostgreSQL regtype. -type RegtypeType struct{} - -var _ DoltgresType = RegtypeType{} - -// Alignment implements the DoltgresType interface. -func (b RegtypeType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b RegtypeType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Regtype -} - -// BaseName implements the DoltgresType interface. -func (b RegtypeType) BaseName() string { - return "regtype" -} - -// Category implements the DoltgresType interface. -func (b RegtypeType) Category() TypeCategory { - return TypeCategory_NumericTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b RegtypeType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b RegtypeType) Compare(v1 any, v2 any) (int, error) { - return OidType{}.Compare(v1, v2) -} - -// Convert implements the DoltgresType interface. -func (b RegtypeType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case uint32: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b RegtypeType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b RegtypeType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b RegtypeType) GetSerializationID() SerializationID { - return SerializationID_Invalid +var Regtype = &DoltgresType{ + OID: uint32(oid.T_regtype), + Name: "regtype", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_NumericTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__regtype), + InputFunc: toFuncID("regtypein", oid.T_cstring), + OutputFunc: toFuncID("regtypeout", oid.T_regtype), + ReceiveFunc: toFuncID("regtyperecv", oid.T_internal), + SendFunc: toFuncID("regtypesend", oid.T_regtype), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), } // Regtype_IoInput is the implementation for IoInput that is being set from another package to avoid circular dependencies. var Regtype_IoInput func(ctx *sql.Context, input string) (uint32, error) -// IoInput implements the DoltgresType interface. -func (b RegtypeType) IoInput(ctx *sql.Context, input string) (any, error) { - return Regtype_IoInput(ctx, input) -} - // Regtype_IoOutput is the implementation for IoOutput that is being set from another package to avoid circular dependencies. var Regtype_IoOutput func(ctx *sql.Context, oid uint32) (string, error) - -// IoOutput implements the DoltgresType interface. -func (b RegtypeType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return Regtype_IoOutput(ctx, converted.(uint32)) -} - -// IsPreferredType implements the DoltgresType interface. -func (b RegtypeType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b RegtypeType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b RegtypeType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b RegtypeType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 4 -} - -// OID implements the DoltgresType interface. -func (b RegtypeType) OID() uint32 { - return uint32(oid.T_regtype) -} - -// Promote implements the DoltgresType interface. -func (b RegtypeType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b RegtypeType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b RegtypeType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b RegtypeType) String() string { - return "regtype" -} - -// ToArrayType implements the DoltgresType interface. -func (b RegtypeType) ToArrayType() DoltgresArrayType { - return RegtypeArray -} - -// Type implements the DoltgresType interface. -func (b RegtypeType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b RegtypeType) ValueType() reflect.Type { - return reflect.TypeOf(uint32(0)) -} - -// Zero implements the DoltgresType interface. -func (b RegtypeType) Zero() any { - return uint32(0) -} - -// SerializeType implements the DoltgresType interface. -func (b RegtypeType) SerializeType() ([]byte, error) { - return nil, fmt.Errorf("%s cannot be serialized", b.String()) -} - -// deserializeType implements the DoltgresType interface. -func (b RegtypeType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - return nil, fmt.Errorf("%s cannot be deserialized", b.String()) -} - -// SerializeValue implements the DoltgresType interface. -func (b RegtypeType) SerializeValue(val any) ([]byte, error) { - return nil, fmt.Errorf("%s cannot serialize values", b.String()) -} - -// DeserializeValue implements the DoltgresType interface. -func (b RegtypeType) DeserializeValue(val []byte) (any, error) { - return nil, fmt.Errorf("%s cannot deserialize values", b.String()) -} diff --git a/server/types/regtype_array.go b/server/types/regtype_array.go index 5b8e34669e..5deae25429 100644 --- a/server/types/regtype_array.go +++ b/server/types/regtype_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // RegtypeArray is the array variant of Regtype. -var RegtypeArray = createArrayType(Regtype, SerializationID_Invalid, oid.T__regtype) +var RegtypeArray = CreateArrayTypeFromBaseType(Regtype) diff --git a/server/types/resolvable.go b/server/types/resolvable.go deleted file mode 100644 index d106816bd8..0000000000 --- a/server/types/resolvable.go +++ /dev/null @@ -1,182 +0,0 @@ -// Copyright 2024 Dolthub, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package types - -import ( - "fmt" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" - - "github.com/dolthub/doltgresql/postgres/parser/sem/tree" -) - -// ResolvableType represents any non-built-in type -// that needs resolution at analyzer stage. -// It is used for domain types, and it can be used -// for other user-defined types we don't support yet. -type ResolvableType struct { - Typ tree.ResolvableTypeReference -} - -var _ DoltgresType = ResolvableType{} - -// Alignment implements the DoltgresType interface. -func (b ResolvableType) Alignment() TypeAlignment { - panic("ResolvableType is a placeholder type, but Alignment() was called") -} - -// BaseID implements the DoltgresType interface. -func (b ResolvableType) BaseID() DoltgresTypeBaseID { - panic("ResolvableType is a placeholder type, but BaseID() was called") -} - -// BaseName implements the DoltgresType interface. -func (b ResolvableType) BaseName() string { - return fmt.Sprintf("ResolvableType(%s)", b.Typ.SQLString()) -} - -// Category implements the DoltgresType interface. -func (b ResolvableType) Category() TypeCategory { - panic("ResolvableType is a placeholder type, but Category() was called") -} - -// CollationCoercibility implements the DoltgresType interface. -func (b ResolvableType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - panic("ResolvableType is a placeholder type, but CollationCoercibility() was called") -} - -// Compare implements the DoltgresType interface. -func (b ResolvableType) Compare(v1 any, v2 any) (int, error) { - panic("ResolvableType is a placeholder type, but Compare() was called") -} - -// Convert implements the DoltgresType interface. -func (b ResolvableType) Convert(val any) (any, sql.ConvertInRange, error) { - panic("ResolvableType is a placeholder type, but Convert() was called") -} - -// Equals implements the DoltgresType interface. -func (b ResolvableType) Equals(otherType sql.Type) bool { - panic("ResolvableType is a placeholder type, but Equals() was called") -} - -// FormatValue implements the DoltgresType interface. -func (b ResolvableType) FormatValue(val any) (string, error) { - panic("ResolvableType is a placeholder type, but FormatValue() was called") -} - -// GetSerializationID implements the DoltgresType interface. -func (b ResolvableType) GetSerializationID() SerializationID { - panic("ResolvableType is a placeholder type, but GetSerializationID() was called") -} - -// IoInput implements the DoltgresType interface. -func (b ResolvableType) IoInput(ctx *sql.Context, input string) (any, error) { - panic("ResolvableType is a placeholder type, but IoInput() was called") -} - -// IoOutput implements the DoltgresType interface. -func (b ResolvableType) IoOutput(ctx *sql.Context, output any) (string, error) { - panic("ResolvableType is a placeholder type, but IoOutput() was called") -} - -// IsPreferredType implements the DoltgresType interface. -func (b ResolvableType) IsPreferredType() bool { - panic("ResolvableType is a placeholder type, but IsPreferredType() was called") -} - -// IsUnbounded implements the DoltgresType interface. -func (b ResolvableType) IsUnbounded() bool { - panic("ResolvableType is a placeholder type, but IsUnbounded() was called") -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b ResolvableType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - panic("ResolvableType is a placeholder type, but MaxSerializedWidth() was called") -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b ResolvableType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - panic("ResolvableType is a placeholder type, but MaxTextResponseByteLength() was called") -} - -// OID implements the DoltgresType interface. -func (b ResolvableType) OID() uint32 { - panic("ResolvableType is a placeholder type, but OID() was called") -} - -// Promote implements the DoltgresType interface. -func (b ResolvableType) Promote() sql.Type { - panic("ResolvableType is a placeholder type, but Promote() was called") -} - -// SerializedCompare implements the DoltgresType interface. -func (b ResolvableType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - panic("ResolvableType is a placeholder type, but SerializedCompare() was called") -} - -// SQL implements the DoltgresType interface. -func (b ResolvableType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - panic("ResolvableType is a placeholder type, but SQL() was called") -} - -// String implements the DoltgresType interface. -func (b ResolvableType) String() string { - return fmt.Sprintf("ResolvableType(%s)", b.Typ.SQLString()) -} - -// ToArrayType implements the DoltgresType interface. -func (b ResolvableType) ToArrayType() DoltgresArrayType { - panic("ResolvableType is a placeholder type, but ToArrayType() was called") -} - -// Type implements the DoltgresType interface. -func (b ResolvableType) Type() query.Type { - panic("ResolvableType is a placeholder type, but Type() was called") -} - -// ValueType implements the DoltgresType interface. -func (b ResolvableType) ValueType() reflect.Type { - panic("ResolvableType is a placeholder type, but ValueType() was called") -} - -// Zero implements the DoltgresType interface. -func (b ResolvableType) Zero() any { - panic("ResolvableType is a placeholder type, but Zero() was called") -} - -// SerializeType implements the DoltgresType interface. -func (b ResolvableType) SerializeType() ([]byte, error) { - panic("ResolvableType is a placeholder type, but SerializeType() was called") -} - -// deserializeType implements the DoltgresType interface. -func (b ResolvableType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - panic("ResolvableType is a placeholder type, but deserializeType() was called") -} - -// SerializeValue implements the DoltgresType interface. -func (b ResolvableType) SerializeValue(val any) ([]byte, error) { - panic("ResolvableType is a placeholder type, but SerializeValue() was called") -} - -// DeserializeValue implements the DoltgresType interface. -func (b ResolvableType) DeserializeValue(val []byte) (any, error) { - panic("ResolvableType is a placeholder type, but DeserializeValue() was called") -} diff --git a/server/types/serialization.go b/server/types/serialization.go index d12879c7ee..cfad486c81 100644 --- a/server/types/serialization.go +++ b/server/types/serialization.go @@ -15,197 +15,146 @@ package types import ( - "encoding/binary" "fmt" + "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/types" -) - -// SerializationID is an ID unique to Doltgres that can uniquely identify any type for the purposes of Serialization. -// These are different from OIDs, as they are unchanging and unique. If we need to add a new type that does not already -// have a pre-defined ID, then it must use a new number that has never been previously used. -type SerializationID uint16 -// These are declared as constant numbers to signify their intent. Under no circumstances should we use iota, as that -// runs the risk of an accidental reordering potentially causing data loss. In addition, numbers for pre-existing IDs -// should never be changed. -const ( - SerializationID_Invalid SerializationID = 0 - SerializationID_Bit SerializationID = 1 - SerializationID_BitArray SerializationID = 2 - SerializationID_Bool SerializationID = 3 - SerializationID_BoolArray SerializationID = 4 - SerializationID_Box SerializationID = 5 - SerializationID_BoxArray SerializationID = 6 - SerializationID_Bytea SerializationID = 7 - SerializationID_ByteaArray SerializationID = 8 - SerializationID_Char SerializationID = 9 - SerializationID_CharArray SerializationID = 10 - SerializationID_Cidr SerializationID = 11 - SerializationID_CidrArray SerializationID = 12 - SerializationID_Circle SerializationID = 13 - SerializationID_CircleArray SerializationID = 14 - SerializationID_Date SerializationID = 15 - SerializationID_DateArray SerializationID = 16 - SerializationID_DateMultirange SerializationID = 17 - SerializationID_DateRange SerializationID = 18 - SerializationID_Enum SerializationID = 19 - SerializationID_EnumArray SerializationID = 20 - SerializationID_Float32 SerializationID = 21 - SerializationID_Float32Array SerializationID = 22 - SerializationID_Float64 SerializationID = 23 - SerializationID_Float64Array SerializationID = 24 - SerializationID_Inet SerializationID = 25 - SerializationID_InetArray SerializationID = 26 - SerializationID_Int16 SerializationID = 27 - SerializationID_Int16Array SerializationID = 28 - SerializationID_Int32 SerializationID = 29 - SerializationID_Int32Array SerializationID = 30 - SerializationID_Int32Multirange SerializationID = 31 - SerializationID_Int32Range SerializationID = 32 - SerializationID_Int64 SerializationID = 33 - SerializationID_Int64Array SerializationID = 34 - SerializationID_Int64Multirange SerializationID = 35 - SerializationID_Int64Range SerializationID = 36 - SerializationID_Interval SerializationID = 37 - SerializationID_IntervalArray SerializationID = 38 - SerializationID_Json SerializationID = 39 - SerializationID_JsonArray SerializationID = 40 - SerializationID_JsonB SerializationID = 41 - SerializationID_JsonBArray SerializationID = 42 - SerializationID_Line SerializationID = 43 - SerializationID_LineArray SerializationID = 44 - SerializationID_LineSegment SerializationID = 45 - SerializationID_LineSegmentArray SerializationID = 46 - SerializationID_MacAddress SerializationID = 47 - SerializationID_MacAddress8 SerializationID = 48 - SerializationID_MacAddress8Array SerializationID = 49 - SerializationID_MacAddressArray SerializationID = 50 - SerializationID_Money SerializationID = 51 - SerializationID_MoneyArray SerializationID = 52 - SerializationID_Null SerializationID = 53 - SerializationID_Numeric SerializationID = 54 - SerializationID_NumericArray SerializationID = 55 - SerializationID_NumericMultirange SerializationID = 56 - SerializationID_NumericRange SerializationID = 57 - SerializationID_Path SerializationID = 58 - SerializationID_PathArray SerializationID = 59 - SerializationID_Point SerializationID = 60 - SerializationID_PointArray SerializationID = 61 - SerializationID_Polygon SerializationID = 62 - SerializationID_PolygonArray SerializationID = 63 - SerializationID_Text SerializationID = 64 - SerializationID_TextArray SerializationID = 65 - SerializationID_Time SerializationID = 66 - SerializationID_TimeArray SerializationID = 67 - SerializationID_TimeTZ SerializationID = 68 - SerializationID_TimeTZArray SerializationID = 69 - SerializationID_Timestamp SerializationID = 70 - SerializationID_TimestampArray SerializationID = 71 - SerializationID_TimestampMultirange SerializationID = 72 - SerializationID_TimestampRange SerializationID = 73 - SerializationID_TimestampTZ SerializationID = 74 - SerializationID_TimestampTZArray SerializationID = 75 - SerializationID_TimestampTZMultirange SerializationID = 76 - SerializationID_TimestampTZRange SerializationID = 77 - SerializationID_TsQuery SerializationID = 78 - SerializationID_TsQueryArray SerializationID = 79 - SerializationID_TsVector SerializationID = 80 - SerializationID_TsVectorArray SerializationID = 81 - SerializationID_Uuid SerializationID = 82 - SerializationID_UuidArray SerializationID = 83 - SerializationID_VarBit SerializationID = 84 - SerializationID_VarBitArray SerializationID = 85 - SerializationID_VarChar SerializationID = 86 - SerializationID_VarCharArray SerializationID = 87 - SerializationID_Xml SerializationID = 88 - SerializationID_XmlArray SerializationID = 89 - SerializationID_Name SerializationID = 90 - SerializationID_NameArray SerializationID = 91 - SerializationID_Oid SerializationID = 92 - SerializationID_OidArray SerializationID = 93 - SerializationID_Xid SerializationID = 94 - SerializationID_XidArray SerializationID = 95 - SerializationID_InternalChar SerializationID = 96 - SerializationID_InternalCharArray SerializationID = 97 - SerializationId_Domain SerializationID = 98 + "github.com/dolthub/doltgresql/utils" ) -// serializationIDToType is a map from each SerializationID to its matching DoltgresType. -var serializationIDToType = map[SerializationID]DoltgresType{} - // init sets the serialization and deserialization functions. func init() { types.SetExtendedTypeSerializers(SerializeType, DeserializeType) - for _, t := range typesFromBaseID { - sID := t.GetSerializationID() - if sID == SerializationID_Invalid { - continue - } - if _, ok := serializationIDToType[sID]; ok { - panic("duplicate serialization IDs in use") - } - serializationIDToType[sID] = t - } - serializationIDToType[SerializationId_Domain] = DomainType{} } // SerializeType is able to serialize the given extended type into a byte slice. All extended types will be defined // by DoltgreSQL. func SerializeType(extendedType types.ExtendedType) ([]byte, error) { - if doltgresType, ok := extendedType.(DoltgresType); ok { - return doltgresType.SerializeType() + if doltgresType, ok := extendedType.(*DoltgresType); ok { + return doltgresType.Serialize(), nil } return nil, fmt.Errorf("unknown type to serialize") } -// MustSerializeType internally calls SerializeType and panics on error. In general, panics should only occur when a -// type has not yet had its Serialization implemented yet. -func MustSerializeType(extendedType types.ExtendedType) []byte { - // MustSerializeType is often used to efficiently compare any two types, so we'll make a special exception for types - // that cannot be normally serialized. This is okay since these types cannot be deserialized, preventing them from - // being used outside of comparisons. - switch extendedType.(type) { - case AnyArrayType: - return []byte{0} - case UnknownType: - return []byte{1} - } - serializedType, err := SerializeType(extendedType) - if err != nil { - panic(err) - } - return serializedType -} - // DeserializeType is able to deserialize the given serialized type into an appropriate extended type. All extended // types will be defined by DoltgreSQL. func DeserializeType(serializedType []byte) (types.ExtendedType, error) { - if len(serializedType) < serializationIDHeaderSize { - return nil, fmt.Errorf("cannot deserialize an empty type") + if len(serializedType) == 0 { + return nil, fmt.Errorf("deserializing empty type data") } - serializationID, version := SerializationIDFromBytes(serializedType) - targetType, ok := serializationIDToType[serializationID] - if !ok { - return nil, fmt.Errorf("serialization ID %d does not have a matching type for deserialization", serializationID) + + typ := &DoltgresType{} + reader := utils.NewReader(serializedType) + version := reader.VariableUint() + if version != 0 { + return nil, fmt.Errorf("version %d of types is not supported, please upgrade the server", version) } - return targetType.deserializeType(version, serializedType[serializationIDHeaderSize:]) -} -// serializationIDHeaderSize is the size of the header that applies to all serialization IDs. -const serializationIDHeaderSize = 4 + typ.OID = reader.Uint32() + typ.Name = reader.String() + typ.Schema = reader.String() + typ.Owner = reader.String() + typ.TypLength = reader.Int16() + typ.PassedByVal = reader.Bool() + typ.TypType = TypeType(reader.String()) + typ.TypCategory = TypeCategory(reader.String()) + typ.IsPreferred = reader.Bool() + typ.IsDefined = reader.Bool() + typ.Delimiter = reader.String() + typ.RelID = reader.Uint32() + typ.SubscriptFunc = globalFunctionRegistry.StringToID(reader.String()) + typ.Elem = reader.Uint32() + typ.Array = reader.Uint32() + typ.InputFunc = globalFunctionRegistry.StringToID(reader.String()) + typ.OutputFunc = globalFunctionRegistry.StringToID(reader.String()) + typ.ReceiveFunc = globalFunctionRegistry.StringToID(reader.String()) + typ.SendFunc = globalFunctionRegistry.StringToID(reader.String()) + typ.ModInFunc = globalFunctionRegistry.StringToID(reader.String()) + typ.ModOutFunc = globalFunctionRegistry.StringToID(reader.String()) + typ.AnalyzeFunc = globalFunctionRegistry.StringToID(reader.String()) + typ.Align = TypeAlignment(reader.String()) + typ.Storage = TypeStorage(reader.String()) + typ.NotNull = reader.Bool() + typ.BaseTypeOID = reader.Uint32() + typ.TypMod = reader.Int32() + typ.NDims = reader.Int32() + typ.TypCollation = reader.Uint32() + typ.DefaulBin = reader.String() + typ.Default = reader.String() + numOfAcl := reader.VariableUint() + for k := uint64(0); k < numOfAcl; k++ { + ac := reader.String() + typ.Acl = append(typ.Acl, ac) + } + numOfChecks := reader.VariableUint() + for k := uint64(0); k < numOfChecks; k++ { + checkName := reader.String() + checkExpr := reader.String() + typ.Checks = append(typ.Checks, &sql.CheckDefinition{ + Name: checkName, + CheckExpression: checkExpr, + Enforced: true, + }) + } + typ.attTypMod = reader.Int32() + typ.CompareFunc = globalFunctionRegistry.StringToID(reader.String()) + typ.InternalName = reader.String() + if !reader.IsEmpty() { + return nil, fmt.Errorf("extra data found while deserializing type %s", typ.Name) + } -// ToByteSlice returns the ID as a byte slice. -func (id SerializationID) ToByteSlice(version uint16) []byte { - b := make([]byte, serializationIDHeaderSize) - binary.LittleEndian.PutUint16(b, uint16(id)) - binary.LittleEndian.PutUint16(b[2:], version) - return b + // Return the deserialized object + return typ, nil } -// SerializationIDFromBytes reads a SerializationID and version from the given byte slice. The slice must have a length -// of at least 4 bytes. This function does not perform any validation, and is merely a convenience to ensure that the -// ID is read correctly. -func SerializationIDFromBytes(b []byte) (SerializationID, uint16) { - return SerializationID(binary.LittleEndian.Uint16(b)), binary.LittleEndian.Uint16(b[2:]) +// Serialize returns the DoltgresType as a byte slice. +func (t *DoltgresType) Serialize() []byte { + writer := utils.NewWriter(256) + writer.VariableUint(0) // Version + // Write the type to the writer + writer.Uint32(t.OID) + writer.String(t.Name) + writer.String(t.Schema) + writer.String(t.Owner) + writer.Int16(t.TypLength) + writer.Bool(t.PassedByVal) + writer.String(string(t.TypType)) + writer.String(string(t.TypCategory)) + writer.Bool(t.IsPreferred) + writer.Bool(t.IsDefined) + writer.String(t.Delimiter) + writer.Uint32(t.RelID) + writer.String(globalFunctionRegistry.GetFullString(t.SubscriptFunc)) + writer.Uint32(t.Elem) + writer.Uint32(t.Array) + writer.String(globalFunctionRegistry.GetFullString(t.InputFunc)) + writer.String(globalFunctionRegistry.GetFullString(t.OutputFunc)) + writer.String(globalFunctionRegistry.GetFullString(t.ReceiveFunc)) + writer.String(globalFunctionRegistry.GetFullString(t.SendFunc)) + writer.String(globalFunctionRegistry.GetFullString(t.ModInFunc)) + writer.String(globalFunctionRegistry.GetFullString(t.ModOutFunc)) + writer.String(globalFunctionRegistry.GetFullString(t.AnalyzeFunc)) + writer.String(string(t.Align)) + writer.String(string(t.Storage)) + writer.Bool(t.NotNull) + writer.Uint32(t.BaseTypeOID) + writer.Int32(t.TypMod) + writer.Int32(t.NDims) + writer.Uint32(t.TypCollation) + writer.String(t.DefaulBin) + writer.String(t.Default) + writer.VariableUint(uint64(len(t.Acl))) + for _, ac := range t.Acl { + writer.String(ac) + } + writer.VariableUint(uint64(len(t.Checks))) + for _, check := range t.Checks { + writer.String(check.Name) + writer.String(check.CheckExpression) + } + writer.Int32(t.attTypMod) + writer.String(globalFunctionRegistry.GetFullString(t.CompareFunc)) + writer.String(t.InternalName) + return writer.Data() } diff --git a/server/types/serialization_test.go b/server/types/serialization_test.go index 23b9b0b7d6..fcd449cd29 100644 --- a/server/types/serialization_test.go +++ b/server/types/serialization_test.go @@ -20,146 +20,14 @@ import ( "github.com/stretchr/testify/require" ) -// TestSerialization operates as a line of defense to prevent accidental changes to pre-existing serialization IDs. -// If this test fails, then a SerializationID was changed that should not have been changed. -func TestSerialization(t *testing.T) { - ids := []struct { - SerializationID - ID uint16 - Name string - }{ - {SerializationID_Invalid, 0, "Invalid"}, - {SerializationID_Bit, 1, "Bit"}, - {SerializationID_BitArray, 2, "BitArray"}, - {SerializationID_Bool, 3, "Bool"}, - {SerializationID_BoolArray, 4, "BoolArray"}, - {SerializationID_Box, 5, "Box"}, - {SerializationID_BoxArray, 6, "BoxArray"}, - {SerializationID_Bytea, 7, "Bytea"}, - {SerializationID_ByteaArray, 8, "ByteaArray"}, - {SerializationID_Char, 9, "Char"}, - {SerializationID_CharArray, 10, "CharArray"}, - {SerializationID_Cidr, 11, "Cidr"}, - {SerializationID_CidrArray, 12, "CidrArray"}, - {SerializationID_Circle, 13, "Circle"}, - {SerializationID_CircleArray, 14, "CircleArray"}, - {SerializationID_Date, 15, "Date"}, - {SerializationID_DateArray, 16, "DateArray"}, - {SerializationID_DateMultirange, 17, "DateMultirange"}, - {SerializationID_DateRange, 18, "DateRange"}, - {SerializationID_Enum, 19, "Enum"}, - {SerializationID_EnumArray, 20, "EnumArray"}, - {SerializationID_Float32, 21, "Float32"}, - {SerializationID_Float32Array, 22, "Float32Array"}, - {SerializationID_Float64, 23, "Float64"}, - {SerializationID_Float64Array, 24, "Float64Array"}, - {SerializationID_Inet, 25, "Inet"}, - {SerializationID_InetArray, 26, "InetArray"}, - {SerializationID_Int16, 27, "Int16"}, - {SerializationID_Int16Array, 28, "Int16Array"}, - {SerializationID_Int32, 29, "Int32"}, - {SerializationID_Int32Array, 30, "Int32Array"}, - {SerializationID_Int32Multirange, 31, "Int32Multirange"}, - {SerializationID_Int32Range, 32, "Int32Range"}, - {SerializationID_Int64, 33, "Int64"}, - {SerializationID_Int64Array, 34, "Int64Array"}, - {SerializationID_Int64Multirange, 35, "Int64Multirange"}, - {SerializationID_Int64Range, 36, "Int64Range"}, - {SerializationID_Interval, 37, "Interval"}, - {SerializationID_IntervalArray, 38, "IntervalArray"}, - {SerializationID_Json, 39, "Json"}, - {SerializationID_JsonArray, 40, "JsonArray"}, - {SerializationID_JsonB, 41, "JsonB"}, - {SerializationID_JsonBArray, 42, "JsonBArray"}, - {SerializationID_Line, 43, "Line"}, - {SerializationID_LineArray, 44, "LineArray"}, - {SerializationID_LineSegment, 45, "LineSegment"}, - {SerializationID_LineSegmentArray, 46, "LineSegmentArray"}, - {SerializationID_MacAddress, 47, "MacAddress"}, - {SerializationID_MacAddress8, 48, "MacAddress8"}, - {SerializationID_MacAddress8Array, 49, "MacAddress8Array"}, - {SerializationID_MacAddressArray, 50, "MacAddressArray"}, - {SerializationID_Money, 51, "Money"}, - {SerializationID_MoneyArray, 52, "MoneyArray"}, - {SerializationID_Null, 53, "Null"}, - {SerializationID_Numeric, 54, "Numeric"}, - {SerializationID_NumericArray, 55, "NumericArray"}, - {SerializationID_NumericMultirange, 56, "NumericMultirange"}, - {SerializationID_NumericRange, 57, "NumericRange"}, - {SerializationID_Path, 58, "Path"}, - {SerializationID_PathArray, 59, "PathArray"}, - {SerializationID_Point, 60, "Point"}, - {SerializationID_PointArray, 61, "PointArray"}, - {SerializationID_Polygon, 62, "Polygon"}, - {SerializationID_PolygonArray, 63, "PolygonArray"}, - {SerializationID_Text, 64, "Text"}, - {SerializationID_TextArray, 65, "TextArray"}, - {SerializationID_Time, 66, "Time"}, - {SerializationID_TimeArray, 67, "TimeArray"}, - {SerializationID_TimeTZ, 68, "TimeTZ"}, - {SerializationID_TimeTZArray, 69, "TimeTZArray"}, - {SerializationID_Timestamp, 70, "Timestamp"}, - {SerializationID_TimestampArray, 71, "TimestampArray"}, - {SerializationID_TimestampMultirange, 72, "TimestampMultirange"}, - {SerializationID_TimestampRange, 73, "TimestampRange"}, - {SerializationID_TimestampTZ, 74, "TimestampTZ"}, - {SerializationID_TimestampTZArray, 75, "TimestampTZArray"}, - {SerializationID_TimestampTZMultirange, 76, "TimestampTZMultirange"}, - {SerializationID_TimestampTZRange, 77, "TimestampTZRange"}, - {SerializationID_TsQuery, 78, "TsQuery"}, - {SerializationID_TsQueryArray, 79, "TsQueryArray"}, - {SerializationID_TsVector, 80, "TsVector"}, - {SerializationID_TsVectorArray, 81, "TsVectorArray"}, - {SerializationID_Uuid, 82, "Uuid"}, - {SerializationID_UuidArray, 83, "UuidArray"}, - {SerializationID_VarBit, 84, "VarBit"}, - {SerializationID_VarBitArray, 85, "VarBitArray"}, - {SerializationID_VarChar, 86, "VarChar"}, - {SerializationID_VarCharArray, 87, "VarCharArray"}, - {SerializationID_Xml, 88, "Xml"}, - {SerializationID_XmlArray, 89, "XmlArray"}, - {SerializationID_Name, 90, "Name"}, - {SerializationID_NameArray, 91, "NameArray"}, - {SerializationID_Oid, 92, "Oid"}, - {SerializationID_OidArray, 93, "OidArray"}, - {SerializationID_Xid, 94, "Xid"}, - {SerializationID_XidArray, 95, "XidArray"}, - {SerializationID_InternalChar, 96, "InternalChar"}, - {SerializationID_InternalCharArray, 97, "InternalCharArray"}, - {SerializationId_Domain, 98, "Domain"}, - } - allIds := make(map[uint16]string) - for _, id := range ids { - if uint16(id.SerializationID) != id.ID { - t.Logf("Serialization ID `%s` has been changed from its permanent value of `%d` to `%d`", - id.Name, id.ID, uint16(id.SerializationID)) - t.Fail() - } else if existingName, ok := allIds[id.ID]; ok { - t.Logf("Serialization ID `%s` has the same value as `%s`: `%d`", - id.Name, existingName, id.ID) - t.Fail() - } else { - allIds[id.ID] = id.Name - } - } -} - -// TestSerializationIDConsistency checks that all types use the same SerializationID that they report in -// GetSerializationID and output in SerializeType. -func TestSerializationIDConsistency(t *testing.T) { - for _, typ := range typesFromBaseID { - t.Run(typ.String(), func(t *testing.T) { - sID := typ.GetSerializationID() - if sID == SerializationID_Invalid { - _, err := typ.SerializeType() - require.Error(t, err) - } else { - serializedType, err := typ.SerializeType() - require.NoError(t, err) - require.True(t, len(serializedType) >= serializationIDHeaderSize) - idPrefix := sID.ToByteSlice(0)[:2] - require.Equal(t, idPrefix, serializedType[:2]) - } +// TestSerializationConsistency checks that all types serialization and deserialization. +func TestSerializationConsistency(t *testing.T) { + for _, typ := range typesFromOID { + t.Run(typ.Name, func(t *testing.T) { + serializedType := typ.Serialize() + dt, err := DeserializeType(serializedType) + require.NoError(t, err) + require.Equal(t, typ, dt.(*DoltgresType)) }) } } diff --git a/server/types/text.go b/server/types/text.go index f7e038bf52..d1e4f2f2d2 100644 --- a/server/types/text.go +++ b/server/types/text.go @@ -15,260 +15,43 @@ package types import ( - "bytes" - "fmt" - "math" - "reflect" - - "github.com/dolthub/doltgresql/utils" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Text is the text type. -var Text = TextType{} - -// TextType is the extended type implementation of the PostgreSQL text. -type TextType struct{} - -var _ DoltgresType = TextType{} - -// Alignment implements the DoltgresType interface. -func (b TextType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b TextType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Text -} - -// BaseName implements the DoltgresType interface. -func (b TextType) BaseName() string { - return "text" -} - -// Category implements the DoltgresType interface. -func (b TextType) Category() TypeCategory { - return TypeCategory_StringTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b TextType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b TextType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(string) - bb := bc.(string) - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b TextType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case string: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b TextType) Equals(otherType sql.Type) bool { - if _, ok := otherType.(TextType); !ok { - return false - } - - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b TextType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b TextType) GetSerializationID() SerializationID { - return SerializationID_Text -} - -// IoInput implements the DoltgresType interface. -func (b TextType) IoInput(ctx *sql.Context, input string) (any, error) { - return input, nil -} - -// IoOutput implements the DoltgresType interface. -func (b TextType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return converted.(string), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b TextType) IsPreferredType() bool { - return true -} - -// IsUnbounded implements the DoltgresType interface. -func (b TextType) IsUnbounded() bool { - return true -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b TextType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_Unbounded -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b TextType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return math.MaxUint32 -} - -// OID implements the DoltgresType interface. -func (b TextType) OID() uint32 { - return uint32(oid.T_text) -} - -// Promote implements the DoltgresType interface. -func (b TextType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b TextType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - return serializedStringCompare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b TextType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b TextType) String() string { - return "text" -} - -// ToArrayType implements the DoltgresType interface. -func (b TextType) ToArrayType() DoltgresArrayType { - return TextArray -} - -// Type implements the DoltgresType interface. -func (b TextType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b TextType) ValueType() reflect.Type { - return reflect.TypeOf("") -} - -// Zero implements the DoltgresType interface. -func (b TextType) Zero() any { - return "" -} - -// SerializeType implements the DoltgresType interface. -func (b TextType) SerializeType() ([]byte, error) { - return SerializationID_Text.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b TextType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Text, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b TextType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - str := converted.(string) - writer := utils.NewWriter(uint64(len(str) + 4)) - writer.String(str) - return writer.Data(), nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b TextType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - reader := utils.NewReader(val) - return reader.String(), nil -} - -// serializedStringCompare handles the efficient comparison of two strings that have been serialized using utils.Writer. -// The writer writes the string by prepending the string length, which prevents direct comparison of the byte slices. We -// thus read the string length manually, and extract the byte slices without converting to a string. This function -// assumes that neither byte slice is nil or empty. -func serializedStringCompare(v1 []byte, v2 []byte) int { - readerV1 := utils.NewReader(v1) - readerV2 := utils.NewReader(v2) - v1Bytes := utils.AdvanceReader(readerV1, readerV1.VariableUint()) - v2Bytes := utils.AdvanceReader(readerV2, readerV2.VariableUint()) - return bytes.Compare(v1Bytes, v2Bytes) +var Text = &DoltgresType{ + OID: uint32(oid.T_text), + Name: "text", + Schema: "pg_catalog", + TypLength: int16(-1), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_StringTypes, + IsPreferred: true, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__text), + InputFunc: toFuncID("textin", oid.T_cstring), + OutputFunc: toFuncID("textout", oid.T_text), + ReceiveFunc: toFuncID("textrecv", oid.T_internal), + SendFunc: toFuncID("textsend", oid.T_text), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Extended, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 100, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("bttextcmp", oid.T_text, oid.T_text), } diff --git a/server/types/text_array.go b/server/types/text_array.go index f2732301db..c3c0a51714 100644 --- a/server/types/text_array.go +++ b/server/types/text_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // TextArray is the array variant of Text. -var TextArray = createArrayType(Text, SerializationID_TextArray, oid.T__text) +var TextArray = CreateArrayTypeFromBaseType(Text) diff --git a/server/types/time.go b/server/types/time.go index a76bd2ca7a..d4663fece1 100644 --- a/server/types/time.go +++ b/server/types/time.go @@ -15,260 +15,73 @@ package types import ( - "bytes" "fmt" - "reflect" - "time" - "github.com/dolthub/doltgresql/postgres/parser/sem/tree" - "github.com/dolthub/doltgresql/postgres/parser/timeofday" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Time is the time without a time zone. Precision is unbounded. -var Time = TimeType{-1} - -// TimeType is the extended type implementation of the PostgreSQL time without time zone. -type TimeType struct { - // TODO: implement precision - Precision int8 -} - -var _ DoltgresType = TimeType{} - -// Alignment implements the DoltgresType interface. -func (b TimeType) Alignment() TypeAlignment { - return TypeAlignment_Double -} - -// BaseID implements the DoltgresType interface. -func (b TimeType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Time -} - -// BaseName implements the DoltgresType interface. -func (b TimeType) BaseName() string { - return "time" -} - -// Category implements the DoltgresType interface. -func (b TimeType) Category() TypeCategory { - return TypeCategory_DateTimeTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b TimeType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b TimeType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(time.Time) - bb := bc.(time.Time) - return ab.Compare(bb), nil -} - -// Convert implements the DoltgresType interface. -func (b TimeType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case time.Time: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b TimeType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b TimeType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b TimeType) GetSerializationID() SerializationID { - return SerializationID_Time -} - -// IoInput implements the DoltgresType interface. -func (b TimeType) IoInput(ctx *sql.Context, input string) (any, error) { - p := b.Precision - if p == -1 { - p = 6 - } - t, _, err := tree.ParseDTime(nil, input, tree.TimeFamilyPrecisionToRoundDuration(int32(p))) +var Time = &DoltgresType{ + OID: uint32(oid.T_time), + Name: "time", + Schema: "pg_catalog", + TypLength: int16(8), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_DateTimeTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__time), + InputFunc: toFuncID("time_in", oid.T_cstring, oid.T_oid, oid.T_int4), + OutputFunc: toFuncID("time_out", oid.T_time), + ReceiveFunc: toFuncID("time_recv", oid.T_internal, oid.T_oid, oid.T_int4), + SendFunc: toFuncID("time_send", oid.T_time), + ModInFunc: toFuncID("timetypmodin", oid.T__cstring), + ModOutFunc: toFuncID("timetypmodout", oid.T_int4), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Double, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("time_cmp", oid.T_time, oid.T_time), +} + +// NewTimeType returns Time type with typmod set. // TODO: implement precision +func NewTimeType(precision int32) (*DoltgresType, error) { + typmod, err := GetTypmodFromTimePrecision(precision) if err != nil { return nil, err } - return timeofday.TimeOfDay(*t).ToTime(), nil -} - -// IoOutput implements the DoltgresType interface. -func (b TimeType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return converted.(time.Time).Format("15:04:05.999999999"), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b TimeType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b TimeType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b TimeType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b TimeType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 8 -} - -// OID implements the DoltgresType interface. -func (b TimeType) OID() uint32 { - return uint32(oid.T_time) -} - -// Promote implements the DoltgresType interface. -func (b TimeType) Promote() sql.Type { - return Time -} - -// SerializedCompare implements the DoltgresType interface. -func (b TimeType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - // The marshalled time format is byte-comparable - return bytes.Compare(v1, v2), nil + newType := *Time.WithAttTypMod(typmod) + return &newType, nil } -// SQL implements the DoltgresType interface. -func (b TimeType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err +// GetTypmodFromTimePrecision takes Time type precision and returns the type modifier value. +func GetTypmodFromTimePrecision(precision int32) (int32, error) { + if precision < 0 { + // TIME(-1) precision must not be negative + return 0, fmt.Errorf("TIME(%v) precision must be not be negative", precision) } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b TimeType) String() string { - if b.Precision == -1 { - return "time" + if precision > 6 { + precision = 6 + //WARNING: TIME(7) precision reduced to maximum allowed, 6 } - return fmt.Sprintf("time(%d)", b.Precision) -} - -// ToArrayType implements the DoltgresType interface. -func (b TimeType) ToArrayType() DoltgresArrayType { - return createArrayType(b, SerializationID_TimeArray, oid.T__time) + return precision, nil } -// Type implements the DoltgresType interface. -func (b TimeType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b TimeType) ValueType() reflect.Type { - return reflect.TypeOf(time.Time{}) -} - -// Zero implements the DoltgresType interface. -func (b TimeType) Zero() any { - return time.Time{} -} - -// SerializeType implements the DoltgresType interface. -func (b TimeType) SerializeType() ([]byte, error) { - t := make([]byte, serializationIDHeaderSize+1) - copy(t, SerializationID_Time.ToByteSlice(0)) - t[serializationIDHeaderSize] = byte(b.Precision) - return t, nil -} - -// deserializeType implements the DoltgresType interface. -func (b TimeType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return TimeType{ - Precision: int8(metadata[0]), - }, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b TimeType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - return converted.(time.Time).MarshalBinary() -} - -// DeserializeValue implements the DoltgresType interface. -func (b TimeType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - t := time.Time{} - if err := t.UnmarshalBinary(val); err != nil { - return nil, err - } - return t, nil +// GetTimePrecisionFromTypMod takes Time type modifier and returns precision value. +func GetTimePrecisionFromTypMod(typmod int32) int32 { + return typmod } diff --git a/server/types/time_array.go b/server/types/time_array.go index 7a5aa36626..a9358d5bc6 100644 --- a/server/types/time_array.go +++ b/server/types/time_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // TimeArray is the array variant of Time. -var TimeArray = createArrayType(Time, SerializationID_TimeArray, oid.T__time) +var TimeArray = CreateArrayTypeFromBaseType(Time) diff --git a/server/types/timestamp.go b/server/types/timestamp.go index 00b8ccf5d0..f580aa8c2f 100644 --- a/server/types/timestamp.go +++ b/server/types/timestamp.go @@ -15,259 +15,53 @@ package types import ( - "bytes" - "fmt" - "reflect" - "time" - - "github.com/dolthub/doltgresql/postgres/parser/sem/tree" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Timestamp is the timestamp without a time zone. Precision is unbounded. -var Timestamp = TimestampType{-1} - -// TimestampType is the extended type implementation of the PostgreSQL timestamp without time zone. -type TimestampType struct { - // TODO: implement precision - Precision int8 -} - -var _ DoltgresType = TimestampType{} - -// Alignment implements the DoltgresType interface. -func (b TimestampType) Alignment() TypeAlignment { - return TypeAlignment_Double -} - -// BaseID implements the DoltgresType interface. -func (b TimestampType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Timestamp -} - -// BaseName implements the DoltgresType interface. -func (b TimestampType) BaseName() string { - return "timestamp" -} - -// Category implements the DoltgresType interface. -func (b TimestampType) Category() TypeCategory { - return TypeCategory_DateTimeTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b TimestampType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b TimestampType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) +var Timestamp = &DoltgresType{ + OID: uint32(oid.T_timestamp), + Name: "timestamp", + Schema: "pg_catalog", + TypLength: int16(8), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_DateTimeTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__timestamp), + InputFunc: toFuncID("timestamp_in", oid.T_cstring, oid.T_oid, oid.T_int4), + OutputFunc: toFuncID("timestamp_out", oid.T_timestamp), + ReceiveFunc: toFuncID("timestamp_recv", oid.T_internal, oid.T_oid, oid.T_int4), + SendFunc: toFuncID("timestamp_send", oid.T_timestamp), + ModInFunc: toFuncID("timestamptypmodin", oid.T__cstring), + ModOutFunc: toFuncID("timestamptypmodout", oid.T_int4), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Double, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("timestamp_cmp", oid.T_timestamp, oid.T_timestamp), +} + +// NewTimestampType returns Timestamp type with typmod set. // TODO: implement precision +func NewTimestampType(precision int32) (*DoltgresType, error) { + typmod, err := GetTypmodFromTimePrecision(precision) if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(time.Time) - bb := bc.(time.Time) - return ab.Compare(bb), nil -} - -// Convert implements the DoltgresType interface. -func (b TimestampType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case time.Time: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b TimestampType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b TimestampType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b TimestampType) GetSerializationID() SerializationID { - return SerializationID_Timestamp -} - -// IoInput implements the DoltgresType interface. -func (b TimestampType) IoInput(ctx *sql.Context, input string) (any, error) { - p := b.Precision - if p == -1 { - p = 6 - } - t, _, err := tree.ParseDTimestamp(nil, input, tree.TimeFamilyPrecisionToRoundDuration(int32(p))) - if err != nil { - return nil, err - } - return t.Time, nil -} - -// IoOutput implements the DoltgresType interface. -func (b TimestampType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return converted.(time.Time).Format("2006-01-02 15:04:05.999999999"), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b TimestampType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b TimestampType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b TimestampType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b TimestampType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 8 -} - -// OID implements the DoltgresType interface. -func (b TimestampType) OID() uint32 { - return uint32(oid.T_timestamp) -} - -// Promote implements the DoltgresType interface. -func (b TimestampType) Promote() sql.Type { - return Timestamp -} - -// SerializedCompare implements the DoltgresType interface. -func (b TimestampType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - // The marshalled time format is byte-comparable - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b TimestampType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b TimestampType) String() string { - if b.Precision == -1 { - return "timestamp" - } - return fmt.Sprintf("timestamp(%d)", b.Precision) -} - -// ToArrayType implements the DoltgresType interface. -func (b TimestampType) ToArrayType() DoltgresArrayType { - return createArrayType(b, SerializationID_TimestampArray, oid.T__timestamp) -} - -// Type implements the DoltgresType interface. -func (b TimestampType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b TimestampType) ValueType() reflect.Type { - return reflect.TypeOf(time.Time{}) -} - -// Zero implements the DoltgresType interface. -func (b TimestampType) Zero() any { - return time.Time{} -} - -// SerializeType implements the DoltgresType interface. -func (b TimestampType) SerializeType() ([]byte, error) { - t := make([]byte, serializationIDHeaderSize+1) - copy(t, SerializationID_Timestamp.ToByteSlice(0)) - t[serializationIDHeaderSize] = byte(b.Precision) - return t, nil -} - -// deserializeType implements the DoltgresType interface. -func (b TimestampType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return TimestampType{ - Precision: int8(metadata[0]), - }, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b TimestampType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - return converted.(time.Time).MarshalBinary() -} - -// DeserializeValue implements the DoltgresType interface. -func (b TimestampType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - t := time.Time{} - if err := t.UnmarshalBinary(val); err != nil { return nil, err } - return t, nil + newType := *Timestamp.WithAttTypMod(typmod) + return &newType, nil } diff --git a/server/types/timestamp_array.go b/server/types/timestamp_array.go index 442e5b1c7f..35b18bb3c3 100644 --- a/server/types/timestamp_array.go +++ b/server/types/timestamp_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // TimestampArray is the array variant of Timestamp. -var TimestampArray = createArrayType(Timestamp, SerializationID_TimestampArray, oid.T__timestamp) +var TimestampArray = CreateArrayTypeFromBaseType(Timestamp) diff --git a/server/types/timestamptz.go b/server/types/timestamptz.go index e72c157ca6..2b6adecdea 100644 --- a/server/types/timestamptz.go +++ b/server/types/timestamptz.go @@ -15,273 +15,53 @@ package types import ( - "bytes" - "fmt" - "reflect" - "time" - - "github.com/dolthub/doltgresql/postgres/parser/sem/tree" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // TimestampTZ is the timestamp with a time zone. Precision is unbounded. -var TimestampTZ = TimestampTZType{-1} - -// TimestampTZType is the extended type implementation of the PostgreSQL timestamp with time zone. -type TimestampTZType struct { - // TODO: implement precision - Precision int8 -} - -var _ DoltgresType = TimestampTZType{} - -// Alignment implements the DoltgresType interface. -func (b TimestampTZType) Alignment() TypeAlignment { - return TypeAlignment_Double -} - -// BaseID implements the DoltgresType interface. -func (b TimestampTZType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_TimestampTZ -} - -// BaseName implements the DoltgresType interface. -func (b TimestampTZType) BaseName() string { - return "timestamptz" -} - -// Category implements the DoltgresType interface. -func (b TimestampTZType) Category() TypeCategory { - return TypeCategory_DateTimeTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b TimestampTZType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b TimestampTZType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(time.Time) - bb := bc.(time.Time) - return ab.Compare(bb), nil -} - -// Convert implements the DoltgresType interface. -func (b TimestampTZType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case time.Time: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b TimestampTZType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b TimestampTZType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b TimestampTZType) GetSerializationID() SerializationID { - return SerializationID_TimestampTZ -} - -// IoInput implements the DoltgresType interface. -func (b TimestampTZType) IoInput(ctx *sql.Context, input string) (any, error) { - p := b.Precision - if p == -1 { - p = 6 - } - loc, err := GetServerLocation(ctx) - if err != nil { - return nil, err - } - t, _, err := tree.ParseDTimestampTZ(nil, input, tree.TimeFamilyPrecisionToRoundDuration(int32(p)), loc) - if err != nil { - return nil, err - } - return t.Time, nil -} - -// IoOutput implements the DoltgresType interface. -func (b TimestampTZType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) +var TimestampTZ = &DoltgresType{ + OID: uint32(oid.T_timestamptz), + Name: "timestamptz", + Schema: "pg_catalog", + TypLength: int16(8), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_DateTimeTypes, + IsPreferred: true, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__timestamptz), + InputFunc: toFuncID("timestamptz_in", oid.T_cstring, oid.T_oid, oid.T_int4), + OutputFunc: toFuncID("timestamptz_out", oid.T_timestamptz), + ReceiveFunc: toFuncID("timestamptz_recv", oid.T_internal, oid.T_oid, oid.T_int4), + SendFunc: toFuncID("timestamptz_send", oid.T_timestamptz), + ModInFunc: toFuncID("timestamptztypmodin", oid.T__cstring), + ModOutFunc: toFuncID("timestamptztypmodout", oid.T_int4), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Double, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("timestamptz_cmp", oid.T_timestamptz, oid.T_timestamptz), +} + +// NewTimestampTZType returns TimestampTZ type with typmod set. // TODO: implement precision +func NewTimestampTZType(precision int32) (*DoltgresType, error) { + typmod, err := GetTypmodFromTimePrecision(precision) if err != nil { - return "", err - } - serverLoc, err := GetServerLocation(ctx) - if err != nil { - return "", err - } - t := converted.(time.Time).In(serverLoc) - _, offset := t.Zone() - if offset%3600 != 0 { - return t.Format("2006-01-02 15:04:05.999999999-07:00"), nil - } else { - return t.Format("2006-01-02 15:04:05.999999999-07"), nil - } -} - -// IsPreferredType implements the DoltgresType interface. -func (b TimestampTZType) IsPreferredType() bool { - return true -} - -// IsUnbounded implements the DoltgresType interface. -func (b TimestampTZType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b TimestampTZType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b TimestampTZType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 8 -} - -// OID implements the DoltgresType interface. -func (b TimestampTZType) OID() uint32 { - return uint32(oid.T_timestamptz) -} - -// Promote implements the DoltgresType interface. -func (b TimestampTZType) Promote() sql.Type { - return TimestampTZ -} - -// SerializedCompare implements the DoltgresType interface. -func (b TimestampTZType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - // The marshalled time format is byte-comparable - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b TimestampTZType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b TimestampTZType) String() string { - if b.Precision == -1 { - return "timestamptz" - } - return fmt.Sprintf("timestamptz(%d)", b.Precision) -} - -// ToArrayType implements the DoltgresType interface. -func (b TimestampTZType) ToArrayType() DoltgresArrayType { - return createArrayType(b, SerializationID_TimestampTZArray, oid.T__timestamptz) -} - -// Type implements the DoltgresType interface. -func (b TimestampTZType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b TimestampTZType) ValueType() reflect.Type { - return reflect.TypeOf(time.Time{}) -} - -// Zero implements the DoltgresType interface. -func (b TimestampTZType) Zero() any { - return time.Time{} -} - -// SerializeType implements the DoltgresType interface. -func (b TimestampTZType) SerializeType() ([]byte, error) { - t := make([]byte, serializationIDHeaderSize+1) - copy(t, SerializationID_TimestampTZ.ToByteSlice(0)) - t[serializationIDHeaderSize] = byte(b.Precision) - return t, nil -} - -// deserializeType implements the DoltgresType interface. -func (b TimestampTZType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return TimestampTZType{ - Precision: int8(metadata[0]), - }, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b TimestampTZType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - return converted.(time.Time).MarshalBinary() -} - -// DeserializeValue implements the DoltgresType interface. -func (b TimestampTZType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - t := time.Time{} - if err := t.UnmarshalBinary(val); err != nil { return nil, err } - return t, nil + newType := *TimestampTZ.WithAttTypMod(typmod) + return &newType, nil } diff --git a/server/types/timestamptz_array.go b/server/types/timestamptz_array.go index 8f92d5dd54..3722b8295f 100644 --- a/server/types/timestamptz_array.go +++ b/server/types/timestamptz_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // TimestampTZArray is the array variant of TimestampTZ. -var TimestampTZArray = createArrayType(TimestampTZ, SerializationID_TimestampTZArray, oid.T__timestamptz) +var TimestampTZArray = CreateArrayTypeFromBaseType(TimestampTZ) diff --git a/server/types/timetz.go b/server/types/timetz.go index e987b800fe..240350e720 100644 --- a/server/types/timetz.go +++ b/server/types/timetz.go @@ -15,266 +15,53 @@ package types import ( - "bytes" - "fmt" - "reflect" - "time" - - "github.com/dolthub/doltgresql/postgres/parser/sem/tree" - "github.com/dolthub/doltgresql/postgres/parser/timetz" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // TimeTZ is the time with a time zone. Precision is unbounded. -var TimeTZ = TimeTZType{-1} - -// TimeTZType is the extended type implementation of the PostgreSQL time with time zone. -type TimeTZType struct { - // TODO: implement precision - Precision int8 -} - -var _ DoltgresType = TimeTZType{} - -// Alignment implements the DoltgresType interface. -func (b TimeTZType) Alignment() TypeAlignment { - return TypeAlignment_Double -} - -// BaseID implements the DoltgresType interface. -func (b TimeTZType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_TimeTZ -} - -// BaseName implements the DoltgresType interface. -func (b TimeTZType) BaseName() string { - return "timetz" -} - -// Category implements the DoltgresType interface. -func (b TimeTZType) Category() TypeCategory { - return TypeCategory_DateTimeTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b TimeTZType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b TimeTZType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(time.Time) - bb := bc.(time.Time) - return ab.Compare(bb), nil -} - -// Convert implements the DoltgresType interface. -func (b TimeTZType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case time.Time: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b TimeTZType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b TimeTZType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b TimeTZType) GetSerializationID() SerializationID { - return SerializationID_TimeTZ -} - -// IoInput implements the DoltgresType interface. -func (b TimeTZType) IoInput(ctx *sql.Context, input string) (any, error) { - p := b.Precision - if p == -1 { - p = 6 - } - loc, err := GetServerLocation(ctx) - if err != nil { - return nil, err - } - t, _, err := timetz.ParseTimeTZ(time.Now().In(loc), input, tree.TimeFamilyPrecisionToRoundDuration(int32(p))) - if err != nil { - return nil, err - } - return t.ToTime(), nil -} - -// IoOutput implements the DoltgresType interface. -func (b TimeTZType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - // TODO: this always displays the time with an offset relevant to the server location - t := converted.(time.Time) - return timetz.MakeTimeTZFromTime(t).String(), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b TimeTZType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b TimeTZType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b TimeTZType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b TimeTZType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 12 -} - -// OID implements the DoltgresType interface. -func (b TimeTZType) OID() uint32 { - return uint32(oid.T_timetz) -} - -// Promote implements the DoltgresType interface. -func (b TimeTZType) Promote() sql.Type { - return TimeTZ -} - -// SerializedCompare implements the DoltgresType interface. -func (b TimeTZType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - // The marshalled time format is byte-comparable - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b TimeTZType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b TimeTZType) String() string { - if b.Precision == -1 { - return "timetz" - } - return fmt.Sprintf("timetz(%d)", b.Precision) -} - -// ToArrayType implements the DoltgresType interface. -func (b TimeTZType) ToArrayType() DoltgresArrayType { - return createArrayType(b, SerializationID_TimeTZArray, oid.T__timetz) -} - -// Type implements the DoltgresType interface. -func (b TimeTZType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b TimeTZType) ValueType() reflect.Type { - return reflect.TypeOf(time.Time{}) -} - -// Zero implements the DoltgresType interface. -func (b TimeTZType) Zero() any { - return time.Time{} -} - -// SerializeType implements the DoltgresType interface. -func (b TimeTZType) SerializeType() ([]byte, error) { - t := make([]byte, serializationIDHeaderSize+1) - copy(t, SerializationID_TimeTZ.ToByteSlice(0)) - t[serializationIDHeaderSize] = byte(b.Precision) - return t, nil -} - -// deserializeType implements the DoltgresType interface. -func (b TimeTZType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return TimeTZType{ - Precision: int8(metadata[0]), - }, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b TimeTZType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) +var TimeTZ = &DoltgresType{ + OID: uint32(oid.T_timetz), + Name: "timetz", + Schema: "pg_catalog", + TypLength: int16(12), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_DateTimeTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__timetz), + InputFunc: toFuncID("timetz_in", oid.T_cstring, oid.T_oid, oid.T_int4), + OutputFunc: toFuncID("timetz_out", oid.T_timetz), + ReceiveFunc: toFuncID("timetz_recv", oid.T_internal, oid.T_oid, oid.T_int4), + SendFunc: toFuncID("timetz_send", oid.T_timetz), + ModInFunc: toFuncID("timetztypmodin", oid.T__cstring), + ModOutFunc: toFuncID("timetztypmodout", oid.T_int4), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Double, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("timetz_cmp", oid.T_timetz, oid.T_timetz), +} + +// NewTimeTZType returns TimeTZ type with typmod set. // TODO: implement precision +func NewTimeTZType(precision int32) (*DoltgresType, error) { + typmod, err := GetTypmodFromTimePrecision(precision) if err != nil { return nil, err } - return converted.(time.Time).MarshalBinary() -} - -// DeserializeValue implements the DoltgresType interface. -func (b TimeTZType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - t := time.Time{} - if err := t.UnmarshalBinary(val); err != nil { - return nil, err - } - return t, nil + newType := *TimeTZ.WithAttTypMod(typmod) + return &newType, nil } diff --git a/server/types/timetz_array.go b/server/types/timetz_array.go index 201d667ace..cd023d1b16 100644 --- a/server/types/timetz_array.go +++ b/server/types/timetz_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // TimeTZArray is the array variant of TimeTZ. -var TimeTZArray = createArrayType(TimeTZ, SerializationID_TimeTZArray, oid.T__timetz) +var TimeTZArray = CreateArrayTypeFromBaseType(TimeTZ) diff --git a/server/types/type.go b/server/types/type.go new file mode 100644 index 0000000000..de9db446f4 --- /dev/null +++ b/server/types/type.go @@ -0,0 +1,818 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package types + +import ( + "bytes" + "fmt" + "math" + "reflect" + "time" + + "github.com/dolthub/go-mysql-server/sql" + "github.com/dolthub/go-mysql-server/sql/types" + "github.com/dolthub/vitess/go/sqltypes" + "github.com/dolthub/vitess/go/vt/proto/query" + "github.com/lib/pq/oid" + "github.com/shopspring/decimal" + + "github.com/dolthub/doltgresql/postgres/parser/duration" + "github.com/dolthub/doltgresql/postgres/parser/uuid" + "github.com/dolthub/doltgresql/utils" +) + +// DoltgresType represents a single type. +type DoltgresType struct { + OID uint32 + Name string + Schema string // TODO: should be `uint32`. + Owner string // TODO: should be `uint32`. + TypLength int16 + PassedByVal bool + TypType TypeType + TypCategory TypeCategory + IsPreferred bool + IsDefined bool + Delimiter string + RelID uint32 // for Composite types + SubscriptFunc uint32 + Elem uint32 + Array uint32 + InputFunc uint32 + OutputFunc uint32 + ReceiveFunc uint32 + SendFunc uint32 + ModInFunc uint32 + ModOutFunc uint32 + AnalyzeFunc uint32 + Align TypeAlignment + Storage TypeStorage + NotNull bool // for Domain types + BaseTypeOID uint32 // for Domain types + TypMod int32 // for Domain types + NDims int32 // for Domain types + TypCollation uint32 + DefaulBin string // for Domain types + Default string + Acl []string // TODO: list of privileges + + // Below are not part of pg_type fields + Checks []*sql.CheckDefinition // TODO: should be in `pg_constraint` for Domain types + attTypMod int32 // TODO: should be in `pg_attribute.atttypmod` + CompareFunc uint32 // TODO: should be in `pg_amproc` + InternalName string // Name and InternalName differ for some types. e.g.: "int2" vs "smallint" + + // Below are not stored + IsSerial bool // used for serial types only (e.g.: smallserial) + BaseTypeForInternal uint32 // used for INTERNAL type only +} + +var _ types.ExtendedType = &DoltgresType{} + +// NewUnresolvedDoltgresType returns DoltgresType that is not resolved. +// The type will have 0 as OID and the schema and name defined with given values. +func NewUnresolvedDoltgresType(sch, name string) *DoltgresType { + return &DoltgresType{ + OID: 0, + Name: name, + Schema: sch, + } +} + +// AnalyzeFuncName returns the name that would be displayed in pg_type for the `typanalyze` field. +func (t *DoltgresType) AnalyzeFuncName() string { + return globalFunctionRegistry.GetString(t.AnalyzeFunc) +} + +// ArrayBaseType returns a base type of given array type. +// If this type is not an array type, it returns itself. +func (t *DoltgresType) ArrayBaseType() *DoltgresType { + if !t.IsArrayType() { + return t + } + elem, ok := OidToBuiltInDoltgresType[t.Elem] + if !ok { + panic(fmt.Sprintf("cannot get base type from: %s", t.Name)) + } + newElem := *elem.WithAttTypMod(t.attTypMod) + return &newElem +} + +// CharacterSet implements the sql.StringType interface. +func (t *DoltgresType) CharacterSet() sql.CharacterSetID { + switch oid.Oid(t.OID) { + case oid.T_varchar, oid.T_text, oid.T_name: + return sql.CharacterSet_binary + default: + return sql.CharacterSet_Unspecified + } +} + +// Collation implements the sql.StringType interface. +func (t *DoltgresType) Collation() sql.CollationID { + switch oid.Oid(t.OID) { + case oid.T_varchar, oid.T_text, oid.T_name: + return sql.Collation_Default + default: + return sql.Collation_Unspecified + } +} + +// CollationCoercibility implements the types.ExtendedType interface. +func (t *DoltgresType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { + return sql.Collation_binary, 5 +} + +// Compare implements the types.ExtendedType interface. +func (t *DoltgresType) Compare(v1 interface{}, v2 interface{}) (int, error) { + // TODO: use IoCompare + if v1 == nil && v2 == nil { + return 0, nil + } else if v1 != nil && v2 == nil { + return 1, nil + } else if v1 == nil && v2 != nil { + return -1, nil + } + + switch ab := v1.(type) { + case bool: + bb := v2.(bool) + if ab == bb { + return 0, nil + } else if !ab { + return -1, nil + } else { + return 1, nil + } + case float32: + bb := v2.(float32) + if ab == bb { + return 0, nil + } else if ab < bb { + return -1, nil + } else { + return 1, nil + } + case float64: + bb := v2.(float64) + if ab == bb { + return 0, nil + } else if ab < bb { + return -1, nil + } else { + return 1, nil + } + case int16: + bb := v2.(int16) + if ab == bb { + return 0, nil + } else if ab < bb { + return -1, nil + } else { + return 1, nil + } + case int32: + bb := v2.(int32) + if ab == bb { + return 0, nil + } else if ab < bb { + return -1, nil + } else { + return 1, nil + } + case int64: + bb := v2.(int64) + if ab == bb { + return 0, nil + } else if ab < bb { + return -1, nil + } else { + return 1, nil + } + case uint32: + bb := v2.(uint32) + if ab == bb { + return 0, nil + } else if ab < bb { + return -1, nil + } else { + return 1, nil + } + case string: + bb := v2.(string) + if ab == bb { + return 0, nil + } else if ab < bb { + return -1, nil + } else { + return 1, nil + } + case []byte: + bb := v2.([]byte) + return bytes.Compare(ab, bb), nil + case time.Time: + bb := v2.(time.Time) + return ab.Compare(bb), nil + case duration.Duration: + bb := v2.(duration.Duration) + return ab.Compare(bb), nil + case JsonDocument: + bb := v2.(JsonDocument) + return JsonValueCompare(ab.Value, bb.Value), nil + case decimal.Decimal: + bb := v2.(decimal.Decimal) + return ab.Cmp(bb), nil + case uuid.UUID: + bb := v2.(uuid.UUID) + return bytes.Compare(ab.GetBytesMut(), bb.GetBytesMut()), nil + case []any: + if !t.IsArrayType() { + return 0, fmt.Errorf("array value received in Compare for non array type") + } + bb := v2.([]any) + minLength := utils.Min(len(ab), len(bb)) + for i := 0; i < minLength; i++ { + res, err := t.ArrayBaseType().Compare(ab[i], bb[i]) + if err != nil { + return 0, err + } + if res != 0 { + return res, nil + } + } + if len(ab) == len(bb) { + return 0, nil + } else if len(ab) < len(bb) { + return -1, nil + } else { + return 1, nil + } + default: + return 0, fmt.Errorf("unhandled type %T in Compare", v1) + } +} + +// Convert implements the types.ExtendedType interface. +func (t *DoltgresType) Convert(v interface{}) (interface{}, sql.ConvertInRange, error) { + if v == nil { + return nil, sql.InRange, nil + } + switch oid.Oid(t.OID) { + case oid.T_bool: + if _, ok := v.(bool); ok { + return v, sql.InRange, nil + } + case oid.T_bytea: + if _, ok := v.([]byte); ok { + return v, sql.InRange, nil + } + case oid.T_bpchar, oid.T_char, oid.T_json, oid.T_name, oid.T_text, oid.T_unknown, oid.T_varchar: + if _, ok := v.(string); ok { + return v, sql.InRange, nil + } + case oid.T_date, oid.T_time, oid.T_timestamp, oid.T_timestamptz, oid.T_timetz: + if _, ok := v.(time.Time); ok { + return v, sql.InRange, nil + } + case oid.T_float4: + if _, ok := v.(float32); ok { + return v, sql.InRange, nil + } + case oid.T_float8: + if _, ok := v.(float64); ok { + return v, sql.InRange, nil + } + case oid.T_int2: + if _, ok := v.(int16); ok { + return v, sql.InRange, nil + } + case oid.T_int4: + if _, ok := v.(int32); ok { + return v, sql.InRange, nil + } + case oid.T_int8: + if _, ok := v.(int64); ok { + return v, sql.InRange, nil + } + case oid.T_interval: + if _, ok := v.(duration.Duration); ok { + return v, sql.InRange, nil + } + case oid.T_jsonb: + if _, ok := v.(JsonDocument); ok { + return v, sql.InRange, nil + } + case oid.T_oid, oid.T_regclass, oid.T_regproc, oid.T_regtype, oid.T_xid: + if _, ok := v.(uint32); ok { + return v, sql.InRange, nil + } + case oid.T_uuid: + if _, ok := v.(uuid.UUID); ok { + return v, sql.InRange, nil + } + default: + return v, sql.InRange, nil + } + return nil, sql.OutOfRange, ErrUnhandledType.New(t.String(), v) +} + +// DomainUnderlyingBaseType returns an underlying base type of this domain type. +// It can be a nested domain type, so it recursively searches for a valid base type. +func (t *DoltgresType) DomainUnderlyingBaseType() *DoltgresType { + // TODO: handle user-defined type + bt, ok := OidToBuiltInDoltgresType[t.BaseTypeOID] + if !ok { + panic(fmt.Sprintf("unable to get DoltgresType from OID: %v", t.BaseTypeOID)) + } + if bt.TypType == TypeType_Domain { + return bt.DomainUnderlyingBaseType() + } else { + return bt + } +} + +// Equals implements the types.ExtendedType interface. +func (t *DoltgresType) Equals(otherType sql.Type) bool { + if otherExtendedType, ok := otherType.(*DoltgresType); ok { + return bytes.Equal(t.Serialize(), otherExtendedType.Serialize()) + } + return false +} + +// FormatValue implements the types.ExtendedType interface. +func (t *DoltgresType) FormatValue(val any) (string, error) { + if val == nil { + return "", nil + } + // TODO: need valid sql.Context + return t.IoOutput(nil, val) +} + +// GetAttTypMod returns the attTypMod field of the type. +func (t *DoltgresType) GetAttTypMod() int32 { + return t.attTypMod +} + +// InputFuncName returns the name that would be displayed in pg_type for the `typinput` field. +func (t *DoltgresType) InputFuncName() string { + return globalFunctionRegistry.GetString(t.InputFunc) +} + +// IoInput converts input string value to given type value. +func (t *DoltgresType) IoInput(ctx *sql.Context, input string) (any, error) { + if t.ModInFunc != 0 || t.TypType == TypeType_Domain || t.IsArrayType() { + if t.Elem != 0 { + return globalFunctionRegistry.GetFunction(t.InputFunc).CallVariadic(ctx, input, t.Elem, t.attTypMod) + } else { + return globalFunctionRegistry.GetFunction(t.InputFunc).CallVariadic(ctx, input, t.OID, t.attTypMod) + } + } else { + return globalFunctionRegistry.GetFunction(t.InputFunc).CallVariadic(ctx, input) + } +} + +// IoOutput converts given type value to output string. +func (t *DoltgresType) IoOutput(ctx *sql.Context, val any) (string, error) { + var o any + var err error + if t.ModInFunc != 0 || t.IsArrayType() { + send := globalFunctionRegistry.GetFunction(t.OutputFunc) + resolvedTypes := send.ResolvedTypes() + resolvedTypes[0] = t + o, err = send.WithResolvedTypes(resolvedTypes).(QuickFunction).CallVariadic(ctx, val) + } else { + o, err = globalFunctionRegistry.GetFunction(t.OutputFunc).CallVariadic(ctx, val) + } + if err != nil { + return "", err + } + return o.(string), nil +} + +// IsArrayType returns true if the type is of 'array' category +func (t *DoltgresType) IsArrayType() bool { + return t.TypCategory == TypeCategory_ArrayTypes && t.Elem != 0 +} + +// IsEmptyType returns true if the type has no valid OID or Name. +func (t *DoltgresType) IsEmptyType() bool { + return t == nil +} + +// IsPolymorphicType types are special built-in pseudo-types +// that are used during function resolution to allow a function +// to handle multiple types from a single definition. +// All polymorphic types have "any" as a prefix. +// The exception is the "any" type, which is not a polymorphic type. +func (t *DoltgresType) IsPolymorphicType() bool { + switch oid.Oid(t.OID) { + case oid.T_anyelement, oid.T_anyarray, oid.T_anynonarray: + // TODO: add other polymorphic types + // https://www.postgresql.org/docs/15/extend-type-system.html#EXTEND-TYPES-POLYMORPHIC-TABLE + return true + default: + return false + } +} + +// IsResolvedType whether the type is resolved and has complete information. +// This is used to resolve types during analyzing when non-built-in type is used. +func (t *DoltgresType) IsResolvedType() bool { + // temporary serial types have 0 OID but are resolved. + return t.OID != 0 || t.IsSerial +} + +// IsValidForPolymorphicType returns whether the given type is valid for the calling polymorphic type. +func (t *DoltgresType) IsValidForPolymorphicType(target *DoltgresType) bool { + switch oid.Oid(t.OID) { + case oid.T_anyelement: + return true + case oid.T_anyarray: + return target.TypCategory == TypeCategory_ArrayTypes + case oid.T_anynonarray: + return target.TypCategory != TypeCategory_ArrayTypes + default: + // TODO: add other polymorphic types + // https://www.postgresql.org/docs/15/extend-type-system.html#EXTEND-TYPES-POLYMORPHIC-TABLE + return false + } +} + +// Length implements the sql.StringType interface. +func (t *DoltgresType) Length() int64 { + switch oid.Oid(t.OID) { + case oid.T_varchar: + if t.attTypMod == -1 { + return StringUnbounded + } else { + return int64(GetCharLengthFromTypmod(t.attTypMod)) + } + case oid.T_text: + return StringUnbounded + case oid.T_name: + return int64(t.TypLength) + default: + return int64(0) + } +} + +// MaxByteLength implements the sql.StringType interface. +func (t *DoltgresType) MaxByteLength() int64 { + if t.OID == uint32(oid.T_varchar) { + return t.Length() * 4 + } else if t.TypLength == -1 { + return StringUnbounded + } else { + return int64(t.TypLength) * 4 + } +} + +// MaxCharacterLength implements the sql.StringType interface. +func (t *DoltgresType) MaxCharacterLength() int64 { + if t.OID == uint32(oid.T_varchar) { + return t.Length() + } else if t.TypLength == -1 { + return StringUnbounded + } else { + return int64(t.TypLength) + } +} + +// MaxSerializedWidth implements the types.ExtendedType interface. +func (t *DoltgresType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { + // TODO: need better way to get accurate result + switch t.TypCategory { + case TypeCategory_ArrayTypes: + return types.ExtendedTypeSerializedWidth_Unbounded + case TypeCategory_BooleanTypes: + return types.ExtendedTypeSerializedWidth_64K + case TypeCategory_CompositeTypes, TypeCategory_EnumTypes, TypeCategory_GeometricTypes, TypeCategory_NetworkAddressTypes, + TypeCategory_RangeTypes, TypeCategory_PseudoTypes, TypeCategory_UserDefinedTypes, TypeCategory_BitStringTypes, + TypeCategory_InternalUseTypes: + return types.ExtendedTypeSerializedWidth_Unbounded + case TypeCategory_DateTimeTypes: + return types.ExtendedTypeSerializedWidth_64K + case TypeCategory_NumericTypes: + return types.ExtendedTypeSerializedWidth_64K + case TypeCategory_StringTypes, TypeCategory_UnknownTypes: + if t.OID == uint32(oid.T_varchar) { + l := t.Length() + if l != StringUnbounded && l <= stringInline { + return types.ExtendedTypeSerializedWidth_64K + } + } + return types.ExtendedTypeSerializedWidth_Unbounded + case TypeCategory_TimespanTypes: + return types.ExtendedTypeSerializedWidth_64K + default: + // shouldn't happen + return types.ExtendedTypeSerializedWidth_Unbounded + } +} + +// MaxTextResponseByteLength implements the types.ExtendedType interface. +func (t *DoltgresType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { + if t.OID == uint32(oid.T_varchar) { + l := t.Length() + if l == StringUnbounded { + return math.MaxUint32 + } else { + return uint32(l * 4) + } + } else if t.TypLength == -1 { + return math.MaxUint32 + } else { + return uint32(t.TypLength) + } +} + +// ModInFuncName returns the name that would be displayed in pg_type for the `typmodin` field. +func (t *DoltgresType) ModInFuncName() string { + return globalFunctionRegistry.GetString(t.ModInFunc) +} + +// ModOutFuncName returns the name that would be displayed in pg_type for the `typmodout` field. +func (t *DoltgresType) ModOutFuncName() string { + return globalFunctionRegistry.GetString(t.ModOutFunc) +} + +// OutputFuncName returns the name that would be displayed in pg_type for the `typoutput` field. +func (t *DoltgresType) OutputFuncName() string { + return globalFunctionRegistry.GetString(t.OutputFunc) +} + +// Promote implements the types.ExtendedType interface. +func (t *DoltgresType) Promote() sql.Type { + return t +} + +// ReceiveFuncName returns the name that would be displayed in pg_type for the `typreceive` field. +func (t *DoltgresType) ReceiveFuncName() string { + return globalFunctionRegistry.GetString(t.ReceiveFunc) +} + +// SendFuncName returns the name that would be displayed in pg_type for the `typsend` field. +func (t *DoltgresType) SendFuncName() string { + return globalFunctionRegistry.GetString(t.SendFunc) +} + +// SerializedCompare implements the types.ExtendedType interface. +func (t *DoltgresType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { + if len(v1) == 0 && len(v2) == 0 { + return 0, nil + } else if len(v1) > 0 && len(v2) == 0 { + return 1, nil + } else if len(v1) == 0 && len(v2) > 0 { + return -1, nil + } + + if t.TypCategory == TypeCategory_StringTypes { + return serializedStringCompare(v1, v2), nil + } + return bytes.Compare(v1, v2), nil +} + +// SQL implements the types.ExtendedType interface. +func (t *DoltgresType) SQL(ctx *sql.Context, dest []byte, v interface{}) (sqltypes.Value, error) { + if v == nil { + return sqltypes.NULL, nil + } + value, err := sqlString(ctx, t, v) + if err != nil { + return sqltypes.Value{}, err + } + + // TODO: check type + return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil +} + +// String implements the types.ExtendedType interface. +func (t *DoltgresType) String() string { + str := t.InternalName + if t.InternalName == "" { + str = t.Name + } + if t.attTypMod != -1 { + // TODO: need valid sql.Context + if l, err := t.TypModOut(nil, t.attTypMod); err == nil { + str = fmt.Sprintf("%s%s", str, l) + } + } + return str +} + +// SubscriptFuncName returns the name that would be displayed in pg_type for the `typsubscript` field. +func (t *DoltgresType) SubscriptFuncName() string { + return globalFunctionRegistry.GetString(t.SubscriptFunc) +} + +// ToArrayType returns an array type of given base type. +// For array types, ToArrayType causes them to return themselves. +func (t *DoltgresType) ToArrayType() *DoltgresType { + if t.IsArrayType() { + return t + } + arr, ok := OidToBuiltInDoltgresType[t.Array] + if !ok { + panic(fmt.Sprintf("cannot get array type from: %s", t.Name)) + } + newArr := *arr.WithAttTypMod(t.attTypMod) + newArr.InternalName = fmt.Sprintf("%s[]", t.String()) + return &newArr +} + +// Type implements the types.ExtendedType interface. +func (t *DoltgresType) Type() query.Type { + // TODO: need better way to get accurate result + switch t.TypCategory { + case TypeCategory_ArrayTypes: + return sqltypes.Text + case TypeCategory_BooleanTypes: + return sqltypes.Text + case TypeCategory_CompositeTypes, TypeCategory_EnumTypes, TypeCategory_GeometricTypes, TypeCategory_NetworkAddressTypes, + TypeCategory_RangeTypes, TypeCategory_PseudoTypes, TypeCategory_UserDefinedTypes, TypeCategory_BitStringTypes, + TypeCategory_InternalUseTypes: + return sqltypes.Text + case TypeCategory_DateTimeTypes: + switch oid.Oid(t.OID) { + case oid.T_date: + return sqltypes.Date + case oid.T_time: + return sqltypes.Time + default: + return sqltypes.Timestamp + } + case TypeCategory_NumericTypes: + switch oid.Oid(t.OID) { + case oid.T_float4: + return sqltypes.Float32 + case oid.T_float8: + return sqltypes.Float64 + case oid.T_int2: + return sqltypes.Int16 + case oid.T_int4: + return sqltypes.Int32 + case oid.T_int8: + return sqltypes.Int64 + case oid.T_numeric: + return sqltypes.Decimal + case oid.T_oid: + return sqltypes.Uint32 + case oid.T_regclass, oid.T_regproc, oid.T_regtype: + return sqltypes.Text + default: + // TODO + return sqltypes.Int64 + } + case TypeCategory_StringTypes, TypeCategory_UnknownTypes: + if t.OID == uint32(oid.T_varchar) { + return sqltypes.VarChar + } + return sqltypes.Text + case TypeCategory_TimespanTypes: + return sqltypes.Text + default: + // shouldn't happen + return sqltypes.Text + } +} + +// TypModIn encodes given text array value to type modifier in int32 format. +func (t *DoltgresType) TypModIn(ctx *sql.Context, val []any) (int32, error) { + if t.ModInFunc == 0 { + return 0, fmt.Errorf("typmodin function for type '%s' doesn't exist", t.Name) + } + o, err := globalFunctionRegistry.GetFunction(t.ModInFunc).CallVariadic(ctx, val) + if err != nil { + return 0, err + } + output, ok := o.(int32) + if !ok { + return 0, fmt.Errorf(`expected int32, got %T`, output) + } + return output, nil +} + +// TypModOut decodes type modifier in int32 format to string representation of it. +func (t *DoltgresType) TypModOut(ctx *sql.Context, val int32) (string, error) { + if t.ModOutFunc == 0 { + return "", fmt.Errorf("typmodout function for type '%s' doesn't exist", t.Name) + } + o, err := globalFunctionRegistry.GetFunction(t.ModOutFunc).CallVariadic(ctx, val) + if err != nil { + return "", err + } + output, ok := o.(string) + if !ok { + return "", fmt.Errorf(`expected string, got %T`, output) + } + return output, nil +} + +// ValueType implements the types.ExtendedType interface. +func (t *DoltgresType) ValueType() reflect.Type { + return reflect.TypeOf(t.Zero()) +} + +// WithAttTypMod returns a copy of the type with attTypMod +// defined with given value. This function should be used +// to set attTypMod only, as it creates a copy of the type +// to avoid updating the original type. +func (t *DoltgresType) WithAttTypMod(tm int32) *DoltgresType { + newDt := *t + newDt.attTypMod = tm + return &newDt +} + +// Zero implements the types.ExtendedType interface. +func (t *DoltgresType) Zero() interface{} { + // TODO: need better way to get accurate result + switch t.TypCategory { + case TypeCategory_ArrayTypes: + return []any{} + case TypeCategory_BooleanTypes: + return false + case TypeCategory_CompositeTypes, TypeCategory_EnumTypes, TypeCategory_GeometricTypes, TypeCategory_NetworkAddressTypes, + TypeCategory_RangeTypes, TypeCategory_PseudoTypes, TypeCategory_UserDefinedTypes, TypeCategory_BitStringTypes, + TypeCategory_InternalUseTypes: + return any(nil) + case TypeCategory_DateTimeTypes: + return time.Time{} + case TypeCategory_NumericTypes: + switch oid.Oid(t.OID) { + case oid.T_float4: + return float32(0) + case oid.T_float8: + return float64(0) + case oid.T_int2: + return int16(0) + case oid.T_int4: + return int32(0) + case oid.T_int8: + return int64(0) + case oid.T_numeric: + return decimal.Zero + case oid.T_oid, oid.T_regclass, oid.T_regproc, oid.T_regtype: + return uint32(0) + default: + // TODO + return int64(0) + } + case TypeCategory_StringTypes, TypeCategory_UnknownTypes: + return "" + case TypeCategory_TimespanTypes: + return duration.MakeDuration(0, 0, 0) + default: + // shouldn't happen + return any(nil) + } +} + +// SerializeValue implements the types.ExtendedType interface. +func (t *DoltgresType) SerializeValue(val any) ([]byte, error) { + if val == nil { + return nil, nil + } + var o any + var err error + if t.ModInFunc != 0 || t.IsArrayType() { + send := globalFunctionRegistry.GetFunction(t.SendFunc) + resolvedTypes := send.ResolvedTypes() + resolvedTypes[0] = t + o, err = send.WithResolvedTypes(resolvedTypes).(QuickFunction).CallVariadic(nil, val) + } else { + o, err = globalFunctionRegistry.GetFunction(t.SendFunc).CallVariadic(nil, val) + } + if err != nil || o == nil { + return nil, err + } + return o.([]byte), nil +} + +// DeserializeValue implements the types.ExtendedType interface. +func (t *DoltgresType) DeserializeValue(val []byte) (any, error) { + if len(val) == 0 { + return nil, nil + } + if t.ModInFunc != 0 || t.TypType == TypeType_Domain || t.IsArrayType() { + if t.Elem != 0 { + return globalFunctionRegistry.GetFunction(t.ReceiveFunc).CallVariadic(nil, val, t.Elem, t.attTypMod) + } else { + return globalFunctionRegistry.GetFunction(t.ReceiveFunc).CallVariadic(nil, val, t.OID, t.attTypMod) + } + } else { + return globalFunctionRegistry.GetFunction(t.ReceiveFunc).CallVariadic(nil, val) + } +} diff --git a/server/types/unknown.go b/server/types/unknown.go index 3ea516aaa3..4509d54008 100644 --- a/server/types/unknown.go +++ b/server/types/unknown.go @@ -15,187 +15,43 @@ package types import ( - "fmt" - "math" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Unknown represents an invalid or indeterminate type. This is primarily used internally. -var Unknown = UnknownType{} - -// UnknownType is the extended type implementation of the PostgreSQL unknown type. -type UnknownType struct{} - -var _ DoltgresType = UnknownType{} -var _ DoltgresArrayType = UnknownType{} - -// Alignment implements the DoltgresType interface. -func (u UnknownType) Alignment() TypeAlignment { - return TypeAlignment_Char -} - -// BaseID implements the DoltgresType interface. -func (u UnknownType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Unknown -} - -// BaseName implements the DoltgresType interface. -func (u UnknownType) BaseName() string { - return "unknown" -} - -// Category implements the DoltgresType interface. -func (u UnknownType) Category() TypeCategory { - return TypeCategory_UnknownTypes -} - -// BaseType implements the DoltgresArrayType interface. -func (u UnknownType) BaseType() DoltgresType { - return Unknown -} - -// CollationCoercibility implements the DoltgresType interface. -func (u UnknownType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (u UnknownType) Compare(v1 any, v2 any) (int, error) { - return 0, fmt.Errorf("%s cannot compare values", u.String()) -} - -// Convert implements the DoltgresType interface. -func (u UnknownType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case string: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", u.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (u UnknownType) Equals(otherType sql.Type) bool { - _, ok := otherType.(UnknownType) - return ok -} - -// FormatValue implements the DoltgresType interface. -func (u UnknownType) FormatValue(val any) (string, error) { - return "", fmt.Errorf("%s cannot format values", u.String()) -} - -// GetSerializationID implements the DoltgresType interface. -func (u UnknownType) GetSerializationID() SerializationID { - return SerializationID_Invalid -} - -// IoInput implements the DoltgresType interface. -func (u UnknownType) IoInput(ctx *sql.Context, input string) (any, error) { - return input, nil -} - -// IoOutput implements the DoltgresType interface. -func (u UnknownType) IoOutput(ctx *sql.Context, output any) (string, error) { - return output.(string), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b UnknownType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (u UnknownType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (u UnknownType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_Unbounded -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (u UnknownType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return math.MaxUint32 -} - -// OID implements the DoltgresType interface. -func (u UnknownType) OID() uint32 { - return uint32(oid.T_unknown) -} - -// Promote implements the DoltgresType interface. -func (u UnknownType) Promote() sql.Type { - return u -} - -// SerializedCompare implements the DoltgresType interface. -func (u UnknownType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - return 0, fmt.Errorf("%s cannot compare serialized values", u.String()) -} - -// SQL implements the DoltgresType interface. -func (u UnknownType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := u.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(u.Type(), types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (u UnknownType) String() string { - return "unknown" -} - -// ToArrayType implements the DoltgresType interface. -func (u UnknownType) ToArrayType() DoltgresArrayType { - return u -} - -// Type implements the DoltgresType interface. -func (u UnknownType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (u UnknownType) ValueType() reflect.Type { - return reflect.TypeOf(any(nil)) -} - -// Zero implements the DoltgresType interface. -func (u UnknownType) Zero() any { - return "" -} - -// SerializeType implements the DoltgresType interface. -func (u UnknownType) SerializeType() ([]byte, error) { - return nil, fmt.Errorf("%s cannot be serialized", u.String()) -} - -// deserializeType implements the DoltgresType interface. -func (u UnknownType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - return nil, fmt.Errorf("%s cannot be deserialized", u.String()) -} - -// SerializeValue implements the DoltgresType interface. -func (u UnknownType) SerializeValue(val any) ([]byte, error) { - return nil, fmt.Errorf("%s cannot serialize values", u.String()) -} - -// DeserializeValue implements the DoltgresType interface. -func (u UnknownType) DeserializeValue(val []byte) (any, error) { - return nil, fmt.Errorf("%s cannot deserialize values", u.String()) +var Unknown = &DoltgresType{ + OID: uint32(oid.T_unknown), + Name: "unknown", + Schema: "pg_catalog", + TypLength: int16(-2), + PassedByVal: false, + TypType: TypeType_Pseudo, + TypCategory: TypeCategory_UnknownTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: 0, + InputFunc: toFuncID("unknownin", oid.T_cstring), + OutputFunc: toFuncID("unknownout", oid.T_unknown), + ReceiveFunc: toFuncID("unknownrecv", oid.T_internal), + SendFunc: toFuncID("unknownsend", oid.T_unknown), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Char, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), } diff --git a/server/types/utils.go b/server/types/utils.go index f9fc2e281a..669f3712bf 100644 --- a/server/types/utils.go +++ b/server/types/utils.go @@ -15,104 +15,152 @@ package types import ( + "bytes" "fmt" "strings" - "time" - "unicode/utf8" "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/types" "github.com/dolthub/vitess/go/vt/proto/query" + "github.com/lib/pq/oid" + "gopkg.in/src-d/go-errors.v1" + + "github.com/dolthub/doltgresql/utils" ) -// QuoteString will quote the string according to the type given. This means that some types will quote, and others will -// not, or they may quote in a special way that is unique to that type. -func QuoteString(baseID DoltgresTypeBaseID, str string) string { - switch baseID { - case DoltgresTypeBaseID_Char, DoltgresTypeBaseID_Name, DoltgresTypeBaseID_Text, DoltgresTypeBaseID_VarChar, DoltgresTypeBaseID_Unknown: - return `'` + strings.ReplaceAll(str, `'`, `''`) + `'` - default: - return str - } -} +// ErrTypeAlreadyExists is returned when creating given type when it already exists. +var ErrTypeAlreadyExists = errors.NewKind(`type "%s" already exists`) -// truncateString returns a string that has been truncated to the given length. Uses the rune count rather than the -// byte count. Returns the input string if it's smaller than the length. Also returns the rune count of the string. -func truncateString(val string, runeLimit uint32) (string, uint32) { - runeLength := uint32(utf8.RuneCountInString(val)) - if runeLength > runeLimit { - // TODO: figure out if there's a faster way to truncate based on rune count - startString := val - for i := uint32(0); i < runeLimit; i++ { - _, size := utf8.DecodeRuneInString(val) - val = val[size:] - } - return startString[:len(startString)-len(val)], runeLength +// ErrTypeDoesNotExist is returned when using given type that does not exist. +var ErrTypeDoesNotExist = errors.NewKind(`type "%s" does not exist`) + +// ErrUnhandledType is returned when the type of value does not match given type. +var ErrUnhandledType = errors.NewKind(`%s: unhandled type: %T`) + +// ErrInvalidSyntaxForType is returned when the type of value is invalid for given type. +var ErrInvalidSyntaxForType = errors.NewKind(`invalid input syntax for type %s: %q`) + +// ErrValueIsOutOfRangeForType is returned when the value is out-of-range for given type. +var ErrValueIsOutOfRangeForType = errors.NewKind(`value %q is out of range for type %s`) + +// ErrTypmodArrayMustBe1D is returned when type modifier value is empty array. +var ErrTypmodArrayMustBe1D = errors.NewKind(`typmod array must be one-dimensional`) + +// ErrInvalidTypMod is returned when given value is invalid for type modifier. +var ErrInvalidTypMod = errors.NewKind(`invalid %s type modifier`) + +// FromGmsType returns a DoltgresType that is most similar to the given GMS type. +// It returns UNKNOWN type for GMS types that are not handled. +func FromGmsType(typ sql.Type) *DoltgresType { + dt, err := FromGmsTypeToDoltgresType(typ) + if err != nil { + return Unknown } - return val, runeLength + return dt } -// FromGmsType returns a DoltgresType that is most similar to the given GMS type. -func FromGmsType(typ sql.Type) DoltgresType { +// FromGmsTypeToDoltgresType returns a DoltgresType that is most similar to the given GMS type. +// It errors if GMS type is not handled. +func FromGmsTypeToDoltgresType(typ sql.Type) (*DoltgresType, error) { switch typ.Type() { - case query.Type_INT8: + case query.Type_INT8, query.Type_INT16: // Special treatment for boolean types when we can detect them if typ == types.Boolean { - return Bool + return Bool, nil } - return Int32 - case query.Type_INT16, query.Type_INT24, query.Type_INT32, query.Type_YEAR, query.Type_ENUM: - return Int32 - case query.Type_INT64, query.Type_SET, query.Type_BIT, query.Type_UINT8, query.Type_UINT16, query.Type_UINT24, query.Type_UINT32: - return Int64 - case query.Type_UINT64: - return Numeric + return Int16, nil + case query.Type_INT24, query.Type_INT32: + return Int32, nil + case query.Type_INT64: + return Int64, nil + case query.Type_UINT8, query.Type_UINT16, query.Type_UINT24, query.Type_UINT32, query.Type_UINT64: + return Int64, nil + case query.Type_YEAR: + return Int16, nil case query.Type_FLOAT32: - return Float32 + return Float32, nil case query.Type_FLOAT64: - return Float64 + return Float64, nil case query.Type_DECIMAL: - return Numeric - case query.Type_DATE, query.Type_DATETIME, query.Type_TIMESTAMP: - return Timestamp + return Numeric, nil + case query.Type_DATE: + return Date, nil case query.Type_TIME: - return Text + return Text, nil + case query.Type_DATETIME, query.Type_TIMESTAMP: + return Timestamp, nil case query.Type_CHAR, query.Type_VARCHAR, query.Type_TEXT, query.Type_BINARY, query.Type_VARBINARY, query.Type_BLOB: - return Text + return Text, nil case query.Type_JSON: - return Json - case query.Type_NULL_TYPE: - return Unknown - case query.Type_GEOMETRY: - return Unknown + return Json, nil + case query.Type_ENUM: + return Int16, nil + case query.Type_SET: + return Int64, nil + case query.Type_NULL_TYPE, query.Type_GEOMETRY: + return Unknown, nil default: - return Unknown + return nil, fmt.Errorf("encountered a GMS type that cannot be handled") } } -// GetServerLocation returns timezone value set for the server. -func GetServerLocation(ctx *sql.Context) (*time.Location, error) { - if ctx == nil { - return time.Local, nil - } - val, err := ctx.GetSessionVariable(ctx, "timezone") - if err != nil { - return nil, err - } +// serializedStringCompare handles the efficient comparison of two strings that have been serialized using utils.Writer. +// The writer writes the string by prepending the string length, which prevents direct comparison of the byte slices. We +// thus read the string length manually, and extract the byte slices without converting to a string. This function +// assumes that neither byte slice is nil nor empty. +func serializedStringCompare(v1 []byte, v2 []byte) int { + readerV1 := utils.NewReader(v1) + readerV2 := utils.NewReader(v2) + v1Bytes := utils.AdvanceReader(readerV1, readerV1.VariableUint()) + v2Bytes := utils.AdvanceReader(readerV2, readerV2.VariableUint()) + return bytes.Compare(v1Bytes, v2Bytes) +} - tz := val.(string) - loc, err := time.LoadLocation(tz) - if err == nil { - return loc, nil +// sqlString converts given type value to output string. This is the same as IoOutput function +// with an exception to BOOLEAN type. It returns "t" instead of "true". +func sqlString(ctx *sql.Context, t *DoltgresType, val any) (string, error) { + if t.IsArrayType() { + baseType := t.ArrayBaseType() + return ArrToString(ctx, val.([]any), baseType, true) } + return t.IoOutput(ctx, val) +} - var t time.Time - if t, err = time.Parse("Z07", tz); err == nil { - } else if t, err = time.Parse("Z07:00", tz); err == nil { - } else if t, err = time.Parse("Z07:00:00", tz); err != nil { - return nil, err +// ArrToString is used for array_out function. |trimBool| parameter allows replacing +// boolean result of "true" to "t" if the function is `Type.SQL()`. +func ArrToString(ctx *sql.Context, arr []any, baseType *DoltgresType, trimBool bool) (string, error) { + sb := strings.Builder{} + sb.WriteRune('{') + for i, v := range arr { + if i > 0 { + sb.WriteString(",") + } + if v != nil { + str, err := baseType.IoOutput(ctx, v) + if err != nil { + return "", err + } + if baseType.OID == uint32(oid.T_bool) && trimBool { + str = string(str[0]) + } + shouldQuote := false + for _, r := range str { + switch r { + case ' ', ',', '{', '}', '\\', '"': + shouldQuote = true + } + } + if shouldQuote || strings.EqualFold(str, "NULL") { + sb.WriteRune('"') + sb.WriteString(strings.ReplaceAll(str, `"`, `\"`)) + sb.WriteRune('"') + } else { + sb.WriteString(str) + } + } else { + sb.WriteString("NULL") + } } - - _, offsetSecsUnconverted := t.Zone() - return time.FixedZone(fmt.Sprintf("fixed offset:%d", offsetSecsUnconverted), -offsetSecsUnconverted), nil + sb.WriteRune('}') + return sb.String(), nil } diff --git a/server/types/uuid.go b/server/types/uuid.go index 7e394ca2f6..ad75cc3377 100644 --- a/server/types/uuid.go +++ b/server/types/uuid.go @@ -15,234 +15,43 @@ package types import ( - "bytes" - "fmt" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" - - "github.com/dolthub/doltgresql/postgres/parser/uuid" ) // Uuid is the UUID type. -var Uuid = UuidType{} - -// UuidType is the extended type implementation of the PostgreSQL UUID. -type UuidType struct{} - -var _ DoltgresType = UuidType{} - -// Alignment implements the DoltgresType interface. -func (b UuidType) Alignment() TypeAlignment { - return TypeAlignment_Char -} - -// BaseID implements the DoltgresType interface. -func (b UuidType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Uuid -} - -// BaseName implements the DoltgresType interface. -func (b UuidType) BaseName() string { - return "uuid" -} - -// Category implements the DoltgresType interface. -func (b UuidType) Category() TypeCategory { - return TypeCategory_UserDefinedTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b UuidType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b UuidType) Compare(v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } - - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(uuid.UUID) - bb := bc.(uuid.UUID) - return bytes.Compare(ab.GetBytesMut(), bb.GetBytesMut()), nil -} - -// Convert implements the DoltgresType interface. -func (b UuidType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case uuid.UUID: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b UuidType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b UuidType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b UuidType) GetSerializationID() SerializationID { - return SerializationID_Uuid -} - -// IoInput implements the DoltgresType interface. -func (b UuidType) IoInput(ctx *sql.Context, input string) (any, error) { - return uuid.FromString(input) -} - -// IoOutput implements the DoltgresType interface. -func (b UuidType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return converted.(uuid.UUID).String(), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b UuidType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b UuidType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b UuidType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b UuidType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 16 -} - -// OID implements the DoltgresType interface. -func (b UuidType) OID() uint32 { - return uint32(oid.T_uuid) -} - -// Promote implements the DoltgresType interface. -func (b UuidType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b UuidType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b UuidType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, _, err := b.Convert(v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value.(uuid.UUID).String()))), nil -} - -// String implements the DoltgresType interface. -func (b UuidType) String() string { - return "uuid" -} - -// ToArrayType implements the DoltgresType interface. -func (b UuidType) ToArrayType() DoltgresArrayType { - return UuidArray -} - -// Type implements the DoltgresType interface. -func (b UuidType) Type() query.Type { - return sqltypes.Text -} - -// ValueType implements the DoltgresType interface. -func (b UuidType) ValueType() reflect.Type { - return reflect.TypeOf(uuid.UUID{}) -} - -// Zero implements the DoltgresType interface. -func (b UuidType) Zero() any { - return uuid.UUID{} -} - -// SerializeType implements the DoltgresType interface. -func (b UuidType) SerializeType() ([]byte, error) { - return SerializationID_Uuid.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b UuidType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Uuid, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b UuidType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - return converted.(uuid.UUID).GetBytes(), nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b UuidType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - return uuid.FromBytes(val) +var Uuid = &DoltgresType{ + OID: uint32(oid.T_uuid), + Name: "uuid", + Schema: "pg_catalog", + TypLength: int16(16), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_UserDefinedTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__uuid), + InputFunc: toFuncID("uuid_in", oid.T_cstring), + OutputFunc: toFuncID("uuid_out", oid.T_uuid), + ReceiveFunc: toFuncID("uuid_recv", oid.T_internal), + SendFunc: toFuncID("uuid_send", oid.T_uuid), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Char, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("uuid_cmp", oid.T_uuid, oid.T_uuid), } diff --git a/server/types/uuid_array.go b/server/types/uuid_array.go index f33e22948c..dabf7b2c04 100644 --- a/server/types/uuid_array.go +++ b/server/types/uuid_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // UuidArray is the array variant of Uuid. -var UuidArray = createArrayType(Uuid, SerializationID_UuidArray, oid.T__uuid) +var UuidArray = CreateArrayTypeFromBaseType(Uuid) diff --git a/server/types/varchar.go b/server/types/varchar.go index 5f76e46b27..8034b2e3f9 100644 --- a/server/types/varchar.go +++ b/server/types/varchar.go @@ -15,19 +15,8 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "math" - "reflect" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" - - "github.com/dolthub/doltgresql/utils" + "gopkg.in/src-d/go-errors.v1" ) const ( @@ -35,297 +24,86 @@ const ( StringMaxLength = 10485760 // stringInline is the maximum number of characters (not bytes) that are "guaranteed" to fit inline. stringInline = 16383 - // stringUnbounded is used to represent that a type does not define a limit on the strings that it accepts. Values + // StringUnbounded is used to represent that a type does not define a limit on the strings that it accepts. Values // are still limited by the field size limit, but it won't be enforced by the type. - stringUnbounded = 0 + StringUnbounded = 0 ) -// VarChar is a varchar that has an unbounded length. -var VarChar = VarCharType{MaxChars: stringUnbounded} - -// VarCharType is the extended type implementation of the PostgreSQL varchar. -type VarCharType struct { - // MaxChars represents the maximum number of characters that the type may hold. - // When this is zero, we treat it as completely unbounded (which is still limited by the field size limit). - MaxChars uint32 -} - -var _ DoltgresType = VarCharType{} -var _ sql.StringType = VarCharType{} - -// Alignment implements the DoltgresType interface. -func (b VarCharType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b VarCharType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_VarChar -} - -// BaseName implements the DoltgresType interface. -func (b VarCharType) BaseName() string { - return "varchar" -} - -// Category implements the DoltgresType interface. -func (b VarCharType) Category() TypeCategory { - return TypeCategory_StringTypes -} - -// CharacterSet implements the sql.StringType interface. -func (b VarCharType) CharacterSet() sql.CharacterSetID { - return sql.CharacterSet_binary // TODO -} - -// Collation implements the sql.StringType interface. -func (b VarCharType) Collation() sql.CollationID { - return sql.Collation_Default // TODO -} - -// CollationCoercibility implements the DoltgresType interface. -func (b VarCharType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b VarCharType) Compare(v1 any, v2 any) (int, error) { - return compareVarChar(b, v1, v2) -} +// ErrLengthMustBeAtLeast1 is returned when given character length is less than 1. +var ErrLengthMustBeAtLeast1 = errors.NewKind(`length for type %s must be at least 1`) -func compareVarChar(b DoltgresType, v1 any, v2 any) (int, error) { - if v1 == nil && v2 == nil { - return 0, nil - } else if v1 != nil && v2 == nil { - return 1, nil - } else if v1 == nil && v2 != nil { - return -1, nil - } +// ErrLengthCannotExceed is returned when given character length exceeds the upper bound, 10485760. +var ErrLengthCannotExceed = errors.NewKind(`length for type %s cannot exceed 10485760`) - ac, _, err := b.Convert(v1) - if err != nil { - return 0, err - } - bc, _, err := b.Convert(v2) - if err != nil { - return 0, err - } - - ab := ac.(string) - bb := bc.(string) - if ab == bb { - return 0, nil - } else if ab < bb { - return -1, nil - } else { - return 1, nil - } -} - -// Convert implements the DoltgresType interface. -func (b VarCharType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case string: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b VarCharType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b VarCharType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b VarCharType) GetSerializationID() SerializationID { - return SerializationID_VarChar -} - -// IoInput implements the DoltgresType interface. -func (b VarCharType) IoInput(ctx *sql.Context, input string) (any, error) { - if b.IsUnbounded() { - return input, nil - } - input, runeLength := truncateString(input, b.MaxChars) - if runeLength > b.MaxChars { - return input, fmt.Errorf("value too long for type %s", b.String()) - } else { - return input, nil - } -} - -// IoOutput implements the DoltgresType interface. -func (b VarCharType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) +// VarChar is a varchar that has an unbounded length. +var VarChar = &DoltgresType{ + OID: uint32(oid.T_varchar), + Name: "varchar", + Schema: "pg_catalog", + TypLength: int16(-1), + PassedByVal: false, + TypType: TypeType_Base, + TypCategory: TypeCategory_StringTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__varchar), + InputFunc: toFuncID("varcharin", oid.T_cstring, oid.T_oid, oid.T_int4), + OutputFunc: toFuncID("varcharout", oid.T_varchar), + ReceiveFunc: toFuncID("varcharrecv", oid.T_internal, oid.T_oid, oid.T_int4), + SendFunc: toFuncID("varcharsend", oid.T_varchar), + ModInFunc: toFuncID("varchartypmodin", oid.T__cstring), + ModOutFunc: toFuncID("varchartypmodout", oid.T_int4), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Extended, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 100, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("bttextcmp", oid.T_text, oid.T_text), // TODO: temporarily added +} + +// NewVarCharType returns VarChar type with type modifier set +// representing the maximum number of characters that the type may hold. +func NewVarCharType(maxChars int32) (*DoltgresType, error) { + typmod, err := GetTypModFromCharLength("varchar", maxChars) if err != nil { - return "", err - } - if b.IsUnbounded() { - return converted.(string), nil - } - str, _ := truncateString(converted.(string), b.MaxChars) - return str, nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b VarCharType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b VarCharType) IsUnbounded() bool { - return b.MaxChars == stringUnbounded -} - -// Length implements the sql.StringType interface. -func (b VarCharType) Length() int64 { - return int64(b.MaxChars) -} - -// MaxByteLength implements the sql.StringType interface. -func (b VarCharType) MaxByteLength() int64 { - return b.Length() * 4 // TODO -} - -// MaxCharacterLength implements the sql.StringType interface. -func (b VarCharType) MaxCharacterLength() int64 { - return b.Length() -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b VarCharType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - if b.MaxChars != stringUnbounded && b.MaxChars <= stringInline { - return types.ExtendedTypeSerializedWidth_64K - } else { - return types.ExtendedTypeSerializedWidth_Unbounded - } -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b VarCharType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - if b.MaxChars == stringUnbounded { - return math.MaxUint32 - } else { - return b.MaxChars * 4 - } -} - -// OID implements the DoltgresType interface. -func (b VarCharType) OID() uint32 { - return uint32(oid.T_varchar) -} - -// Promote implements the DoltgresType interface. -func (b VarCharType) Promote() sql.Type { - return VarChar -} - -// SerializedCompare implements the DoltgresType interface. -func (b VarCharType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil + return nil, err } - return serializedStringCompare(v1, v2), nil + newType := *VarChar.WithAttTypMod(typmod) + return &newType, nil } -// SQL implements the DoltgresType interface. -func (b VarCharType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) +// MustCreateNewVarCharType panics if used with out-of-bound value. +func MustCreateNewVarCharType(maxChars int32) *DoltgresType { + newType, err := NewVarCharType(maxChars) if err != nil { - return sqltypes.Value{}, err + panic(err) } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil + return newType } -// String implements the DoltgresType interface. -func (b VarCharType) String() string { - if b.MaxChars == stringUnbounded { - return "varchar" +// GetTypModFromCharLength takes character type and its length and returns the type modifier value. +func GetTypModFromCharLength(typName string, l int32) (int32, error) { + if l < 1 { + return 0, ErrLengthMustBeAtLeast1.New(typName) + } else if l > StringMaxLength { + return 0, ErrLengthCannotExceed.New(typName) } - return fmt.Sprintf("varchar(%d)", b.MaxChars) -} - -// ToArrayType implements the DoltgresType interface. -func (b VarCharType) ToArrayType() DoltgresArrayType { - return createArrayType(b, SerializationID_VarCharArray, oid.T__varchar) -} - -// Type implements the DoltgresType interface. -func (b VarCharType) Type() query.Type { - return sqltypes.VarChar -} - -// ValueType implements the DoltgresType interface. -func (b VarCharType) ValueType() reflect.Type { - return reflect.TypeOf("") + return l + 4, nil } -// Zero implements the DoltgresType interface. -func (b VarCharType) Zero() any { - return "" -} - -// SerializeType implements the DoltgresType interface. -func (b VarCharType) SerializeType() ([]byte, error) { - t := make([]byte, serializationIDHeaderSize+4) - copy(t, SerializationID_VarChar.ToByteSlice(0)) - binary.LittleEndian.PutUint32(t[serializationIDHeaderSize:], b.MaxChars) - return t, nil -} - -// deserializeType implements the DoltgresType interface. -func (b VarCharType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return VarCharType{ - MaxChars: binary.LittleEndian.Uint32(metadata), - }, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b VarCharType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - str := converted.(string) - writer := utils.NewWriter(uint64(len(str) + 4)) - writer.String(str) - return writer.Data(), nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b VarCharType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - reader := utils.NewReader(val) - return reader.String(), nil +// GetCharLengthFromTypmod takes character type modifier and returns length value. +func GetCharLengthFromTypmod(typmod int32) int32 { + return typmod - 4 } diff --git a/server/types/varchar_array.go b/server/types/varchar_array.go index 5ee38feda0..2d88f8dde3 100644 --- a/server/types/varchar_array.go +++ b/server/types/varchar_array.go @@ -14,9 +14,5 @@ package types -import ( - "github.com/lib/pq/oid" -) - // VarCharArray is the array variant of VarChar. -var VarCharArray = createArrayType(VarChar, SerializationID_VarCharArray, oid.T__varchar) +var VarCharArray = CreateArrayTypeFromBaseType(VarChar) diff --git a/server/types/xid.go b/server/types/xid.go index 3a5bbf372e..e70deda658 100644 --- a/server/types/xid.go +++ b/server/types/xid.go @@ -15,222 +15,43 @@ package types import ( - "bytes" - "encoding/binary" - "fmt" - "reflect" - "strconv" - "strings" - - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/sqltypes" - "github.com/dolthub/vitess/go/vt/proto/query" "github.com/lib/pq/oid" ) // Xid is a data type used for internal transaction IDs. It is implemented as an unsigned 32 bit integer. -var Xid = XidType{} - -// XidType is the extended type implementation of the PostgreSQL xid. -type XidType struct{} - -var _ DoltgresType = XidType{} - -// Alignment implements the DoltgresType interface. -func (b XidType) Alignment() TypeAlignment { - return TypeAlignment_Int -} - -// BaseID implements the DoltgresType interface. -func (b XidType) BaseID() DoltgresTypeBaseID { - return DoltgresTypeBaseID_Xid -} - -// BaseName implements the DoltgresType interface. -func (b XidType) BaseName() string { - return "xid" -} - -// Category implements the DoltgresType interface. -func (b XidType) Category() TypeCategory { - return TypeCategory_UserDefinedTypes -} - -// CollationCoercibility implements the DoltgresType interface. -func (b XidType) CollationCoercibility(ctx *sql.Context) (collation sql.CollationID, coercibility byte) { - return sql.Collation_binary, 5 -} - -// Compare implements the DoltgresType interface. -func (b XidType) Compare(v1 any, v2 any) (int, error) { - return compareUint32(b, v1, v2) -} - -// Convert implements the DoltgresType interface. -func (b XidType) Convert(val any) (any, sql.ConvertInRange, error) { - switch val := val.(type) { - case uint32: - return val, sql.InRange, nil - case nil: - return nil, sql.InRange, nil - default: - return nil, sql.OutOfRange, fmt.Errorf("%s: unhandled type: %T", b.String(), val) - } -} - -// Equals implements the DoltgresType interface. -func (b XidType) Equals(otherType sql.Type) bool { - if otherExtendedType, ok := otherType.(types.ExtendedType); ok { - return bytes.Equal(MustSerializeType(b), MustSerializeType(otherExtendedType)) - } - return false -} - -// FormatValue implements the DoltgresType interface. -func (b XidType) FormatValue(val any) (string, error) { - if val == nil { - return "", nil - } - return b.IoOutput(sql.NewEmptyContext(), val) -} - -// GetSerializationID implements the DoltgresType interface. -func (b XidType) GetSerializationID() SerializationID { - return SerializationID_Xid -} - -// IoInput implements the DoltgresType interface. -func (b XidType) IoInput(ctx *sql.Context, input string) (any, error) { - val, err := strconv.ParseInt(strings.TrimSpace(input), 10, 64) - if err != nil { - return uint32(0), nil - } - return uint32(val), nil -} - -// IoOutput implements the DoltgresType interface. -func (b XidType) IoOutput(ctx *sql.Context, output any) (string, error) { - converted, _, err := b.Convert(output) - if err != nil { - return "", err - } - return strconv.FormatUint(uint64(converted.(uint32)), 10), nil -} - -// IsPreferredType implements the DoltgresType interface. -func (b XidType) IsPreferredType() bool { - return false -} - -// IsUnbounded implements the DoltgresType interface. -func (b XidType) IsUnbounded() bool { - return false -} - -// MaxSerializedWidth implements the DoltgresType interface. -func (b XidType) MaxSerializedWidth() types.ExtendedTypeSerializedWidth { - return types.ExtendedTypeSerializedWidth_64K -} - -// MaxTextResponseByteLength implements the DoltgresType interface. -func (b XidType) MaxTextResponseByteLength(ctx *sql.Context) uint32 { - return 4 -} - -// OID implements the DoltgresType interface. -func (b XidType) OID() uint32 { - return uint32(oid.T_xid) -} - -// Promote implements the DoltgresType interface. -func (b XidType) Promote() sql.Type { - return b -} - -// SerializedCompare implements the DoltgresType interface. -func (b XidType) SerializedCompare(v1 []byte, v2 []byte) (int, error) { - if len(v1) == 0 && len(v2) == 0 { - return 0, nil - } else if len(v1) > 0 && len(v2) == 0 { - return 1, nil - } else if len(v1) == 0 && len(v2) > 0 { - return -1, nil - } - - return bytes.Compare(v1, v2), nil -} - -// SQL implements the DoltgresType interface. -func (b XidType) SQL(ctx *sql.Context, dest []byte, v any) (sqltypes.Value, error) { - if v == nil { - return sqltypes.NULL, nil - } - value, err := b.IoOutput(ctx, v) - if err != nil { - return sqltypes.Value{}, err - } - return sqltypes.MakeTrusted(sqltypes.Text, types.AppendAndSliceBytes(dest, []byte(value))), nil -} - -// String implements the DoltgresType interface. -func (b XidType) String() string { - return "xid" -} - -// ToArrayType implements the DoltgresType interface. -func (b XidType) ToArrayType() DoltgresArrayType { - return XidArray -} - -// Type implements the DoltgresType interface. -func (b XidType) Type() query.Type { - return sqltypes.Uint32 -} - -// ValueType implements the DoltgresType interface. -func (b XidType) ValueType() reflect.Type { - return reflect.TypeOf(uint32(0)) -} - -// Zero implements the DoltgresType interface. -func (b XidType) Zero() any { - return uint32(0) -} - -// SerializeType implements the DoltgresType interface. -func (b XidType) SerializeType() ([]byte, error) { - return SerializationID_Xid.ToByteSlice(0), nil -} - -// deserializeType implements the DoltgresType interface. -func (b XidType) deserializeType(version uint16, metadata []byte) (DoltgresType, error) { - switch version { - case 0: - return Xid, nil - default: - return nil, fmt.Errorf("version %d is not yet supported for %s", version, b.String()) - } -} - -// SerializeValue implements the DoltgresType interface. -func (b XidType) SerializeValue(val any) ([]byte, error) { - if val == nil { - return nil, nil - } - converted, _, err := b.Convert(val) - if err != nil { - return nil, err - } - retVal := make([]byte, 4) - binary.BigEndian.PutUint32(retVal, uint32(converted.(uint32))) - return retVal, nil -} - -// DeserializeValue implements the DoltgresType interface. -func (b XidType) DeserializeValue(val []byte) (any, error) { - if len(val) == 0 { - return nil, nil - } - return uint32(binary.BigEndian.Uint32(val)), nil +var Xid = &DoltgresType{ + OID: uint32(oid.T_xid), + Name: "xid", + Schema: "pg_catalog", + TypLength: int16(4), + PassedByVal: true, + TypType: TypeType_Base, + TypCategory: TypeCategory_UserDefinedTypes, + IsPreferred: false, + IsDefined: true, + Delimiter: ",", + RelID: 0, + SubscriptFunc: toFuncID("-"), + Elem: 0, + Array: uint32(oid.T__xid), + InputFunc: toFuncID("xidin", oid.T_cstring), + OutputFunc: toFuncID("xidout", oid.T_xid), + ReceiveFunc: toFuncID("xidrecv", oid.T_internal), + SendFunc: toFuncID("xidsend", oid.T_xid), + ModInFunc: toFuncID("-"), + ModOutFunc: toFuncID("-"), + AnalyzeFunc: toFuncID("-"), + Align: TypeAlignment_Int, + Storage: TypeStorage_Plain, + NotNull: false, + BaseTypeOID: 0, + TypMod: -1, + NDims: 0, + TypCollation: 0, + DefaulBin: "", + Default: "", + Acl: nil, + Checks: nil, + attTypMod: -1, + CompareFunc: toFuncID("-"), } diff --git a/server/types/xid_array.go b/server/types/xid_array.go index fd54d3bff2..9a7e9841f4 100644 --- a/server/types/xid_array.go +++ b/server/types/xid_array.go @@ -14,7 +14,5 @@ package types -import "github.com/lib/pq/oid" - // XidArray is the array variant of Xid. -var XidArray = createArrayType(Xid, SerializationID_XidArray, oid.T__xid) +var XidArray = CreateArrayTypeFromBaseType(Xid) diff --git a/core/dataloader/csvdataloader_test.go b/testing/dataloader/csvdataloader_test.go similarity index 92% rename from core/dataloader/csvdataloader_test.go rename to testing/dataloader/csvdataloader_test.go index 937844947a..2f6653d33f 100644 --- a/core/dataloader/csvdataloader_test.go +++ b/testing/dataloader/csvdataloader_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dataloader +package _dataloader import ( "bufio" @@ -25,6 +25,8 @@ import ( "github.com/dolthub/go-mysql-server/sql" "github.com/stretchr/testify/require" + "github.com/dolthub/doltgresql/core/dataloader" + "github.com/dolthub/doltgresql/server/initialization" "github.com/dolthub/doltgresql/server/types" ) @@ -32,6 +34,7 @@ import ( func TestCsvDataLoader(t *testing.T) { db := memory.NewDatabase("mydb") provider := memory.NewDBProvider(db) + initialization.Initialize(nil) ctx := &sql.Context{ Context: context.Background(), @@ -47,7 +50,7 @@ func TestCsvDataLoader(t *testing.T) { // Tests that a basic CSV document can be loaded as a single chunk. t.Run("basic case", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewCsvDataLoader(ctx, table, ",", false) + dataLoader, err := dataloader.NewCsvDataLoader(ctx, table, ",", false) require.NoError(t, err) // Load all the data as a single chunk @@ -69,7 +72,7 @@ func TestCsvDataLoader(t *testing.T) { // partial record must be buffered and prepended to the next chunk. t.Run("record split across two chunks", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewCsvDataLoader(ctx, table, ",", false) + dataLoader, err := dataloader.NewCsvDataLoader(ctx, table, ",", false) require.NoError(t, err) // Load the first chunk @@ -98,7 +101,7 @@ func TestCsvDataLoader(t *testing.T) { // header row is present. t.Run("record split across two chunks, with header", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewCsvDataLoader(ctx, table, ",", true) + dataLoader, err := dataloader.NewCsvDataLoader(ctx, table, ",", true) require.NoError(t, err) // Load the first chunk @@ -127,7 +130,7 @@ func TestCsvDataLoader(t *testing.T) { // across two chunks. t.Run("quoted newlines across two chunks", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewCsvDataLoader(ctx, table, ",", false) + dataLoader, err := dataloader.NewCsvDataLoader(ctx, table, ",", false) require.NoError(t, err) // Load the first chunk @@ -155,7 +158,7 @@ func TestCsvDataLoader(t *testing.T) { // Test that calling Abort() does not insert any data into the table. t.Run("abort cancels data load", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewCsvDataLoader(ctx, table, ",", false) + dataLoader, err := dataloader.NewCsvDataLoader(ctx, table, ",", false) require.NoError(t, err) // Load the first chunk @@ -180,7 +183,7 @@ func TestCsvDataLoader(t *testing.T) { // and a header row is present. t.Run("delimiter='|', record split across two chunks, with header", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewCsvDataLoader(ctx, table, "|", true) + dataLoader, err := dataloader.NewCsvDataLoader(ctx, table, "|", true) require.NoError(t, err) // Load the first chunk diff --git a/core/dataloader/csvreader_test.go b/testing/dataloader/csvreader_test.go similarity index 90% rename from core/dataloader/csvreader_test.go rename to testing/dataloader/csvreader_test.go index 3934d3531b..11db2671ee 100644 --- a/core/dataloader/csvreader_test.go +++ b/testing/dataloader/csvreader_test.go @@ -12,13 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dataloader +package _dataloader import ( "bytes" "io" "testing" + "github.com/dolthub/doltgresql/core/dataloader" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -72,7 +74,7 @@ bash" // TestCsvReader tests various cases of CSV data parsing. func TestCsvReader(t *testing.T) { t.Run("basic CSV data", func(t *testing.T) { - csvReader, err := newCsvReader(newReader(basicCsvData)) + csvReader, err := dataloader.NewCsvReader(newReader(basicCsvData)) require.NoError(t, err) // Read the first row @@ -95,7 +97,7 @@ func TestCsvReader(t *testing.T) { }) t.Run("wrong number of fields", func(t *testing.T) { - csvReader, err := newCsvReader(newReader(wrongNumberOfFieldsCsvData)) + csvReader, err := dataloader.NewCsvReader(newReader(wrongNumberOfFieldsCsvData)) require.NoError(t, err) // Read the first row @@ -114,7 +116,7 @@ func TestCsvReader(t *testing.T) { }) t.Run("incomplete line, no newline ending", func(t *testing.T) { - csvReader, err := newCsvReader(newReader(partialLineErrorCsvData)) + csvReader, err := dataloader.NewCsvReader(newReader(partialLineErrorCsvData)) require.NoError(t, err) // Read the first row @@ -142,7 +144,7 @@ func TestCsvReader(t *testing.T) { }) t.Run("null and empty string quoting", func(t *testing.T) { - csvReader, err := newCsvReader(newReader(nullAndEmptyStringQuotingCsvData)) + csvReader, err := dataloader.NewCsvReader(newReader(nullAndEmptyStringQuotingCsvData)) require.NoError(t, err) // Read the first row @@ -160,7 +162,7 @@ func TestCsvReader(t *testing.T) { }) t.Run("quote escaping", func(t *testing.T) { - csvReader, err := newCsvReader(newReader(escapedQuotesCsvData)) + csvReader, err := dataloader.NewCsvReader(newReader(escapedQuotesCsvData)) require.NoError(t, err) // Read the first row @@ -179,7 +181,7 @@ func TestCsvReader(t *testing.T) { }) t.Run("quoted newlines", func(t *testing.T) { - csvReader, err := newCsvReader(newReader(newLineInQuotedFieldCsvData)) + csvReader, err := dataloader.NewCsvReader(newReader(newLineInQuotedFieldCsvData)) require.NoError(t, err) // Read the first row @@ -195,7 +197,7 @@ func TestCsvReader(t *testing.T) { }) t.Run("quoted end of data marker", func(t *testing.T) { - csvReader, err := newCsvReader(newReader(endOfDataMarkerCsvData)) + csvReader, err := dataloader.NewCsvReader(newReader(endOfDataMarkerCsvData)) require.NoError(t, err) // Read the first row diff --git a/core/dataloader/string_prefix_reader_test.go b/testing/dataloader/string_prefix_reader_test.go similarity index 91% rename from core/dataloader/string_prefix_reader_test.go rename to testing/dataloader/string_prefix_reader_test.go index 47bff70062..1f309d8c14 100644 --- a/core/dataloader/string_prefix_reader_test.go +++ b/testing/dataloader/string_prefix_reader_test.go @@ -12,13 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dataloader +package _dataloader import ( "bytes" "io" "testing" + "github.com/dolthub/doltgresql/core/dataloader" + "github.com/stretchr/testify/require" ) @@ -26,7 +28,7 @@ func TestStringPrefixReader(t *testing.T) { t.Run("Read prefix and all data in single call", func(t *testing.T) { prefix := "prefix" reader := bytes.NewReader([]byte("0123456789")) - prefixReader := newStringPrefixReader(prefix, reader) + prefixReader := dataloader.NewStringPrefixReader(prefix, reader) data := make([]byte, 100) bytesRead, err := prefixReader.Read(data) @@ -42,7 +44,7 @@ func TestStringPrefixReader(t *testing.T) { t.Run("Read part of prefix", func(t *testing.T) { prefix := "prefix" reader := bytes.NewReader([]byte("0123456789")) - prefixReader := newStringPrefixReader(prefix, reader) + prefixReader := dataloader.NewStringPrefixReader(prefix, reader) data := make([]byte, 5) bytesRead, err := prefixReader.Read(data) @@ -77,7 +79,7 @@ func TestStringPrefixReader(t *testing.T) { t.Run("Read to prefix boundary", func(t *testing.T) { prefix := "prefix" reader := bytes.NewReader([]byte("0123456789")) - prefixReader := newStringPrefixReader(prefix, reader) + prefixReader := dataloader.NewStringPrefixReader(prefix, reader) data := make([]byte, 6) bytesRead, err := prefixReader.Read(data) diff --git a/core/dataloader/tabdataloader_test.go b/testing/dataloader/tabdataloader_test.go similarity index 91% rename from core/dataloader/tabdataloader_test.go rename to testing/dataloader/tabdataloader_test.go index 5adea47ecc..61e8c4934c 100644 --- a/core/dataloader/tabdataloader_test.go +++ b/testing/dataloader/tabdataloader_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dataloader +package _dataloader import ( "bufio" @@ -24,6 +24,7 @@ import ( "github.com/dolthub/go-mysql-server/sql" "github.com/stretchr/testify/require" + "github.com/dolthub/doltgresql/core/dataloader" "github.com/dolthub/doltgresql/server/types" ) @@ -45,7 +46,7 @@ func TestTabDataLoader(t *testing.T) { // Tests that a basic tab delimited doc can be loaded as a single chunk. t.Run("basic case", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewTabularDataLoader(ctx, table, "\t", "\\N", false) + dataLoader, err := dataloader.NewTabularDataLoader(ctx, table, "\t", "\\N", false) require.NoError(t, err) // Load all the data as a single chunk @@ -67,7 +68,7 @@ func TestTabDataLoader(t *testing.T) { // partial record must be buffered and prepended to the next chunk. t.Run("record split across two chunks", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewTabularDataLoader(ctx, table, "\t", "\\N", false) + dataLoader, err := dataloader.NewTabularDataLoader(ctx, table, "\t", "\\N", false) require.NoError(t, err) // Load the first chunk @@ -96,7 +97,7 @@ func TestTabDataLoader(t *testing.T) { // header row is present. t.Run("record split across two chunks, with header", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewTabularDataLoader(ctx, table, "\t", "\\N", true) + dataLoader, err := dataloader.NewTabularDataLoader(ctx, table, "\t", "\\N", true) require.NoError(t, err) // Load the first chunk @@ -125,7 +126,7 @@ func TestTabDataLoader(t *testing.T) { // across two chunks. t.Run("quoted newlines across two chunks", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewTabularDataLoader(ctx, table, "\t", "\\N", false) + dataLoader, err := dataloader.NewTabularDataLoader(ctx, table, "\t", "\\N", false) require.NoError(t, err) // Load the first chunk @@ -154,7 +155,7 @@ func TestTabDataLoader(t *testing.T) { // header row is present. t.Run("delimiter='|', record split across two chunks, with header", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewTabularDataLoader(ctx, table, "|", "\\N", true) + dataLoader, err := dataloader.NewTabularDataLoader(ctx, table, "|", "\\N", true) require.NoError(t, err) // Load the first chunk @@ -182,7 +183,7 @@ func TestTabDataLoader(t *testing.T) { // Test that calling Abort() does not insert any data into the table. t.Run("abort cancels data load", func(t *testing.T) { table := memory.NewTable(db, "myTable", pkSchema, nil) - dataLoader, err := NewTabularDataLoader(ctx, table, "\t", "\\N", false) + dataLoader, err := dataloader.NewTabularDataLoader(ctx, table, "\t", "\\N", false) require.NoError(t, err) // Load the first chunk diff --git a/testing/generation/function_coverage/generators.go b/testing/generation/function_coverage/generators.go index 2d993c9bde..91d339e45a 100644 --- a/testing/generation/function_coverage/generators.go +++ b/testing/generation/function_coverage/generators.go @@ -166,14 +166,14 @@ var uuidValueGenerators = utils.Or( ) // valueMappings contains the value generators for the given type. -var valueMappings = map[pgtypes.DoltgresTypeBaseID]utils.StatementGenerator{ - pgtypes.Bool.BaseID(): booleanValueGenerators, - pgtypes.Float32.BaseID(): float32ValueGenerators, - pgtypes.Float64.BaseID(): float64ValueGenerators, - pgtypes.Int16.BaseID(): int16ValueGenerators, - pgtypes.Int32.BaseID(): int32ValueGenerators, - pgtypes.Int64.BaseID(): int64ValueGenerators, - pgtypes.Numeric.BaseID(): numericValueGenerators, - pgtypes.Uuid.BaseID(): uuidValueGenerators, - pgtypes.VarChar.BaseID(): stringValueGenerators, +var valueMappings = map[uint32]utils.StatementGenerator{ + pgtypes.Bool.OID: booleanValueGenerators, + pgtypes.Float32.OID: float32ValueGenerators, + pgtypes.Float64.OID: float64ValueGenerators, + pgtypes.Int16.OID: int16ValueGenerators, + pgtypes.Int32.OID: int32ValueGenerators, + pgtypes.Int64.OID: int64ValueGenerators, + pgtypes.Numeric.OID: numericValueGenerators, + pgtypes.Uuid.OID: uuidValueGenerators, + pgtypes.VarChar.OID: stringValueGenerators, } diff --git a/testing/generation/function_coverage/main.go b/testing/generation/function_coverage/main.go index 187c5992b9..c1e931d541 100644 --- a/testing/generation/function_coverage/main.go +++ b/testing/generation/function_coverage/main.go @@ -61,7 +61,7 @@ func main() { if i > 0 { literalGeneratorParams = append(literalGeneratorParams, utils.Text(", ")) } - if generator, ok := valueMappings[paramType.BaseID()]; ok { + if generator, ok := valueMappings[paramType.OID]; ok { literalGeneratorParams = append(literalGeneratorParams, generator) } else { fmt.Printf("missing support for functions with the parameter type: `%s`\n", paramType.String()) diff --git a/testing/go/alter_table_test.go b/testing/go/alter_table_test.go index d662c10c9d..ff835245ac 100644 --- a/testing/go/alter_table_test.go +++ b/testing/go/alter_table_test.go @@ -193,6 +193,14 @@ func TestAlterTable(t *testing.T) { Query: "select * from test1;", Expected: []sql.Row{{1, 1, 42}}, }, + { + Query: "ALTER TABLE test1 ADD COLUMN l non_existing_type;", + ExpectedErr: `type "non_existing_type" does not exist`, + }, + { + Query: `ALTER TABLE test1 ADD COLUMN m xid;`, + Expected: []sql.Row{}, + }, }, }, { diff --git a/testing/go/framework.go b/testing/go/framework.go index b9b6b0d31a..23e8350ff9 100644 --- a/testing/go/framework.go +++ b/testing/go/framework.go @@ -30,6 +30,7 @@ import ( "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" + "github.com/lib/pq/oid" "github.com/shopspring/decimal" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -356,7 +357,9 @@ func NormalizeRow(fds []pgconn.FieldDescription, row sql.Row, normalize bool) sq } newRow := make(sql.Row, len(row)) for i := range row { - dt, ok := types.OidToBuildInDoltgresType[fds[i].DataTypeOID] + dt, ok := types.OidToBuiltInDoltgresType[fds[i].DataTypeOID] + // TODO: need to set the typmod! + dt = dt.WithAttTypMod(-1) if !ok { panic(fmt.Sprintf("unhandled oid type: %v", fds[i].DataTypeOID)) } @@ -380,14 +383,15 @@ func NormalizeExpectedRow(fds []pgconn.FieldDescription, rows []sql.Row) []sql.R } else { newRow := make(sql.Row, len(row)) for i := range row { - dt, ok := types.OidToBuildInDoltgresType[fds[i].DataTypeOID] + dt, ok := types.OidToBuiltInDoltgresType[fds[i].DataTypeOID] if !ok { panic(fmt.Sprintf("unhandled oid type: %v", fds[i].DataTypeOID)) } - if dt == types.Json { + if dt.OID == uint32(oid.T_json) { newRow[i] = UnmarshalAndMarshalJsonString(row[i].(string)) - } else if dta, ok := dt.(types.DoltgresArrayType); ok && dta.BaseType() == types.Json { - v, err := dta.IoInput(nil, row[i].(string)) + } else if dt.IsArrayType() && dt.ArrayBaseType().OID == uint32(oid.T_json) { + // TODO: need to have valid sql.Context + v, err := dt.IoInput(nil, row[i].(string)) if err != nil { panic(err) } @@ -396,7 +400,7 @@ func NormalizeExpectedRow(fds []pgconn.FieldDescription, rows []sql.Row) []sql.R for j, el := range arr { newArr[j] = UnmarshalAndMarshalJsonString(el.(string)) } - ret, err := dt.IoOutput(nil, newArr) + ret, err := dt.FormatValue(newArr) if err != nil { panic(err) } @@ -434,29 +438,29 @@ func UnmarshalAndMarshalJsonString(val string) string { // |normalizeNumeric| defines whether to normalize Numeric values into either Numeric type or string type. // There are an infinite number of ways to represent the same value in-memory, // so we must at least normalize Numeric values. -func NormalizeValToString(dt types.DoltgresType, v any) any { - switch t := dt.(type) { - case types.JsonType: +func NormalizeValToString(dt *types.DoltgresType, v any) any { + switch oid.Oid(dt.OID) { + case oid.T_json: str, err := json.Marshal(v) if err != nil { panic(err) } - ret, err := t.IoOutput(nil, string(str)) + ret, err := dt.FormatValue(string(str)) if err != nil { panic(err) } return ret - case types.JsonBType: - jv, err := t.ConvertToJsonDocument(v) + case oid.T_jsonb: + jv, err := types.ConvertToJsonDocument(v) if err != nil { panic(err) } - str, err := t.IoOutput(nil, types.JsonDocument{Value: jv}) + str, err := dt.FormatValue(types.JsonDocument{Value: jv}) if err != nil { panic(err) } return str - case types.InternalCharType: + case oid.T_char: if v == nil { return nil } @@ -466,24 +470,24 @@ func NormalizeValToString(dt types.DoltgresType, v any) any { } else { b = []byte{uint8(v.(int32))} } - val, err := t.IoOutput(nil, string(b)) + val, err := dt.FormatValue(string(b)) if err != nil { panic(err) } return val - case types.IntervalType, types.UuidType, types.DateType, types.TimeType, types.TimestampType: + case oid.T_interval, oid.T_uuid, oid.T_date, oid.T_time, oid.T_timestamp: // These values need to be normalized into the appropriate types // before being converted to string type using the Doltgres // IoOutput method. if v == nil { return nil } - tVal, err := dt.IoOutput(nil, NormalizeVal(dt, v)) + tVal, err := dt.FormatValue(NormalizeVal(dt, v)) if err != nil { panic(err) } return tVal - case types.TimestampTZType: + case oid.T_timestamptz: // timestamptz returns a value in server timezone _, offset := v.(time.Time).Zone() if offset%3600 != 0 { @@ -512,8 +516,8 @@ func NormalizeValToString(dt types.DoltgresType, v any) any { return Numeric(decStr) } case []any: - if dta, ok := dt.(types.DoltgresArrayType); ok { - return NormalizeArrayType(dta, val) + if dt.IsArrayType() { + return NormalizeArrayType(dt, val) } } return v @@ -521,20 +525,20 @@ func NormalizeValToString(dt types.DoltgresType, v any) any { // NormalizeArrayType normalizes array types by normalizing its elements first, // then to a string using the type IoOutput method. -func NormalizeArrayType(dta types.DoltgresArrayType, arr []any) any { +func NormalizeArrayType(dt *types.DoltgresType, arr []any) any { newVal := make([]any, len(arr)) for i, el := range arr { - newVal[i] = NormalizeVal(dta.BaseType(), el) + newVal[i] = NormalizeVal(dt.ArrayBaseType(), el) } - baseType := dta.BaseType() - if baseType == types.Bool { - sqlVal, err := dta.SQL(nil, nil, newVal) + baseType := dt.ArrayBaseType() + if baseType.OID == uint32(oid.T_bool) { + sqlVal, err := dt.SQL(sql.NewEmptyContext(), nil, newVal) if err != nil { panic(err) } return sqlVal.ToString() } else { - ret, err := dta.IoOutput(nil, newVal) + ret, err := dt.FormatValue(newVal) if err != nil { panic(err) } @@ -545,16 +549,16 @@ func NormalizeArrayType(dta types.DoltgresArrayType, arr []any) any { // NormalizeVal normalizes values to the Doltgres type expects, so it can be used to // convert the values using the given Doltgres type. This is used to normalize array // types as the type conversion expects certain type values. -func NormalizeVal(dt types.DoltgresType, v any) any { - switch t := dt.(type) { - case types.JsonType: +func NormalizeVal(dt *types.DoltgresType, v any) any { + switch oid.Oid(dt.OID) { + case oid.T_json: str, err := json.Marshal(v) if err != nil { panic(err) } return string(str) - case types.JsonBType: - jv, err := t.ConvertToJsonDocument(v) + case oid.T_jsonb: + jv, err := types.ConvertToJsonDocument(v) if err != nil { panic(err) } @@ -587,8 +591,8 @@ func NormalizeVal(dt types.DoltgresType, v any) any { return u case []any: baseType := dt - if dta, ok := baseType.(types.DoltgresArrayType); ok { - baseType = dta.BaseType() + if baseType.IsArrayType() { + baseType = baseType.ArrayBaseType() } newVal := make([]any, len(val)) for i, el := range val { diff --git a/testing/go/pgcatalog_test.go b/testing/go/pgcatalog_test.go index e1f4f6cfb0..f6728bc1e2 100644 --- a/testing/go/pgcatalog_test.go +++ b/testing/go/pgcatalog_test.go @@ -553,7 +553,11 @@ func TestPgClass(t *testing.T) { // TODO: Now that catalog data is cached for each query, this query no longer iterates the database // 100k times, and this query executes in a couple seconds. This is still slow and should // be improved with lookup index support now that we have cached data available. - Query: `SELECT ix.relname AS index_name, upper(am.amname) AS index_algorithm FROM pg_index i JOIN pg_class t ON t.oid = i.indrelid JOIN pg_class ix ON ix.oid = i.indexrelid JOIN pg_namespace n ON t.relnamespace = n.oid JOIN pg_am AS am ON ix.relam = am.oid WHERE t.relname = 'foo' AND n.nspname = 'public';`, + Query: `SELECT ix.relname AS index_name, upper(am.amname) AS index_algorithm FROM pg_index i +JOIN pg_class t ON t.oid = i.indrelid +JOIN pg_class ix ON ix.oid = i.indexrelid +JOIN pg_namespace n ON t.relnamespace = n.oid +JOIN pg_am AS am ON ix.relam = am.oid WHERE t.relname = 'foo' AND n.nspname = 'public';`, Expected: []sql.Row{{"foo_pkey", "BTREE"}, {"b", "BTREE"}, {"b_2", "BTREE"}}, // TODO: should follow Postgres index naming convention: "foo_pkey", "foo_b_idx", "foo_b_a_idx" }, }, @@ -3811,7 +3815,7 @@ func TestPgType(t *testing.T) { Assertions: []ScriptTestAssertion{ { Query: `SELECT * FROM "pg_catalog"."pg_type" WHERE typname = 'float8';`, - Expected: []sql.Row{{701, "float8", 1879048194, 0, 8, "t", "b", "N", "t", "t", ",", 0, "-", 0, 0, "float8in", "float8out", "float8recv", "float8send", "-", "-", "-", "d", "x", "f", 0, 0, 0, 0, nil, nil, nil}}, + Expected: []sql.Row{{701, "float8", 1879048194, 0, 8, "t", "b", "N", "t", "t", ",", 0, "-", 0, 1022, "float8in", "float8out", "float8recv", "float8send", "-", "-", "-", "d", "p", "f", 0, -1, 0, 0, "", "", "{}"}}, }, { // Different cases and quoted, so it fails Query: `SELECT * FROM "PG_catalog"."pg_type";`, @@ -3832,6 +3836,13 @@ func TestPgType(t *testing.T) { {"varchar"}, }, }, + { + Skip: true, // TODO: use regproc type instead of text type. + Query: `SELECT t1.oid, t1.typname as basetype, t2.typname as arraytype, t2.typsubscript + FROM pg_type t1 LEFT JOIN pg_type t2 ON (t1.typarray = t2.oid) + WHERE t1.typarray <> 0 AND (t2.oid IS NULL OR t2.typsubscript <> 'array_subscript_handler'::regproc);`, + Expected: []sql.Row{}, + }, }, }, { @@ -3839,7 +3850,7 @@ func TestPgType(t *testing.T) { Assertions: []ScriptTestAssertion{ { Query: `SELECT * FROM "pg_catalog"."pg_type" WHERE oid='float8'::regtype;`, - Expected: []sql.Row{{701, "float8", 1879048194, 0, 8, "t", "b", "N", "t", "t", ",", 0, "-", 0, 0, "float8in", "float8out", "float8recv", "float8send", "-", "-", "-", "d", "x", "f", 0, 0, 0, 0, nil, nil, nil}}, + Expected: []sql.Row{{701, "float8", 1879048194, 0, 8, "t", "b", "N", "t", "t", ",", 0, "-", 0, 1022, "float8in", "float8out", "float8recv", "float8send", "-", "-", "-", "d", "p", "f", 0, -1, 0, 0, "", "", "{}"}}, }, { Query: `SELECT oid, typname FROM "pg_catalog"."pg_type" WHERE oid='double precision'::regtype;`, @@ -3887,27 +3898,27 @@ func TestPgType(t *testing.T) { }, { Query: `SELECT * FROM "pg_catalog"."pg_type" WHERE oid='integer[]'::regtype;`, - Expected: []sql.Row{{1007, "_int4", 1879048194, 0, -1, "f", "b", "A", "f", "t", ",", 0, "array_subscript_handler", 0, 0, "array_in", "array_out", "array_recv", "array_send", "-", "-", "array_typanalyze", "i", "x", "f", 0, 0, 0, 0, nil, nil, nil}}, + Expected: []sql.Row{{1007, "_int4", 1879048194, 0, -1, "f", "b", "A", "f", "t", ",", 0, "array_subscript_handler", 23, 0, "array_in", "array_out", "array_recv", "array_send", "-", "-", "array_typanalyze", "i", "x", "f", 0, -1, 0, 0, "", "", "{}"}}, }, { Query: `SELECT * FROM "pg_catalog"."pg_type" WHERE oid='anyarray'::regtype;`, - Expected: []sql.Row{{2277, "anyarray", 1879048194, 0, -1, "f", "p", "P", "f", "t", ",", 0, "-", 0, 0, "anyarray_in", "anyarray_out", "anyarray_recv", "anyarray_send", "-", "-", "-", "d", "x", "f", 0, 0, 0, 0, nil, nil, nil}}, + Expected: []sql.Row{{2277, "anyarray", 1879048194, 0, -1, "f", "p", "P", "f", "t", ",", 0, "-", 0, 0, "anyarray_in", "anyarray_out", "anyarray_recv", "anyarray_send", "-", "-", "-", "d", "x", "f", 0, -1, 0, 0, "", "", "{}"}}, }, { Query: `SELECT * FROM "pg_catalog"."pg_type" WHERE oid='anyelement'::regtype;`, - Expected: []sql.Row{{2283, "anyelement", 1879048194, 0, -1, "t", "p", "P", "f", "t", ",", 0, "-", 0, 0, "anyelement_in", "anyelement_out", "-", "-", "-", "-", "-", "i", "p", "f", 0, 0, 0, 0, nil, nil, nil}}, + Expected: []sql.Row{{2283, "anyelement", 1879048194, 0, 4, "t", "p", "P", "f", "t", ",", 0, "-", 0, 0, "anyelement_in", "anyelement_out", "-", "-", "-", "-", "-", "i", "p", "f", 0, -1, 0, 0, "", "", "{}"}}, }, { Query: `SELECT * FROM "pg_catalog"."pg_type" WHERE oid='json'::regtype;`, - Expected: []sql.Row{{114, "json", 1879048194, 0, -1, "f", "b", "U", "f", "t", ",", 0, "-", 0, 0, "json_in", "json_out", "json_recv", "json_send", "-", "-", "-", "i", "x", "f", 0, 0, 0, 0, nil, nil, nil}}, + Expected: []sql.Row{{114, "json", 1879048194, 0, -1, "f", "b", "U", "f", "t", ",", 0, "-", 0, 199, "json_in", "json_out", "json_recv", "json_send", "-", "-", "-", "i", "x", "f", 0, -1, 0, 0, "", "", "{}"}}, }, { Query: `SELECT * FROM "pg_catalog"."pg_type" WHERE oid='char'::regtype;`, - Expected: []sql.Row{{1042, "bpchar", 1879048194, 0, -1, "f", "b", "S", "f", "t", ",", 0, "-", 0, 0, "bpcharin", "bpcharout", "bpcharrecv", "bpcharsend", "bpchartypmodin", "bpchartypmodout", "-", "i", "x", "f", 0, 0, 0, 0, nil, nil, nil}}, + Expected: []sql.Row{{1042, "bpchar", 1879048194, 0, -1, "f", "b", "S", "f", "t", ",", 0, "-", 0, 1014, "bpcharin", "bpcharout", "bpcharrecv", "bpcharsend", "bpchartypmodin", "bpchartypmodout", "-", "i", "x", "f", 0, -1, 0, 100, "", "", "{}"}}, }, { Query: `SELECT * FROM "pg_catalog"."pg_type" WHERE oid='"char"'::regtype;`, - Expected: []sql.Row{{18, "char", 1879048194, 0, 1, "t", "b", "Z", "f", "t", ",", 0, "-", 0, 0, "charin", "charout", "charrecv", "charsend", "-", "-", "-", "c", "p", "f", 0, 0, 0, 0, nil, nil, nil}}, + Expected: []sql.Row{{18, "char", 1879048194, 0, 1, "t", "b", "Z", "f", "t", ",", 0, "-", 0, 1002, "charin", "charout", "charrecv", "charsend", "-", "-", "-", "c", "p", "f", 0, -1, 0, 0, "", "", "{}"}}, }, }, }, diff --git a/testing/go/prepared_statement_test.go b/testing/go/prepared_statement_test.go index bb89a408a4..9596047b7c 100755 --- a/testing/go/prepared_statement_test.go +++ b/testing/go/prepared_statement_test.go @@ -379,6 +379,21 @@ var preparedStatementTests = []ScriptTest{ }, }, }, + { + Name: "pg_get_viewdef function", + SetUpScript: []string{ + "CREATE TABLE test (id int, name text)", + "INSERT INTO test VALUES (1,'desk'), (2,'chair')", + "CREATE VIEW test_view AS SELECT name FROM test", + }, + Assertions: []ScriptTestAssertion{ + { + Query: `select pg_get_viewdef($1::regclass);`, + BindVars: []any{"test_view"}, + Expected: []sql.Row{{"SELECT name FROM test"}}, + }, + }, + }, } var pgCatalogTests = []ScriptTest{ diff --git a/testing/go/regression_test.go b/testing/go/regression_test.go index 1c9b83f798..a86fb4ae71 100755 --- a/testing/go/regression_test.go +++ b/testing/go/regression_test.go @@ -232,5 +232,31 @@ func TestRegressions(t *testing.T) { }, }, }, + { + Name: "inner join", + SetUpScript: []string{ + "CREATE TABLE J1_TBL (i integer, j integer, t text);", + "CREATE TABLE J2_TBL (i integer, k integer);", + }, + Assertions: []ScriptTestAssertion{ + { + Query: "SELECT * FROM J1_TBL INNER JOIN J2_TBL USING (i);", + Expected: []sql.Row{}, + }, + }, + }, + { + Skip: true, // https://github.com/dolthub/doltgresql/issues/1043 + Name: "use column in function when creating view", + SetUpScript: []string{ + "CREATE TABLE base_tbl (a int PRIMARY KEY, b text DEFAULT 'Unspecified');", + }, + Assertions: []ScriptTestAssertion{ + { + Query: "CREATE VIEW rw_view15 AS SELECT a, upper(b) FROM base_tbl;", + Expected: []sql.Row{}, + }, + }, + }, }) } diff --git a/testing/go/types_test.go b/testing/go/types_test.go index 2d161cdb51..ec1698969a 100644 --- a/testing/go/types_test.go +++ b/testing/go/types_test.go @@ -1546,6 +1546,7 @@ var typesTests = []ScriptTest{ SetUpScript: []string{ "CREATE TABLE t_numeric (id INTEGER primary key, v1 NUMERIC(5,2));", "INSERT INTO t_numeric VALUES (1, 123.45), (2, 67.89), (3, 100.3);", + "CREATE TABLE fract_only (id int, val numeric(4,4));", }, Assertions: []ScriptTestAssertion{ { @@ -1556,6 +1557,10 @@ var typesTests = []ScriptTest{ {3, Numeric("100.30")}, }, }, + { + Query: "INSERT INTO fract_only VALUES (1, '0.0');", + Expected: []sql.Row{}, + }, { Query: "SELECT numeric '10.00';", Expected: []sql.Row{{Numeric("10.00")}}, @@ -1564,6 +1569,18 @@ var typesTests = []ScriptTest{ Query: "SELECT numeric '-10.00';", Expected: []sql.Row{{Numeric("-10.00")}}, }, + { + Query: "select 0.03::numeric(3,3);", + Expected: []sql.Row{{Numeric("0.030")}}, + }, + { + Query: "select 1.03::numeric(2,2);", + ExpectedErr: `numeric field overflow`, + }, + { + Query: "select 1.03::float4::numeric(2,2);", + ExpectedErr: `numeric field overflow`, + }, }, }, { diff --git a/testing/postgres-client-tests/node/fields.js b/testing/postgres-client-tests/node/fields.js index 65215a32a2..b533d65377 100644 --- a/testing/postgres-client-tests/node/fields.js +++ b/testing/postgres-client-tests/node/fields.js @@ -367,7 +367,7 @@ export const schemaNameField = { columnID: 0, dataTypeID: 1043, dataTypeSize: 256, - dataTypeModifier: -1, + dataTypeModifier: 68, format: "text", }; @@ -377,7 +377,7 @@ export const pgTablesFields = [ tableID: 0, columnID: 0, dataTypeID: 19, - dataTypeSize: 252, + dataTypeSize: 64, dataTypeModifier: -1, format: "text", }, @@ -386,7 +386,7 @@ export const pgTablesFields = [ tableID: 0, columnID: 0, dataTypeID: 19, - dataTypeSize: 252, + dataTypeSize: 64, dataTypeModifier: -1, format: "text", }, diff --git a/testing/postgres-client-tests/node/workbenchTests/databases.js b/testing/postgres-client-tests/node/workbenchTests/databases.js index ed545af150..b534e9e8a1 100644 --- a/testing/postgres-client-tests/node/workbenchTests/databases.js +++ b/testing/postgres-client-tests/node/workbenchTests/databases.js @@ -35,7 +35,7 @@ export const databaseTests = [ tableID: 0, columnID: 0, dataTypeID: 19, - dataTypeSize: 252, + dataTypeSize: 64, dataTypeModifier: -1, format: "text", }, @@ -69,7 +69,7 @@ export const databaseTests = [ tableID: 0, columnID: 0, dataTypeID: 19, - dataTypeSize: 252, + dataTypeSize: 64, dataTypeModifier: -1, format: "text", }, diff --git a/testing/postgres-client-tests/node/workbenchTests/table.js b/testing/postgres-client-tests/node/workbenchTests/table.js index b4256fea09..0422e30dab 100644 --- a/testing/postgres-client-tests/node/workbenchTests/table.js +++ b/testing/postgres-client-tests/node/workbenchTests/table.js @@ -25,7 +25,7 @@ const testInfoFields = [ columnID: 0, dataTypeID: 1043, dataTypeSize: 1020, - dataTypeModifier: -1, + dataTypeModifier: 259, format: "text", }, { @@ -109,7 +109,7 @@ export const tableTests = [ columnID: 0, dataTypeID: 1043, dataTypeSize: 256, - dataTypeModifier: -1, + dataTypeModifier: 68, format: "text", }, { @@ -118,7 +118,7 @@ export const tableTests = [ columnID: 0, dataTypeID: 1043, dataTypeSize: 256, - dataTypeModifier: -1, + dataTypeModifier: 68, format: "text", }, { @@ -127,7 +127,7 @@ export const tableTests = [ columnID: 0, dataTypeID: 1043, dataTypeSize: 12, - dataTypeModifier: -1, + dataTypeModifier: 7, format: "text", }, { @@ -225,7 +225,7 @@ export const tableTests = [ columnID: 0, dataTypeID: 1043, dataTypeSize: 256, - dataTypeModifier: -1, + dataTypeModifier: 68, format: "text", }, { @@ -234,7 +234,7 @@ export const tableTests = [ columnID: 0, dataTypeID: 1043, dataTypeSize: 256, - dataTypeModifier: -1, + dataTypeModifier: 68, format: "text", }, ], diff --git a/testing/postgres-client-tests/node/workbenchTests/views.js b/testing/postgres-client-tests/node/workbenchTests/views.js index 40a0b454e8..29eee7cd4f 100644 --- a/testing/postgres-client-tests/node/workbenchTests/views.js +++ b/testing/postgres-client-tests/node/workbenchTests/views.js @@ -92,7 +92,7 @@ export const viewsTests = [ columnID: 0, dataTypeID: 1043, dataTypeSize: 256, - dataTypeModifier: -1, + dataTypeModifier: 68, format: "text", }, ], @@ -113,7 +113,7 @@ export const viewsTests = [ tableID: 0, columnID: 0, dataTypeID: 19, - dataTypeSize: 252, + dataTypeSize: 64, dataTypeModifier: -1, format: "text", }, @@ -148,7 +148,7 @@ export const viewsTests = [ columnID: 0, dataTypeID: 1043, dataTypeSize: 256, - dataTypeModifier: -1, + dataTypeModifier: 68, format: "text", }, { @@ -157,7 +157,7 @@ export const viewsTests = [ columnID: 0, dataTypeID: 1043, dataTypeSize: 256, - dataTypeModifier: -1, + dataTypeModifier: 68, format: "text", }, {