Skip to content

Commit

Permalink
Merge pull request #904 from dolthub/jennifer/type
Browse files Browse the repository at this point in the history
use DoltgresType struct for all types and use pg_type fields for type storage
  • Loading branch information
jennifersp authored Nov 19, 2024
2 parents 165c30e + 5931ec2 commit 40dba4f
Show file tree
Hide file tree
Showing 196 changed files with 8,106 additions and 10,730 deletions.
5 changes: 3 additions & 2 deletions core/dataloader/csvdataloader.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/dolthub/go-mysql-server/sql"
"github.com/sirupsen/logrus"

"github.com/dolthub/doltgresql/server/functions/framework"
"github.com/dolthub/doltgresql/server/types"
)

Expand Down Expand Up @@ -69,7 +70,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)
Expand Down Expand Up @@ -134,7 +135,7 @@ func (cdl *CsvDataLoader) LoadChunk(ctx *sql.Context, data *bufio.Reader) error
if record[i] == nil {
row[i] = nil
} else {
row[i], err = cdl.colTypes[i].IoInput(ctx, fmt.Sprintf("%v", record[i]))
row[i], err = framework.IoInput(ctx, cdl.colTypes[i], fmt.Sprintf("%v", record[i]))
if err != nil {
return err
}
Expand Down
4 changes: 2 additions & 2 deletions core/dataloader/csvreader.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,15 +67,15 @@ 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
// BOM is stripped and the remaining contents of the reader are treated as that
// 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, ",")
}

Expand Down
4 changes: 2 additions & 2 deletions core/dataloader/string_prefix_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
3 changes: 2 additions & 1 deletion core/dataloader/tabdataloader.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/dolthub/go-mysql-server/sql"
"github.com/sirupsen/logrus"

"github.com/dolthub/doltgresql/server/functions/framework"
"github.com/dolthub/doltgresql/server/types"
)

Expand Down Expand Up @@ -132,7 +133,7 @@ func (tdl *TabularDataLoader) LoadChunk(ctx *sql.Context, data *bufio.Reader) er
if values[i] == tdl.nullChar {
row[i] = nil
} else {
row[i], err = tdl.colTypes[i].IoInput(ctx, values[i])
row[i], err = framework.IoInput(ctx, tdl.colTypes[i], values[i])
if err != nil {
return err
}
Expand Down
7 changes: 3 additions & 4 deletions core/typecollection/merge.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
96 changes: 12 additions & 84 deletions core/typecollection/serialization.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
Expand All @@ -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)
Expand All @@ -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)
}
}

Expand All @@ -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 {
Expand All @@ -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
}
Expand Down
47 changes: 24 additions & 23 deletions core/typecollection/typecollection.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()

Expand All @@ -38,12 +38,12 @@ func (pgs *TypeCollection) GetType(schName, typName string) (*types.Type, bool)
return typ, true
}
}
return nil, false
return types.DoltgresType{}, 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()

Expand All @@ -52,19 +52,19 @@ func (pgs *TypeCollection) GetDomainType(schName, typName string) (*types.Type,
return typ, true
}
}
return nil, false
return types.DoltgresType{}, false
}

// 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)
}
Expand All @@ -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 {
Expand All @@ -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()
Expand All @@ -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()

Expand All @@ -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
}
Expand Down
5 changes: 3 additions & 2 deletions server/analyzer/add_implicit_prefix_lengths.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
Expand Down Expand Up @@ -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
}
Expand All @@ -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
}
Expand Down
2 changes: 1 addition & 1 deletion server/analyzer/assign_insert_casts.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ 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)
if !ok {
Expand Down
Loading

0 comments on commit 40dba4f

Please sign in to comment.