Skip to content

Commit

Permalink
Merge pull request #10209 from nvanbenschoten/nvanbenschoten/pg_type
Browse files Browse the repository at this point in the history
sql: Add pg_type table to pg_catalog
  • Loading branch information
nvanbenschoten authored Oct 26, 2016
2 parents 78fa27f + e5379f6 commit a6ad154
Show file tree
Hide file tree
Showing 7 changed files with 367 additions and 71 deletions.
176 changes: 163 additions & 13 deletions pkg/sql/pg_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"fmt"
"hash"
"hash/fnv"
"reflect"
"strconv"

"github.com/cockroachdb/cockroach/pkg/sql/parser"
Expand All @@ -48,6 +49,7 @@ var pgCatalog = virtualSchema{
pgCatalogIndexesTable,
pgCatalogNamespaceTable,
pgCatalogTablesTable,
pgCatalogTypeTable,
},
}

Expand Down Expand Up @@ -123,7 +125,7 @@ CREATE TABLE pg_catalog.pg_attribute (
return addRow(
attRelID, // attrelid
parser.NewDString(column.Name), // attname
h.TypeOid(colTyp), // atttypid
typOid(colTyp), // atttypid
zeroVal, // attstattarget
typLen(colTyp), // attlen
parser.NewDInt(parser.DInt(colNum)), // attnum
Expand Down Expand Up @@ -625,13 +627,172 @@ CREATE TABLE pg_catalog.pg_tables (
},
}

func typLen(typ parser.Type) parser.Datum {
var (
typTypeBase = parser.NewDString("b")
typTypeComposite = parser.NewDString("c")
typTypeDomain = parser.NewDString("d")
typTypeEnum = parser.NewDString("e")
typTypePseudo = parser.NewDString("p")
typTypeRange = parser.NewDString("r")

// Avoid unused warning for constants.
_ = typTypeComposite
_ = typTypeDomain
_ = typTypeEnum
_ = typTypePseudo
_ = typTypeRange

// See https://www.postgresql.org/docs/9.6/static/catalog-pg-type.html#CATALOG-TYPCATEGORY-TABLE.
typCategoryArray = parser.NewDString("A")
typCategoryBoolean = parser.NewDString("B")
typCategoryComposite = parser.NewDString("C")
typCategoryDateTime = parser.NewDString("D")
typCategoryEnum = parser.NewDString("E")
typCategoryGeometric = parser.NewDString("G")
typCategoryNetworkAddr = parser.NewDString("I")
typCategoryNumeric = parser.NewDString("N")
typCategoryPseudo = parser.NewDString("P")
typCategoryRange = parser.NewDString("R")
typCategoryString = parser.NewDString("S")
typCategoryTimespan = parser.NewDString("T")
typCategoryUserDefined = parser.NewDString("U")
typCategoryBitString = parser.NewDString("V")
typCategoryUnknown = parser.NewDString("X")

// Avoid unused warning for constants.
_ = typCategoryArray
_ = typCategoryComposite
_ = typCategoryEnum
_ = typCategoryGeometric
_ = typCategoryNetworkAddr
_ = typCategoryPseudo
_ = typCategoryRange
_ = typCategoryBitString
_ = typCategoryUnknown

typDelim = parser.NewDString(",")
)

// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-type.html.
var pgCatalogTypeTable = virtualSchemaTable{
schema: `
CREATE TABLE pg_catalog.pg_type (
oid INT,
typname STRING NOT NULL DEFAULT '',
typnamespace INT,
typowner INT,
typlen INT,
typbyval BOOL,
typtype CHAR,
typcategory CHAR,
typispreferred BOOL,
typisdefined BOOL,
typdelim CHAR,
typrelid INT,
typelem INT,
typarray INT,
typinput INT,
typoutput INT,
typreceive INT,
typsend INT,
typmodin INT,
typmodout INT,
typanalyze INT,
typalign CHAR,
typstorage CHAR,
typnotnull BOOL,
typbasetype INT,
typtypmod INT,
typndims INT,
typcollation INT,
typdefaultbin STRING,
typdefault STRING,
typacl STRING
);
`,
populate: func(p *planner, addRow func(...parser.Datum) error) error {
for oid, typ := range oidToDatum {
if err := addRow(
parser.NewDInt(parser.DInt(oid)), // oid
parser.NewDString(typ.String()), // typname
parser.DNull, // typnamespace
parser.DNull, // typowner
typLen(typ), // typlen
typByVal(typ), // typbyval
typTypeBase, // typtype
typCategory(typ), // typcategory
parser.MakeDBool(false), // typispreferred
parser.MakeDBool(true), // typisdefined
typDelim, // typdelim
zeroVal, // typrelid
zeroVal, // typelem
zeroVal, // typarray

// regproc references
zeroVal, // typinput
zeroVal, // typoutput
zeroVal, // typreceive
zeroVal, // typsend
zeroVal, // typmodin
zeroVal, // typmodout
zeroVal, // typanalyze

parser.DNull, // typalign
parser.DNull, // typstorage
parser.MakeDBool(false), // typnotnull
zeroVal, // typbasetype
negOneVal, // typtypmod
zeroVal, // typndims
zeroVal, // typcollation
parser.DNull, // typdefaultbin
parser.DNull, // typdefault
parser.DNull, // typacl
); err != nil {
return err
}
}
return nil
},
}

// typOid is the only OID generation approach that does not use oidHasher, because
// object identifiers for types are not arbitrary, but instead need to be kept in
// sync with Postgres.
func typOid(typ parser.Type) *parser.DInt {
oid, _ := DatumToOid(typ)
return parser.NewDInt(parser.DInt(oid))
}

func typLen(typ parser.Type) *parser.DInt {
if sz, variable := typ.Size(); !variable {
return parser.NewDInt(parser.DInt(sz))
}
return negOneVal
}

func typByVal(typ parser.Type) parser.Datum {
_, variable := typ.Size()
return parser.MakeDBool(parser.DBool(!variable))
}

// This mapping should be kept sync with PG's categorization.
var datumToTypeCategory = map[reflect.Type]*parser.DString{
reflect.TypeOf(parser.TypeBool): typCategoryBoolean,
reflect.TypeOf(parser.TypeBytes): typCategoryUserDefined,
reflect.TypeOf(parser.TypeDate): typCategoryDateTime,
reflect.TypeOf(parser.TypeFloat): typCategoryNumeric,
reflect.TypeOf(parser.TypeInt): typCategoryNumeric,
reflect.TypeOf(parser.TypeInterval): typCategoryTimespan,
reflect.TypeOf(parser.TypeDecimal): typCategoryNumeric,
reflect.TypeOf(parser.TypeString): typCategoryString,
reflect.TypeOf(parser.TypeTimestamp): typCategoryDateTime,
reflect.TypeOf(parser.TypeTimestampTZ): typCategoryDateTime,
}

func typCategory(typ parser.Type) parser.Datum {
return datumToTypeCategory[reflect.TypeOf(typ)]
}

// oidHasher provides a consistent hashing mechanism for object identifiers in
// pg_catalog tables, allowing for reliable joins across tables.
//
Expand Down Expand Up @@ -691,7 +852,6 @@ const (
fkConstraintTypeTag
pKeyConstraintTypeTag
uniqueConstraintTypeTag
typeTypeTag
)

func (h oidHasher) writeTypeTag(tag oidTypeTag) {
Expand Down Expand Up @@ -734,10 +894,6 @@ func (h oidHasher) writeForeignKeyReference(fk *sqlbase.ForeignKeyReference) {
h.writeStr(fk.Name)
}

func (h oidHasher) writeType(typ parser.Type) {
h.writeStr(typ.String())
}

func (h oidHasher) DBOid(db *sqlbase.DatabaseDescriptor) *parser.DInt {
h.writeTypeTag(databaseTypeTag)
h.writeDB(db)
Expand Down Expand Up @@ -814,9 +970,3 @@ func (h oidHasher) UniqueConstraintOid(
h.writeIndex(index)
return h.getOid()
}

func (h oidHasher) TypeOid(typ parser.Type) *parser.DInt {
h.writeTypeTag(typeTypeTag)
h.writeType(typ)
return h.getOid()
}
67 changes: 67 additions & 0 deletions pkg/sql/pgtypes.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
// Copyright 2016 The Cockroach Authors.
//
// 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.
//
// Author: Nathan VanBenschoten (nvanbenschoten@gmail.com)

package sql

import (
"reflect"

"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/lib/pq/oid"
)

var oidToDatum = map[oid.Oid]parser.Type{
oid.T_bool: parser.TypeBool,
oid.T_bytea: parser.TypeBytes,
oid.T_date: parser.TypeDate,
oid.T_float4: parser.TypeFloat,
oid.T_float8: parser.TypeFloat,
oid.T_int2: parser.TypeInt,
oid.T_int4: parser.TypeInt,
oid.T_int8: parser.TypeInt,
oid.T_interval: parser.TypeInterval,
oid.T_numeric: parser.TypeDecimal,
oid.T_text: parser.TypeString,
oid.T_timestamp: parser.TypeTimestamp,
oid.T_timestamptz: parser.TypeTimestampTZ,
oid.T_varchar: parser.TypeString,
}

var datumToOid = map[reflect.Type]oid.Oid{
reflect.TypeOf(parser.TypeBool): oid.T_bool,
reflect.TypeOf(parser.TypeBytes): oid.T_bytea,
reflect.TypeOf(parser.TypeDate): oid.T_date,
reflect.TypeOf(parser.TypeFloat): oid.T_float8,
reflect.TypeOf(parser.TypeInt): oid.T_int8,
reflect.TypeOf(parser.TypeInterval): oid.T_interval,
reflect.TypeOf(parser.TypeDecimal): oid.T_numeric,
reflect.TypeOf(parser.TypeString): oid.T_text,
reflect.TypeOf(parser.TypeTimestamp): oid.T_timestamp,
reflect.TypeOf(parser.TypeTimestampTZ): oid.T_timestamptz,
}

// OidToDatum maps Postgres object IDs to CockroachDB types.
func OidToDatum(oid oid.Oid) (parser.Type, bool) {
t, ok := oidToDatum[oid]
return t, ok
}

// DatumToOid maps CockroachDB types to Postgres object IDs, using reflection
// to support unhashable types.
func DatumToOid(typ parser.Type) (oid.Oid, bool) {
oid, ok := datumToOid[reflect.TypeOf(typ)]
return oid, ok
}
4 changes: 2 additions & 2 deletions pkg/sql/pgwire/binary_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,11 @@ import (
"os"
"os/exec"
"path/filepath"
"reflect"
"strings"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/metric"
Expand Down Expand Up @@ -59,7 +59,7 @@ func testBinaryDatumType(t *testing.T, typ string, datumConstructor func(val str
buf.wrapped.Reset()

d := datumConstructor(test.In)
oid := datumToOid[reflect.TypeOf(d.ResolvedType())]
oid, _ := sql.DatumToOid(d.ResolvedType())
func() {
defer func() {
if r := recover(); r != nil {
Expand Down
33 changes: 0 additions & 33 deletions pkg/sql/pgwire/types.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"fmt"
"math"
"math/big"
"reflect"
"strconv"
"strings"
"time"
Expand Down Expand Up @@ -399,38 +398,6 @@ func pgBinaryToDate(i int32) *parser.DDate {
return parser.NewDDate(parser.DDate(daysSinceEpoch))
}

var (
oidToDatum = map[oid.Oid]parser.Type{
oid.T_bool: parser.TypeBool,
oid.T_bytea: parser.TypeBytes,
oid.T_date: parser.TypeDate,
oid.T_float4: parser.TypeFloat,
oid.T_float8: parser.TypeFloat,
oid.T_int2: parser.TypeInt,
oid.T_int4: parser.TypeInt,
oid.T_int8: parser.TypeInt,
oid.T_interval: parser.TypeInterval,
oid.T_numeric: parser.TypeDecimal,
oid.T_text: parser.TypeString,
oid.T_timestamp: parser.TypeTimestamp,
oid.T_timestamptz: parser.TypeTimestampTZ,
oid.T_varchar: parser.TypeString,
}
// Using reflection to support unhashable types.
datumToOid = map[reflect.Type]oid.Oid{
reflect.TypeOf(parser.TypeBool): oid.T_bool,
reflect.TypeOf(parser.TypeBytes): oid.T_bytea,
reflect.TypeOf(parser.TypeDate): oid.T_date,
reflect.TypeOf(parser.TypeFloat): oid.T_float8,
reflect.TypeOf(parser.TypeInt): oid.T_int8,
reflect.TypeOf(parser.TypeInterval): oid.T_interval,
reflect.TypeOf(parser.TypeDecimal): oid.T_numeric,
reflect.TypeOf(parser.TypeString): oid.T_text,
reflect.TypeOf(parser.TypeTimestamp): oid.T_timestamp,
reflect.TypeOf(parser.TypeTimestampTZ): oid.T_timestamptz,
}
)

// decodeOidDatum decodes bytes with specified Oid and format code into
// a datum.
func decodeOidDatum(id oid.Oid, code formatCode, b []byte) (parser.Datum, error) {
Expand Down
5 changes: 2 additions & 3 deletions pkg/sql/pgwire/v3.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"bufio"
"fmt"
"net"
"reflect"
"strconv"

"github.com/lib/pq/oid"
Expand Down Expand Up @@ -366,7 +365,7 @@ func (c *v3Conn) handleParse(ctx context.Context, buf *readBuffer) error {
if t == 0 {
continue
}
v, ok := oidToDatum[t]
v, ok := sql.OidToDatum(t)
if !ok {
return c.sendInternalError(fmt.Sprintf("unknown oid type: %v", t))
}
Expand Down Expand Up @@ -401,7 +400,7 @@ func (c *v3Conn) handleParse(ctx context.Context, buf *readBuffer) error {
if inTypes[i] != 0 {
continue
}
id, ok := datumToOid[reflect.TypeOf(t)]
id, ok := sql.DatumToOid(t)
if !ok {
return c.sendInternalError(fmt.Sprintf("unknown datum type: %s", t))
}
Expand Down
Loading

0 comments on commit a6ad154

Please sign in to comment.