diff --git a/docs/RFCS/array_encoding.md b/docs/RFCS/array_encoding.md index b0721e39fe96..471381c5a993 100644 --- a/docs/RFCS/array_encoding.md +++ b/docs/RFCS/array_encoding.md @@ -76,7 +76,7 @@ enforce the specified dimensionality of the arrays. ### Column Type protobuf encoding The column type protobuf will be modified to include an `ARRAY` value for -the `Kind`, and a nullable field indicating the element type if the type is +the `SemanticType`, and a nullable field indicating the element type if the type is an array. The existing `array_dimensions` field will be replaced with an int field denoting the number of dimensions in the array. @@ -163,7 +163,7 @@ This was deemed wasteful as arrays are restricted to 64MB to begin with, and it is expected that the common case is that arrays do not contain any `NULL`s (and thus the bitmap can be omitted). * The protobuf changes could alternatively be implemented by having the -existing `Kind` denote the element type, with no special cases required, as +existing `SemanticType` denote the element type, with no special cases required, as a 0-dimensional array can be interpreted as a scalar. This option is attractive but it was rejected on the grounds that it overly-centralizes the concept of column types around arrays and makes scalar datum types the special diff --git a/docs/RFCS/typing.md b/docs/RFCS/typing.md index 2a53377db881..45e299d56133 100644 --- a/docs/RFCS/typing.md +++ b/docs/RFCS/typing.md @@ -2019,9 +2019,9 @@ retain exact precision. Next, during type checking, ``NumVals`` will be evalutated as their logical `Datum` types. Here, they will be converted `int`, `float` or -`decimal`, based on their `Value.Kind()` (e.g. using +`decimal`, based on their `Value.SemanticType()` (e.g. using [Int64Val](https://golang.org/pkg/go/constant/#Int64Val) or -`decimal.SetString(Value.String())`. Some Kinds will result in a +`decimal.SetString(Value.String())`. Some Semantic Types will result in a panic because they should not be possible based on our parser. However, we could eventually introduce Complex literals using this approach. diff --git a/pkg/sql/analyze_test.go b/pkg/sql/analyze_test.go index b22ca1af6c90..628696f1f6b6 100644 --- a/pkg/sql/analyze_test.go +++ b/pkg/sql/analyze_test.go @@ -33,22 +33,22 @@ func testTableDesc() *sqlbase.TableDescriptor { ID: 1001, ParentID: 1000, Columns: []sqlbase.ColumnDescriptor{ - {Name: "a", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}}, - {Name: "b", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}}, - {Name: "c", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_BOOL}}, - {Name: "d", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_BOOL}}, - {Name: "e", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_BOOL}}, - {Name: "f", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_BOOL}}, - {Name: "g", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_BOOL}}, - {Name: "h", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_FLOAT}}, - {Name: "i", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_STRING}}, - {Name: "j", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}}, - {Name: "k", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_BYTES}}, - {Name: "l", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_DECIMAL}}, - {Name: "m", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_DECIMAL}}, - {Name: "n", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_DATE}}, - {Name: "o", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_TIMESTAMP}}, - {Name: "p", Type: sqlbase.ColumnType{Kind: sqlbase.ColumnType_TIMESTAMPTZ}}, + {Name: "a", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}}, + {Name: "b", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}}, + {Name: "c", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BOOL}}, + {Name: "d", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BOOL}}, + {Name: "e", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BOOL}}, + {Name: "f", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BOOL}}, + {Name: "g", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BOOL}}, + {Name: "h", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_FLOAT}}, + {Name: "i", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_STRING}}, + {Name: "j", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}}, + {Name: "k", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BYTES}}, + {Name: "l", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_DECIMAL}}, + {Name: "m", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_DECIMAL}}, + {Name: "n", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_DATE}}, + {Name: "o", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_TIMESTAMP}}, + {Name: "p", Type: sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_TIMESTAMPTZ}}, }, PrimaryIndex: sqlbase.IndexDescriptor{ Name: "primary", Unique: true, ColumnNames: []string{"a"}, diff --git a/pkg/sql/copy_in_test.go b/pkg/sql/copy_in_test.go index 80292168eed3..19a089b924e4 100644 --- a/pkg/sql/copy_in_test.go +++ b/pkg/sql/copy_in_test.go @@ -163,7 +163,7 @@ func TestCopyRandom(t *testing.T) { } rng := rand.New(rand.NewSource(0)) - types := []sqlbase.ColumnType_Kind{ + types := []sqlbase.ColumnType_SemanticType{ sqlbase.ColumnType_BOOL, sqlbase.ColumnType_INT, sqlbase.ColumnType_FLOAT, @@ -189,7 +189,7 @@ func TestCopyRandom(t *testing.T) { row[1] = d.String() for j, t := range types { - d := sqlbase.RandDatum(rng, sqlbase.ColumnType{Kind: t}, false) + d := sqlbase.RandDatum(rng, sqlbase.ColumnType{SemanticType: t}, false) ds := parser.AsStringWithFlags(d, parser.FmtBareStrings) switch t { case sqlbase.ColumnType_DECIMAL: diff --git a/pkg/sql/create.go b/pkg/sql/create.go index bcd9ca81a8c5..f5d7c4c74fea 100644 --- a/pkg/sql/create.go +++ b/pkg/sql/create.go @@ -851,9 +851,9 @@ func resolveFK( } for i := range srcCols { - if s, t := srcCols[i], targetCols[i]; s.Type.Kind != t.Type.Kind { + if s, t := srcCols[i], targetCols[i]; s.Type.SemanticType != t.Type.SemanticType { return fmt.Errorf("type of %q (%s) does not match foreign key %q.%q (%s)", - s.Name, s.Type.Kind, target.Name, t.Name, t.Type.Kind) + s.Name, s.Type.SemanticType, target.Name, t.Name, t.Type.SemanticType) } } diff --git a/pkg/sql/distsqlplan/aggregator_funcs_test.go b/pkg/sql/distsqlplan/aggregator_funcs_test.go index 09fef1410210..7a88c3c4782f 100644 --- a/pkg/sql/distsqlplan/aggregator_funcs_test.go +++ b/pkg/sql/distsqlplan/aggregator_funcs_test.go @@ -287,11 +287,11 @@ func TestDistAggregationTable(t *testing.T) { return []parser.Datum{ parser.NewDInt(parser.DInt(row)), parser.NewDInt(parser.DInt(rng.Intn(numRows))), - sqlbase.RandDatum(rng, sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, true), + sqlbase.RandDatum(rng, sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, true), parser.MakeDBool(parser.DBool(rng.Intn(10) == 0)), parser.MakeDBool(parser.DBool(rng.Intn(10) != 0)), - sqlbase.RandDatum(rng, sqlbase.ColumnType{Kind: sqlbase.ColumnType_DECIMAL}, false), - sqlbase.RandDatum(rng, sqlbase.ColumnType{Kind: sqlbase.ColumnType_DECIMAL}, true), + sqlbase.RandDatum(rng, sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_DECIMAL}, false), + sqlbase.RandDatum(rng, sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_DECIMAL}, true), parser.NewDBytes(parser.DBytes(randutil.RandBytes(rng, 10))), } }, diff --git a/pkg/sql/distsqlrun/aggregator.go b/pkg/sql/distsqlrun/aggregator.go index 4d5042d5f861..4d03741ccd65 100644 --- a/pkg/sql/distsqlrun/aggregator.go +++ b/pkg/sql/distsqlrun/aggregator.go @@ -133,7 +133,7 @@ func newAggregator( if col >= uint32(len(inputTypes)) { return nil, errors.Errorf("FilterColIdx out of range (%d)", col) } - t := inputTypes[col].Kind + t := inputTypes[col].SemanticType if t != sqlbase.ColumnType_BOOL && t != sqlbase.ColumnType_NULL { return nil, errors.Errorf( "filter column %d must be of boolean type, not %s", *aggInfo.FilterColIdx, t, diff --git a/pkg/sql/distsqlrun/aggregator_test.go b/pkg/sql/distsqlrun/aggregator_test.go index 53f1e4a3935b..ff4b61dba416 100644 --- a/pkg/sql/distsqlrun/aggregator_test.go +++ b/pkg/sql/distsqlrun/aggregator_test.go @@ -38,14 +38,14 @@ import ( func TestAggregator(t *testing.T) { defer leaktest.AfterTest(t)() - columnTypeInt := sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT} + columnTypeInt := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} v := [15]sqlbase.EncDatum{} null := sqlbase.EncDatum{Datum: parser.DNull} for i := range v { v[i] = sqlbase.DatumToEncDatum(columnTypeInt, parser.NewDInt(parser.DInt(i))) } - columnTypeBool := sqlbase.ColumnType{Kind: sqlbase.ColumnType_BOOL} + columnTypeBool := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BOOL} boolTrue := sqlbase.DatumToEncDatum(columnTypeBool, parser.DBoolTrue) boolFalse := sqlbase.DatumToEncDatum(columnTypeBool, parser.DBoolFalse) boolNULL := sqlbase.DatumToEncDatum(columnTypeBool, parser.DNull) diff --git a/pkg/sql/distsqlrun/algebraic_set_op.go b/pkg/sql/distsqlrun/algebraic_set_op.go index e9b677e6b690..4d94b6a4acf2 100644 --- a/pkg/sql/distsqlrun/algebraic_set_op.go +++ b/pkg/sql/distsqlrun/algebraic_set_op.go @@ -68,10 +68,10 @@ func newAlgebraicSetOp( len(lt), len(rt)) } for i := 0; i < len(lt); i++ { - if lt[i].Kind != rt[i].Kind { + if lt[i].SemanticType != rt[i].SemanticType { return nil, errors.Errorf( "Left column index %d (%s) is not the same as right column index %d (%s)", - i, lt[i].Kind, i, rt[i].Kind) + i, lt[i].SemanticType, i, rt[i].SemanticType) } } diff --git a/pkg/sql/distsqlrun/algebraic_set_op_test.go b/pkg/sql/distsqlrun/algebraic_set_op_test.go index b1e69dc316e5..c00d98c7c34a 100644 --- a/pkg/sql/distsqlrun/algebraic_set_op_test.go +++ b/pkg/sql/distsqlrun/algebraic_set_op_test.go @@ -43,7 +43,7 @@ type testInputs struct { func initTestData() testInputs { v := [15]sqlbase.EncDatum{} for i := range v { - v[i] = sqlbase.DatumToEncDatum(sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, + v[i] = sqlbase.DatumToEncDatum(sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, parser.NewDInt(parser.DInt(i))) } diff --git a/pkg/sql/distsqlrun/distinct_test.go b/pkg/sql/distsqlrun/distinct_test.go index cbc5707527d0..544f796585c4 100644 --- a/pkg/sql/distsqlrun/distinct_test.go +++ b/pkg/sql/distsqlrun/distinct_test.go @@ -31,7 +31,7 @@ func TestDistinct(t *testing.T) { v := [15]sqlbase.EncDatum{} for i := range v { - v[i] = sqlbase.DatumToEncDatum(sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, + v[i] = sqlbase.DatumToEncDatum(sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, parser.NewDInt(parser.DInt(i))) } diff --git a/pkg/sql/distsqlrun/hashjoiner_test.go b/pkg/sql/distsqlrun/hashjoiner_test.go index 14e7ec78287e..caa87bb39172 100644 --- a/pkg/sql/distsqlrun/hashjoiner_test.go +++ b/pkg/sql/distsqlrun/hashjoiner_test.go @@ -34,7 +34,7 @@ import ( func TestHashJoiner(t *testing.T) { defer leaktest.AfterTest(t)() - columnTypeInt := sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT} + columnTypeInt := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} v := [10]sqlbase.EncDatum{} for i := range v { v[i] = sqlbase.DatumToEncDatum(columnTypeInt, parser.NewDInt(parser.DInt(i))) @@ -576,7 +576,7 @@ func checkExpectedRows(expectedRows sqlbase.EncDatumRows, results *RowBuffer) er // the consumer is draining. func TestHashJoinerDrain(t *testing.T) { defer leaktest.AfterTest(t)() - columnTypeInt := sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT} + columnTypeInt := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} v := [10]sqlbase.EncDatum{} for i := range v { v[i] = sqlbase.DatumToEncDatum(columnTypeInt, parser.NewDInt(parser.DInt(i))) @@ -665,7 +665,7 @@ func TestHashJoinerDrain(t *testing.T) { func TestHashJoinerDrainAfterBuildPhaseError(t *testing.T) { defer leaktest.AfterTest(t)() - columnTypeInt := sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT} + columnTypeInt := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} v := [10]sqlbase.EncDatum{} for i := range v { v[i] = sqlbase.DatumToEncDatum(columnTypeInt, parser.NewDInt(parser.DInt(i))) diff --git a/pkg/sql/distsqlrun/input_sync_test.go b/pkg/sql/distsqlrun/input_sync_test.go index e648971a48f3..bbe29a537f22 100644 --- a/pkg/sql/distsqlrun/input_sync_test.go +++ b/pkg/sql/distsqlrun/input_sync_test.go @@ -32,7 +32,7 @@ import ( func TestOrderedSync(t *testing.T) { defer leaktest.AfterTest(t)() - columnTypeInt := &sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT} + columnTypeInt := &sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} v := [6]sqlbase.EncDatum{} for i := range v { v[i] = sqlbase.DatumToEncDatum(*columnTypeInt, parser.NewDInt(parser.DInt(i))) @@ -144,7 +144,7 @@ func TestOrderedSync(t *testing.T) { func TestUnorderedSync(t *testing.T) { defer leaktest.AfterTest(t)() - columnTypeInt := sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT} + columnTypeInt := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} mrc := &MultiplexedRowChannel{} mrc.Init(5, []sqlbase.ColumnType{columnTypeInt}) producerErr := make(chan error, 100) diff --git a/pkg/sql/distsqlrun/joinreader_test.go b/pkg/sql/distsqlrun/joinreader_test.go index fd250399ccbf..3776be3f7959 100644 --- a/pkg/sql/distsqlrun/joinreader_test.go +++ b/pkg/sql/distsqlrun/joinreader_test.go @@ -113,7 +113,7 @@ func TestJoinReader(t *testing.T) { for _, row := range c.input { encRow := make(sqlbase.EncDatumRow, len(row)) for i, d := range row { - encRow[i] = sqlbase.DatumToEncDatum(sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, d) + encRow[i] = sqlbase.DatumToEncDatum(sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, d) } if status := in.Push(encRow, ProducerMetadata{}); status != NeedMoreRows { t.Fatalf("unexpected response: %d", status) @@ -183,7 +183,7 @@ func TestJoinReaderDrain(t *testing.T) { encRow := make(sqlbase.EncDatumRow, 1) encRow[0] = sqlbase.DatumToEncDatum( - sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, parser.NewDInt(1), ) diff --git a/pkg/sql/distsqlrun/mergejoiner_test.go b/pkg/sql/distsqlrun/mergejoiner_test.go index 073acf0804b8..286dc873bdae 100644 --- a/pkg/sql/distsqlrun/mergejoiner_test.go +++ b/pkg/sql/distsqlrun/mergejoiner_test.go @@ -30,7 +30,7 @@ import ( func TestMergeJoiner(t *testing.T) { defer leaktest.AfterTest(t)() - columnTypeInt := sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT} + columnTypeInt := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} v := [10]sqlbase.EncDatum{} for i := range v { v[i] = sqlbase.DatumToEncDatum(columnTypeInt, parser.NewDInt(parser.DInt(i))) diff --git a/pkg/sql/distsqlrun/outbox_test.go b/pkg/sql/distsqlrun/outbox_test.go index baf78295b741..ceea55f6f0c8 100644 --- a/pkg/sql/distsqlrun/outbox_test.go +++ b/pkg/sql/distsqlrun/outbox_test.go @@ -73,7 +73,7 @@ func TestOutbox(t *testing.T) { producerC <- func() error { row := sqlbase.EncDatumRow{ sqlbase.DatumToEncDatum( - sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, parser.NewDInt(parser.DInt(0))), } if consumerStatus := outbox.Push(row, ProducerMetadata{}); consumerStatus != NeedMoreRows { @@ -84,7 +84,7 @@ func TestOutbox(t *testing.T) { for { row = sqlbase.EncDatumRow{ sqlbase.DatumToEncDatum( - sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, parser.NewDInt(parser.DInt(-1))), } consumerStatus := outbox.Push(row, ProducerMetadata{}) @@ -99,7 +99,7 @@ func TestOutbox(t *testing.T) { // Now send another row that the outbox will discard. row = sqlbase.EncDatumRow{ sqlbase.DatumToEncDatum( - sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, parser.NewDInt(parser.DInt(2))), } if consumerStatus := outbox.Push(row, ProducerMetadata{}); consumerStatus != DrainRequested { diff --git a/pkg/sql/distsqlrun/processors_test.go b/pkg/sql/distsqlrun/processors_test.go index 011c714ee10c..ebb6a1405f51 100644 --- a/pkg/sql/distsqlrun/processors_test.go +++ b/pkg/sql/distsqlrun/processors_test.go @@ -31,7 +31,7 @@ import ( func TestPostProcess(t *testing.T) { defer leaktest.AfterTest(t)() - columnTypeInt := sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT} + columnTypeInt := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} v := [10]sqlbase.EncDatum{} for i := range v { v[i] = sqlbase.DatumToEncDatum(columnTypeInt, parser.NewDInt(parser.DInt(i))) diff --git a/pkg/sql/distsqlrun/sorter_test.go b/pkg/sql/distsqlrun/sorter_test.go index 0c824262741e..ccef2eb2a75f 100644 --- a/pkg/sql/distsqlrun/sorter_test.go +++ b/pkg/sql/distsqlrun/sorter_test.go @@ -30,7 +30,7 @@ import ( func TestSorter(t *testing.T) { defer leaktest.AfterTest(t)() - columnTypeInt := sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT} + columnTypeInt := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} v := [6]sqlbase.EncDatum{} for i := range v { v[i] = sqlbase.DatumToEncDatum(columnTypeInt, parser.NewDInt(parser.DInt(i))) diff --git a/pkg/sql/distsqlrun/stream_encoder.go b/pkg/sql/distsqlrun/stream_encoder.go index dff16e47f7a7..bbd60dd548b5 100644 --- a/pkg/sql/distsqlrun/stream_encoder.go +++ b/pkg/sql/distsqlrun/stream_encoder.go @@ -100,7 +100,7 @@ func (se *StreamEncoder) AddRow(row sqlbase.EncDatumRow) error { if !ok { enc = preferredEncoding } - if enc != sqlbase.DatumEncoding_VALUE && sqlbase.HasCompositeKeyEncoding(row[i].Type.Kind) { + if enc != sqlbase.DatumEncoding_VALUE && sqlbase.HasCompositeKeyEncoding(row[i].Type.SemanticType) { // Force VALUE encoding for composite types (key encodings may lose data). enc = sqlbase.DatumEncoding_VALUE } diff --git a/pkg/sql/parser/constant.go b/pkg/sql/parser/constant.go index cb8ea6d848bb..f0fe10fc9f33 100644 --- a/pkg/sql/parser/constant.go +++ b/pkg/sql/parser/constant.go @@ -268,7 +268,7 @@ func (expr *NumVal) ResolveAsType(ctx *SemaContext, typ Type) (Datum, error) { return nil, err } oid := NewDOid(*d.(*DInt)) - oid.kind = oidTypeToColType(typ) + oid.symanticType = oidTypeToColType(typ) return oid, nil default: return nil, fmt.Errorf("could not resolve %T %v into a %T", expr, expr, typ) diff --git a/pkg/sql/parser/datum.go b/pkg/sql/parser/datum.go index 61a2e44915b5..8bc2c225d455 100644 --- a/pkg/sql/parser/datum.go +++ b/pkg/sql/parser/datum.go @@ -2241,16 +2241,16 @@ func (*DTable) Size() uintptr { return unsafe.Sizeof(DTable{}) } type DOid struct { // A DOid embeds a DInt, the underlying integer OID for this OID datum. DInt - // kind indicates the particular variety of OID this datum is, whether raw + // symanticType indicates the particular variety of OID this datum is, whether raw // oid or a reg* type. - kind *OidColType + symanticType *OidColType // name is set to the resolved name of this OID, if available. name string } // MakeDOid is a helper routine to create a DOid initialized from a DInt. func MakeDOid(d DInt) DOid { - return DOid{DInt: d, kind: oidColTypeOid, name: ""} + return DOid{DInt: d, symanticType: oidColTypeOid, name: ""} } // NewDOid is a helper routine to create a *DOid initialized from a DInt. @@ -2263,7 +2263,7 @@ func NewDOid(d DInt) *DOid { // returns it. func (d *DOid) AsRegProc(name string) *DOid { d.name = name - d.kind = oidColTypeRegProc + d.symanticType = oidColTypeRegProc return d } @@ -2291,7 +2291,7 @@ func (d *DOid) Compare(ctx *EvalContext, other Datum) int { // Format implements the Datum interface. func (d *DOid) Format(buf *bytes.Buffer, f FmtFlags) { - if d.kind == oidColTypeOid || d.name == "" { + if d.symanticType == oidColTypeOid || d.name == "" { FormatNode(buf, f, &d.DInt) } else { encodeSQLStringWithFlags(buf, d.name, FmtBareStrings) @@ -2307,18 +2307,18 @@ func (d *DOid) IsMin() bool { return d.DInt.IsMin() } // Next implements the Datum interface. func (d *DOid) Next() (Datum, bool) { next, ok := d.DInt.Next() - return &DOid{*next.(*DInt), d.kind, ""}, ok + return &DOid{*next.(*DInt), d.symanticType, ""}, ok } // Prev implements the Datum interface. func (d *DOid) Prev() (Datum, bool) { prev, ok := d.DInt.Prev() - return &DOid{*prev.(*DInt), d.kind, ""}, ok + return &DOid{*prev.(*DInt), d.symanticType, ""}, ok } // ResolvedType implements the Datum interface. func (d *DOid) ResolvedType() Type { - return oidColTypeToType(d.kind) + return oidColTypeToType(d.symanticType) } // Size implements the Datum interface. @@ -2327,13 +2327,13 @@ func (d *DOid) Size() uintptr { return unsafe.Sizeof(*d) } // max implements the Datum interface. func (d *DOid) max() (Datum, bool) { max, ok := d.DInt.max() - return &DOid{*max.(*DInt), d.kind, ""}, ok + return &DOid{*max.(*DInt), d.symanticType, ""}, ok } // min implements the Datum interface. func (d *DOid) min() (Datum, bool) { min, ok := d.DInt.min() - return &DOid{*min.(*DInt), d.kind, ""}, ok + return &DOid{*min.(*DInt), d.symanticType, ""}, ok } // DOidWrapper is a Datum implementation which is a wrapper around a Datum, allowing diff --git a/pkg/sql/parser/eval.go b/pkg/sql/parser/eval.go index 195d57694745..5444217e365f 100644 --- a/pkg/sql/parser/eval.go +++ b/pkg/sql/parser/eval.go @@ -2038,7 +2038,7 @@ var regTypeInfos = map[*OidColType]regTypeInfo{ func queryOidWithJoin( ctx *EvalContext, typ *OidColType, d Datum, joinClause string, additionalWhere string, ) (*DOid, error) { - ret := &DOid{kind: typ} + ret := &DOid{symanticType: typ} info := regTypeInfos[typ] var queryCol string switch d.(type) { @@ -2388,25 +2388,25 @@ func (expr *CastExpr) Eval(ctx *EvalContext) (Datum, error) { case *DOid: switch typ { case oidColTypeOid: - return &DOid{kind: typ, DInt: v.DInt}, nil + return &DOid{symanticType: typ, DInt: v.DInt}, nil default: oid, err := queryOid(ctx, typ, v) if err != nil { oid = NewDOid(v.DInt) - oid.kind = typ + oid.symanticType = typ } return oid, nil } case *DInt: switch typ { case oidColTypeOid: - return &DOid{kind: typ, DInt: *v}, nil + return &DOid{symanticType: typ, DInt: *v}, nil default: tmpOid := NewDOid(*v) oid, err := queryOid(ctx, typ, tmpOid) if err != nil { oid = tmpOid - oid.kind = typ + oid.symanticType = typ } return oid, nil } @@ -2425,7 +2425,7 @@ func (expr *CastExpr) Eval(ctx *EvalContext) (Datum, error) { if err != nil { return nil, err } - return &DOid{kind: typ, DInt: *i}, nil + return &DOid{symanticType: typ, DInt: *i}, nil case oidColTypeRegProc, oidColTypeRegProcedure: // Trim procedure type parameters, e.g. `max(int)` becomes `max`. // Postgres only does this when the cast is ::regprocedure, but we're @@ -2448,7 +2448,7 @@ func (expr *CastExpr) Eval(ctx *EvalContext) (Datum, error) { colType, err := ParseType(s) if err == nil { datumType := CastTargetToDatumType(colType) - return &DOid{kind: typ, DInt: DInt(datumType.Oid()), name: datumType.SQLName()}, nil + return &DOid{symanticType: typ, DInt: DInt(datumType.Oid()), name: datumType.SQLName()}, nil } // Fall back to searching pg_type, since we don't provide syntax for // every postgres type that we understand OIDs for. diff --git a/pkg/sql/show_fingerprints.go b/pkg/sql/show_fingerprints.go index 5d2b7e962cb4..e0de68e4de99 100644 --- a/pkg/sql/show_fingerprints.go +++ b/pkg/sql/show_fingerprints.go @@ -127,7 +127,7 @@ func (n *showFingerprintsNode) Next(ctx context.Context) (bool, error) { // TODO(dan): This is known to be a flawed way to fingerprint. Any datum // with the same string representation is fingerprinted the same, even // if they're different types. - switch col.Type.Kind { + switch col.Type.SemanticType { case sqlbase.ColumnType_BYTES: cols = append(cols, fmt.Sprintf("%s:::bytes", col.Name)) default: diff --git a/pkg/sql/sqlbase/encoded_datum_test.go b/pkg/sql/sqlbase/encoded_datum_test.go index acefb202dda0..70262da14516 100644 --- a/pkg/sql/sqlbase/encoded_datum_test.go +++ b/pkg/sql/sqlbase/encoded_datum_test.go @@ -42,7 +42,7 @@ func TestEncDatum(t *testing.T) { t.Errorf("empty EncDatum has an encoding") } - x := DatumToEncDatum(ColumnType{Kind: ColumnType_INT}, parser.NewDInt(5)) + x := DatumToEncDatum(ColumnType{SemanticType: ColumnType_INT}, parser.NewDInt(5)) if x.IsUnset() { t.Errorf("unset after DatumToEncDatum()") } @@ -55,7 +55,7 @@ func TestEncDatum(t *testing.T) { t.Fatal(err) } - y := EncDatumFromEncoded(ColumnType{Kind: ColumnType_INT}, DatumEncoding_ASCENDING_KEY, encoded) + y := EncDatumFromEncoded(ColumnType{SemanticType: ColumnType_INT}, DatumEncoding_ASCENDING_KEY, encoded) if y.IsUnset() { t.Errorf("unset after EncDatumFromEncoded") @@ -86,7 +86,7 @@ func TestEncDatum(t *testing.T) { } else if enc != DatumEncoding_ASCENDING_KEY { t.Errorf("invalid encoding %d", enc) } - z := EncDatumFromEncoded(ColumnType{Kind: ColumnType_INT}, DatumEncoding_DESCENDING_KEY, enc2) + z := EncDatumFromEncoded(ColumnType{SemanticType: ColumnType_INT}, DatumEncoding_DESCENDING_KEY, enc2) if enc, ok := z.Encoding(); !ok { t.Error("no encoding") } else if enc != DatumEncoding_DESCENDING_KEY { @@ -112,7 +112,7 @@ func TestEncDatumNull(t *testing.T) { defer leaktest.AfterTest(t)() // Verify DNull is null. - n := DatumToEncDatum(ColumnType{Kind: ColumnType_INT}, parser.DNull) + n := DatumToEncDatum(ColumnType{SemanticType: ColumnType_INT}, parser.DNull) if !n.IsNull() { t.Error("DNull not null") } @@ -130,7 +130,7 @@ func TestEncDatumNull(t *testing.T) { if err != nil { t.Fatal(err) } - b := EncDatumFromEncoded(ColumnType{Kind: ColumnType_INT}, DatumEncoding(enc), encoded) + b := EncDatumFromEncoded(ColumnType{SemanticType: ColumnType_INT}, DatumEncoding(enc), encoded) if a.IsNull() != b.IsNull() { t.Errorf("before: %s (null=%t) after: %s (null=%t)", a.String(), a.IsNull(), b.String(), b.IsNull()) @@ -191,8 +191,8 @@ func TestEncDatumCompare(t *testing.T) { defer evalCtx.Stop(context.Background()) rng, _ := randutil.NewPseudoRand() - for kind := range ColumnType_Kind_name { - kind := ColumnType_Kind(kind) + for kind := range ColumnType_SemanticType_name { + kind := ColumnType_SemanticType(kind) if kind == ColumnType_NULL { continue } @@ -201,7 +201,7 @@ func TestEncDatumCompare(t *testing.T) { kind == ColumnType_INT2VECTOR { continue } - typ := ColumnType{Kind: kind} + typ := ColumnType{SemanticType: kind} if kind == ColumnType_COLLATEDSTRING { typ.Locale = RandCollationLocale(rng) } @@ -267,7 +267,7 @@ func TestEncDatumFromBuffer(t *testing.T) { var buf []byte enc := make([]DatumEncoding, len(ed)) for i := range ed { - if HasCompositeKeyEncoding(ed[i].Type.Kind) { + if HasCompositeKeyEncoding(ed[i].Type.SemanticType) { // There's no way to reconstruct data from the key part of a composite // encoding. enc[i] = DatumEncoding_VALUE @@ -309,7 +309,7 @@ func TestEncDatumRowCompare(t *testing.T) { v := [5]EncDatum{} for i := range v { - v[i] = DatumToEncDatum(ColumnType{Kind: ColumnType_INT}, parser.NewDInt(parser.DInt(i))) + v[i] = DatumToEncDatum(ColumnType{SemanticType: ColumnType_INT}, parser.NewDInt(parser.DInt(i))) } asc := encoding.Ascending diff --git a/pkg/sql/sqlbase/result_columns.go b/pkg/sql/sqlbase/result_columns.go index 41881d905a9b..38f051234ea3 100644 --- a/pkg/sql/sqlbase/result_columns.go +++ b/pkg/sql/sqlbase/result_columns.go @@ -46,7 +46,7 @@ func ResultColumnsFromColDescs(colDescs []ColumnDescriptor) ResultColumns { // Convert the ColumnDescriptor to ResultColumn. typ := colDesc.Type.ToDatumType() if typ == nil { - panic(fmt.Sprintf("unsupported column type: %s", colDesc.Type.Kind)) + panic(fmt.Sprintf("unsupported column type: %s", colDesc.Type.SemanticType)) } hidden := colDesc.Hidden diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index f89501577a69..398614754cfa 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -555,7 +555,7 @@ func (desc *TableDescriptor) ensurePrimaryKey() error { col := ColumnDescriptor{ Name: "rowid", Type: ColumnType{ - Kind: ColumnType_INT, + SemanticType: ColumnType_INT, }, DefaultExpr: &s, Hidden: true, @@ -577,8 +577,8 @@ func (desc *TableDescriptor) ensurePrimaryKey() error { // HasCompositeKeyEncoding returns true if key columns of the given kind can // have a composite encoding. For such types, it can be decided on a // case-by-base basis whether a given Datum requires the composite encoding. -func HasCompositeKeyEncoding(kind ColumnType_Kind) bool { - switch kind { +func HasCompositeKeyEncoding(semanticType ColumnType_SemanticType) bool { + switch semanticType { case ColumnType_COLLATEDSTRING, ColumnType_FLOAT, ColumnType_DECIMAL: @@ -625,7 +625,7 @@ func (desc *TableDescriptor) allocateIndexIDs(columnNames map[string]ColumnID) e isCompositeColumn := make(map[ColumnID]struct{}) for _, col := range desc.Columns { - if HasCompositeKeyEncoding(col.Type.Kind) { + if HasCompositeKeyEncoding(col.Type.SemanticType) { isCompositeColumn[col.ID] = struct{}{} } } @@ -1210,7 +1210,7 @@ const FamilyHeuristicTargetBytes = 256 func upperBoundColumnValueEncodedSize(col ColumnDescriptor) (int, bool) { var typ encoding.Type var size int - switch col.Type.Kind { + switch col.Type.SemanticType { case ColumnType_BOOL: typ = encoding.True case ColumnType_INT, ColumnType_DATE, ColumnType_TIMESTAMP, @@ -1227,7 +1227,7 @@ func upperBoundColumnValueEncodedSize(col ColumnDescriptor) (int, bool) { case ColumnType_DECIMAL: typ, size = encoding.Decimal, int(col.Type.Precision) default: - panic(errors.Errorf("unknown column type: %s", col.Type.Kind)) + panic(errors.Errorf("unknown column type: %s", col.Type.SemanticType)) } return encoding.UpperBoundValueEncodingSize(uint32(col.ID), typ, size) } @@ -1707,7 +1707,7 @@ func ColumnsSelectors(cols []ColumnDescriptor) parser.SelectExprs { // SQLString returns the SQL string corresponding to the type. func (c *ColumnType) SQLString() string { - switch c.Kind { + switch c.SemanticType { case ColumnType_INT: if c.Width > 0 { // A non-zero width indicates a bit array. The syntax "INT(N)" @@ -1716,18 +1716,18 @@ func (c *ColumnType) SQLString() string { } case ColumnType_STRING: if c.Width > 0 { - return fmt.Sprintf("%s(%d)", c.Kind.String(), c.Width) + return fmt.Sprintf("%s(%d)", c.SemanticType.String(), c.Width) } case ColumnType_FLOAT: if c.Precision > 0 { - return fmt.Sprintf("%s(%d)", c.Kind.String(), c.Precision) + return fmt.Sprintf("%s(%d)", c.SemanticType.String(), c.Precision) } case ColumnType_DECIMAL: if c.Precision > 0 { if c.Width > 0 { - return fmt.Sprintf("%s(%d,%d)", c.Kind.String(), c.Precision, c.Width) + return fmt.Sprintf("%s(%d,%d)", c.SemanticType.String(), c.Precision, c.Width) } - return fmt.Sprintf("%s(%d)", c.Kind.String(), c.Precision) + return fmt.Sprintf("%s(%d)", c.SemanticType.String(), c.Precision) } case ColumnType_TIMESTAMPTZ: return "TIMESTAMP WITH TIME ZONE" @@ -1742,14 +1742,14 @@ func (c *ColumnType) SQLString() string { case ColumnType_INT_ARRAY: return "INT[]" } - return c.Kind.String() + return c.SemanticType.String() } // MaxCharacterLength returns the declared maximum length of characters if the // ColumnType is a character or bit string data type. Returns false if the data // type is not a character or bit string, or if the string's length is not bounded. func (c *ColumnType) MaxCharacterLength() (int32, bool) { - switch c.Kind { + switch c.SemanticType { case ColumnType_INT, ColumnType_STRING, ColumnType_COLLATEDSTRING: if c.Width > 0 { return c.Width, true @@ -1762,7 +1762,7 @@ func (c *ColumnType) MaxCharacterLength() (int32, bool) { // datum if the ColumnType is a character string. Returns false if the data type // is not a character string, or if the string's length is not bounded. func (c *ColumnType) MaxOctetLength() (int32, bool) { - switch c.Kind { + switch c.SemanticType { case ColumnType_STRING, ColumnType_COLLATEDSTRING: if c.Width > 0 { return c.Width * utf8.UTFMax, true @@ -1775,7 +1775,7 @@ func (c *ColumnType) MaxOctetLength() (int32, bool) { // data types. Returns false if the data type is not numeric, or if the precision // of the numeric type is not bounded. func (c *ColumnType) NumericPrecision() (int32, bool) { - switch c.Kind { + switch c.SemanticType { case ColumnType_INT: return 64, true case ColumnType_FLOAT: @@ -1795,7 +1795,7 @@ func (c *ColumnType) NumericPrecision() (int32, bool) { // data types. Returns false if the data type is not an exact numeric, or if the // scale of the exact numeric type is not bounded. func (c *ColumnType) NumericScale() (int32, bool) { - switch c.Kind { + switch c.SemanticType { case ColumnType_INT: return 0, true case ColumnType_DECIMAL: @@ -1811,40 +1811,40 @@ func DatumTypeToColumnType(ptyp parser.Type) ColumnType { var ctyp ColumnType switch ptyp { case parser.TypeBool: - ctyp.Kind = ColumnType_BOOL + ctyp.SemanticType = ColumnType_BOOL case parser.TypeInt: - ctyp.Kind = ColumnType_INT + ctyp.SemanticType = ColumnType_INT case parser.TypeFloat: - ctyp.Kind = ColumnType_FLOAT + ctyp.SemanticType = ColumnType_FLOAT case parser.TypeDecimal: - ctyp.Kind = ColumnType_DECIMAL + ctyp.SemanticType = ColumnType_DECIMAL case parser.TypeBytes: - ctyp.Kind = ColumnType_BYTES + ctyp.SemanticType = ColumnType_BYTES case parser.TypeString: - ctyp.Kind = ColumnType_STRING + ctyp.SemanticType = ColumnType_STRING case parser.TypeName: - ctyp.Kind = ColumnType_NAME + ctyp.SemanticType = ColumnType_NAME case parser.TypeDate: - ctyp.Kind = ColumnType_DATE + ctyp.SemanticType = ColumnType_DATE case parser.TypeTimestamp: - ctyp.Kind = ColumnType_TIMESTAMP + ctyp.SemanticType = ColumnType_TIMESTAMP case parser.TypeTimestampTZ: - ctyp.Kind = ColumnType_TIMESTAMPTZ + ctyp.SemanticType = ColumnType_TIMESTAMPTZ case parser.TypeInterval: - ctyp.Kind = ColumnType_INTERVAL + ctyp.SemanticType = ColumnType_INTERVAL case parser.TypeUUID: - ctyp.Kind = ColumnType_UUID + ctyp.SemanticType = ColumnType_UUID case parser.TypeOid: - ctyp.Kind = ColumnType_OID + ctyp.SemanticType = ColumnType_OID case parser.TypeNull: - ctyp.Kind = ColumnType_NULL + ctyp.SemanticType = ColumnType_NULL case parser.TypeIntArray: - ctyp.Kind = ColumnType_INT_ARRAY + ctyp.SemanticType = ColumnType_INT_ARRAY case parser.TypeIntVector: - ctyp.Kind = ColumnType_INT2VECTOR + ctyp.SemanticType = ColumnType_INT2VECTOR default: if t, ok := ptyp.(parser.TCollatedString); ok { - ctyp.Kind = ColumnType_COLLATEDSTRING + ctyp.SemanticType = ColumnType_COLLATEDSTRING ctyp.Locale = &t.Locale } else { panic(fmt.Sprintf("unsupported result type: %s", ptyp)) @@ -1856,7 +1856,7 @@ func DatumTypeToColumnType(ptyp parser.Type) ColumnType { // ToDatumType converts the ColumnType to the correct type, or nil if there is // no correspondence. func (c *ColumnType) ToDatumType() parser.Type { - switch c.Kind { + switch c.SemanticType { case ColumnType_BOOL: return parser.TypeBool case ColumnType_INT: diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index c258bde68998..184b1b43df36 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -55,26 +55,26 @@ func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { return fileDescript // These mirror the types supported by the sql/parser. See // sql/parser/col_types.go. -type ColumnType_Kind int32 +type ColumnType_SemanticType int32 const ( - ColumnType_BOOL ColumnType_Kind = 0 - ColumnType_INT ColumnType_Kind = 1 - ColumnType_FLOAT ColumnType_Kind = 2 - ColumnType_DECIMAL ColumnType_Kind = 3 - ColumnType_DATE ColumnType_Kind = 4 - ColumnType_TIMESTAMP ColumnType_Kind = 5 - ColumnType_INTERVAL ColumnType_Kind = 6 - ColumnType_STRING ColumnType_Kind = 7 - ColumnType_BYTES ColumnType_Kind = 8 - ColumnType_TIMESTAMPTZ ColumnType_Kind = 9 - ColumnType_COLLATEDSTRING ColumnType_Kind = 10 - ColumnType_NAME ColumnType_Kind = 11 - ColumnType_OID ColumnType_Kind = 12 + ColumnType_BOOL ColumnType_SemanticType = 0 + ColumnType_INT ColumnType_SemanticType = 1 + ColumnType_FLOAT ColumnType_SemanticType = 2 + ColumnType_DECIMAL ColumnType_SemanticType = 3 + ColumnType_DATE ColumnType_SemanticType = 4 + ColumnType_TIMESTAMP ColumnType_SemanticType = 5 + ColumnType_INTERVAL ColumnType_SemanticType = 6 + ColumnType_STRING ColumnType_SemanticType = 7 + ColumnType_BYTES ColumnType_SemanticType = 8 + ColumnType_TIMESTAMPTZ ColumnType_SemanticType = 9 + ColumnType_COLLATEDSTRING ColumnType_SemanticType = 10 + ColumnType_NAME ColumnType_SemanticType = 11 + ColumnType_OID ColumnType_SemanticType = 12 // NULL is not supported as a table column type, however it can be // transferred through distsql streams. - ColumnType_NULL ColumnType_Kind = 13 - ColumnType_UUID ColumnType_Kind = 14 + ColumnType_NULL ColumnType_SemanticType = 13 + ColumnType_UUID ColumnType_SemanticType = 14 // Array and vector types. // // TODO(cuongdo): Fix this before allowing persistence of array/vector types @@ -82,11 +82,11 @@ const ( // simply the parameterized type of the array. However, because Kind is used // to determine type information elsewhere, it isn't possible to take the // cleaner approach without an extensive refactoring. - ColumnType_INT_ARRAY ColumnType_Kind = 100 - ColumnType_INT2VECTOR ColumnType_Kind = 200 + ColumnType_INT_ARRAY ColumnType_SemanticType = 100 + ColumnType_INT2VECTOR ColumnType_SemanticType = 200 ) -var ColumnType_Kind_name = map[int32]string{ +var ColumnType_SemanticType_name = map[int32]string{ 0: "BOOL", 1: "INT", 2: "FLOAT", @@ -105,7 +105,7 @@ var ColumnType_Kind_name = map[int32]string{ 100: "INT_ARRAY", 200: "INT2VECTOR", } -var ColumnType_Kind_value = map[string]int32{ +var ColumnType_SemanticType_value = map[string]int32{ "BOOL": 0, "INT": 1, "FLOAT": 2, @@ -125,23 +125,25 @@ var ColumnType_Kind_value = map[string]int32{ "INT2VECTOR": 200, } -func (x ColumnType_Kind) Enum() *ColumnType_Kind { - p := new(ColumnType_Kind) +func (x ColumnType_SemanticType) Enum() *ColumnType_SemanticType { + p := new(ColumnType_SemanticType) *p = x return p } -func (x ColumnType_Kind) String() string { - return proto.EnumName(ColumnType_Kind_name, int32(x)) +func (x ColumnType_SemanticType) String() string { + return proto.EnumName(ColumnType_SemanticType_name, int32(x)) } -func (x *ColumnType_Kind) UnmarshalJSON(data []byte) error { - value, err := proto.UnmarshalJSONEnum(ColumnType_Kind_value, data, "ColumnType_Kind") +func (x *ColumnType_SemanticType) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(ColumnType_SemanticType_value, data, "ColumnType_SemanticType") if err != nil { return err } - *x = ColumnType_Kind(value) + *x = ColumnType_SemanticType(value) return nil } -func (ColumnType_Kind) EnumDescriptor() ([]byte, []int) { return fileDescriptorStructured, []int{0, 0} } +func (ColumnType_SemanticType) EnumDescriptor() ([]byte, []int) { + return fileDescriptorStructured, []int{0, 0} +} // The direction of a column in the index. type IndexDescriptor_Direction int32 @@ -336,7 +338,7 @@ func (TableDescriptor_State) EnumDescriptor() ([]byte, []int) { } type ColumnType struct { - Kind ColumnType_Kind `protobuf:"varint,1,opt,name=kind,enum=cockroach.sql.sqlbase.ColumnType_Kind" json:"kind"` + SemanticType ColumnType_SemanticType `protobuf:"varint,1,opt,name=semantic_type,json=semanticType,enum=cockroach.sql.sqlbase.ColumnType_SemanticType" json:"semantic_type"` // BIT, INT, FLOAT, DECIMAL, CHAR and BINARY Width int32 `protobuf:"varint,2,opt,name=width" json:"width"` // FLOAT and DECIMAL. @@ -1241,7 +1243,7 @@ func init() { proto.RegisterType((*DatabaseDescriptor)(nil), "cockroach.sql.sqlbase.DatabaseDescriptor") proto.RegisterType((*Descriptor)(nil), "cockroach.sql.sqlbase.Descriptor") proto.RegisterEnum("cockroach.sql.sqlbase.ConstraintValidity", ConstraintValidity_name, ConstraintValidity_value) - proto.RegisterEnum("cockroach.sql.sqlbase.ColumnType_Kind", ColumnType_Kind_name, ColumnType_Kind_value) + proto.RegisterEnum("cockroach.sql.sqlbase.ColumnType_SemanticType", ColumnType_SemanticType_name, ColumnType_SemanticType_value) proto.RegisterEnum("cockroach.sql.sqlbase.IndexDescriptor_Direction", IndexDescriptor_Direction_name, IndexDescriptor_Direction_value) proto.RegisterEnum("cockroach.sql.sqlbase.DescriptorMutation_State", DescriptorMutation_State_name, DescriptorMutation_State_value) proto.RegisterEnum("cockroach.sql.sqlbase.DescriptorMutation_Direction", DescriptorMutation_Direction_name, DescriptorMutation_Direction_value) @@ -1264,7 +1266,7 @@ func (m *ColumnType) MarshalTo(dAtA []byte) (int, error) { _ = l dAtA[i] = 0x8 i++ - i = encodeVarintStructured(dAtA, i, uint64(m.Kind)) + i = encodeVarintStructured(dAtA, i, uint64(m.SemanticType)) dAtA[i] = 0x10 i++ i = encodeVarintStructured(dAtA, i, uint64(m.Width)) @@ -2168,7 +2170,7 @@ func encodeVarintStructured(dAtA []byte, offset int, v uint64) int { func (m *ColumnType) Size() (n int) { var l int _ = l - n += 1 + sovStructured(uint64(m.Kind)) + n += 1 + sovStructured(uint64(m.SemanticType)) n += 1 + sovStructured(uint64(m.Width)) n += 1 + sovStructured(uint64(m.Precision)) if len(m.ArrayDimensions) > 0 { @@ -2572,9 +2574,9 @@ func (m *ColumnType) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Kind", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SemanticType", wireType) } - m.Kind = 0 + m.SemanticType = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowStructured @@ -2584,7 +2586,7 @@ func (m *ColumnType) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Kind |= (ColumnType_Kind(b) & 0x7F) << shift + m.SemanticType |= (ColumnType_SemanticType(b) & 0x7F) << shift if b < 0x80 { break } @@ -5940,148 +5942,149 @@ func init() { } var fileDescriptorStructured = []byte{ - // 2282 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x58, 0x4b, 0x6f, 0xdb, 0xd8, - 0x15, 0x36, 0xf5, 0xd6, 0xd1, 0x8b, 0xbe, 0x79, 0x31, 0xee, 0xd4, 0x56, 0x34, 0x33, 0xa9, 0x93, - 0xce, 0xc8, 0x19, 0x67, 0x5a, 0x0c, 0xfa, 0x18, 0x44, 0x2f, 0x27, 0x74, 0x64, 0xc9, 0xa1, 0xe5, - 0xa4, 0x19, 0xb4, 0x15, 0x68, 0xdd, 0x6b, 0x9b, 0x09, 0x45, 0x2a, 0x24, 0xe5, 0xb1, 0xfe, 0x40, - 0x31, 0xcb, 0xa2, 0x3f, 0x60, 0x50, 0xcc, 0xb6, 0x8b, 0x6e, 0xba, 0x28, 0x8a, 0xfe, 0x80, 0x2c, - 0xbb, 0x2c, 0x50, 0xc0, 0x68, 0xdd, 0x1f, 0x51, 0x20, 0xab, 0xe2, 0x5e, 0xde, 0x4b, 0x52, 0x96, - 0x95, 0xb1, 0x93, 0x85, 0x00, 0xf1, 0x3c, 0x3e, 0x9e, 0x7b, 0xee, 0x79, 0x12, 0xee, 0x0e, 0xec, - 0xc1, 0x4b, 0xc7, 0xd6, 0x07, 0x87, 0x6b, 0xa3, 0x97, 0x07, 0x6b, 0xee, 0x2b, 0x93, 0xfe, 0xf6, - 0x74, 0x97, 0xac, 0xb9, 0x9e, 0x33, 0x1e, 0x78, 0x63, 0x87, 0xe0, 0xea, 0xc8, 0xb1, 0x3d, 0x1b, - 0x5d, 0x0b, 0x64, 0xab, 0xee, 0x2b, 0xb3, 0xca, 0xe5, 0x96, 0xca, 0xd3, 0x10, 0xec, 0xdf, 0x68, - 0x6f, 0x0d, 0xeb, 0x9e, 0xee, 0x2b, 0x2e, 0xdd, 0x9e, 0x96, 0x18, 0x7b, 0x86, 0xb9, 0x76, 0x68, - 0x0e, 0xd6, 0x3c, 0x63, 0x48, 0x5c, 0x4f, 0x1f, 0x8e, 0xb8, 0xdc, 0x9d, 0xf9, 0xc6, 0x8c, 0x1c, - 0xe3, 0xc8, 0x30, 0xc9, 0x01, 0xe1, 0xa2, 0x57, 0x0f, 0xec, 0x03, 0x9b, 0xfd, 0x5d, 0xa3, 0xff, - 0x7c, 0x6a, 0xe5, 0xcf, 0x71, 0x80, 0x86, 0x6d, 0x8e, 0x87, 0x56, 0x6f, 0x32, 0x22, 0xe8, 0x01, - 0x24, 0x5e, 0x1a, 0x16, 0x56, 0xa4, 0xb2, 0xb4, 0x5a, 0x5c, 0xbf, 0x5d, 0x3d, 0xd7, 0xfe, 0x6a, - 0xa8, 0x50, 0x7d, 0x6c, 0x58, 0xb8, 0x9e, 0x78, 0x7d, 0xb2, 0xb2, 0xa0, 0x31, 0x4d, 0xb4, 0x04, - 0xc9, 0xaf, 0x0d, 0xec, 0x1d, 0x2a, 0xb1, 0xb2, 0xb4, 0x9a, 0xe4, 0x2c, 0x9f, 0x84, 0x2a, 0x90, - 0x1d, 0x39, 0x64, 0x60, 0xb8, 0x86, 0x6d, 0x29, 0xf1, 0x08, 0x3f, 0x24, 0xa3, 0x3b, 0x20, 0xeb, - 0x8e, 0xa3, 0x4f, 0xfa, 0xd8, 0x18, 0x12, 0x8b, 0x92, 0x5c, 0x25, 0x51, 0x8e, 0xaf, 0x26, 0xb5, - 0x12, 0xa3, 0x37, 0x03, 0x32, 0xba, 0x0e, 0x29, 0xd3, 0x1e, 0xe8, 0x26, 0x51, 0x92, 0x65, 0x69, - 0x35, 0xab, 0xf1, 0xa7, 0xca, 0xbf, 0x24, 0x48, 0x50, 0xbb, 0x50, 0x06, 0x12, 0xf5, 0x6e, 0xb7, - 0x2d, 0x2f, 0xa0, 0x34, 0xc4, 0xd5, 0x4e, 0x4f, 0x96, 0x50, 0x16, 0x92, 0x1b, 0xed, 0x6e, 0xad, - 0x27, 0xc7, 0x50, 0x0e, 0xd2, 0xcd, 0x56, 0x43, 0xdd, 0xaa, 0xb5, 0xe5, 0x38, 0x15, 0x6d, 0xd6, - 0x7a, 0x2d, 0x39, 0x81, 0x0a, 0x90, 0xed, 0xa9, 0x5b, 0xad, 0x9d, 0x5e, 0x6d, 0x6b, 0x5b, 0x4e, - 0xa2, 0x3c, 0x64, 0xd4, 0x4e, 0xaf, 0xa5, 0x3d, 0xad, 0xb5, 0xe5, 0x14, 0x02, 0x48, 0xed, 0xf4, - 0x34, 0xb5, 0xf3, 0x50, 0x4e, 0x53, 0xa8, 0xfa, 0xf3, 0x5e, 0x6b, 0x47, 0xce, 0xa0, 0x12, 0xe4, - 0x02, 0x9d, 0xde, 0x57, 0x72, 0x16, 0x21, 0x28, 0x36, 0xba, 0xed, 0x76, 0xad, 0xd7, 0x6a, 0x72, - 0x79, 0xa0, 0xaf, 0xe8, 0xd4, 0xb6, 0x5a, 0x72, 0x8e, 0x5a, 0xd3, 0x55, 0x9b, 0x72, 0x9e, 0x91, - 0x76, 0xdb, 0x6d, 0xb9, 0x40, 0xff, 0xed, 0xee, 0xaa, 0x4d, 0xb9, 0x48, 0xdf, 0xaf, 0x76, 0x7a, - 0xfd, 0x9a, 0xa6, 0xd5, 0x9e, 0xcb, 0x18, 0x95, 0x00, 0xd4, 0x4e, 0x6f, 0xfd, 0x69, 0xab, 0xd1, - 0xeb, 0x6a, 0xf2, 0x6b, 0xa9, 0xf2, 0x3f, 0x09, 0xae, 0x6c, 0xd8, 0x0e, 0x31, 0x0e, 0xac, 0xc7, - 0x64, 0xa2, 0x91, 0x7d, 0xe2, 0x10, 0x6b, 0x40, 0x50, 0x19, 0x92, 0x9e, 0xbe, 0x67, 0x12, 0x76, - 0x77, 0x85, 0x3a, 0x50, 0xc7, 0xbe, 0x39, 0x59, 0x89, 0xa9, 0x4d, 0xcd, 0x67, 0xa0, 0x8f, 0x21, - 0x69, 0x58, 0x98, 0x1c, 0xb3, 0xab, 0x29, 0xd4, 0x4b, 0x5c, 0x22, 0xad, 0x52, 0x22, 0x15, 0x63, - 0x5c, 0xa4, 0x40, 0xc2, 0xd2, 0x87, 0x84, 0x5d, 0x50, 0x56, 0xdc, 0x2d, 0xa5, 0xa0, 0xc7, 0x90, - 0x39, 0xd2, 0x4d, 0x03, 0x1b, 0xde, 0x44, 0x49, 0xb0, 0x08, 0xb9, 0x33, 0x37, 0x42, 0x2c, 0xd7, - 0x73, 0x74, 0xc3, 0xf2, 0x9e, 0x72, 0x05, 0x0e, 0x14, 0x00, 0xa0, 0x7b, 0xb0, 0xe8, 0x1e, 0xea, - 0x0e, 0xc1, 0xfd, 0x91, 0x43, 0xf6, 0x8d, 0xe3, 0xbe, 0x49, 0x2c, 0x76, 0x91, 0x22, 0x28, 0x4a, - 0x3e, 0x7b, 0x9b, 0x71, 0xdb, 0xc4, 0xaa, 0xfc, 0x2e, 0x06, 0xb2, 0x1f, 0x7a, 0x4d, 0xe2, 0x0e, - 0x1c, 0x63, 0xe4, 0xd9, 0x4e, 0x60, 0xad, 0x34, 0x63, 0xed, 0x6d, 0x88, 0x19, 0x98, 0x9f, 0xf5, - 0x3a, 0xa5, 0x9f, 0x32, 0x6f, 0xbc, 0x39, 0x59, 0xc9, 0xf8, 0x28, 0x6a, 0x53, 0x8b, 0x19, 0x18, - 0xfd, 0x1c, 0x12, 0xde, 0x64, 0xe4, 0x9f, 0x37, 0xb7, 0x7e, 0xeb, 0x7b, 0x63, 0x5e, 0xbc, 0x84, - 0x2a, 0xa1, 0x32, 0x64, 0xac, 0xb1, 0x69, 0x32, 0xc7, 0x53, 0x97, 0x64, 0xc4, 0x39, 0x05, 0x15, - 0xdd, 0x82, 0x3c, 0x26, 0xfb, 0xfa, 0xd8, 0xf4, 0xfa, 0xe4, 0x78, 0xe4, 0xf0, 0x58, 0xcd, 0x71, - 0x5a, 0xeb, 0x78, 0xe4, 0xa0, 0x0f, 0x20, 0x75, 0x68, 0x60, 0x4c, 0x2c, 0x25, 0x15, 0x81, 0xe0, - 0xb4, 0xcd, 0x44, 0x26, 0x23, 0x67, 0x37, 0x13, 0x99, 0xac, 0x0c, 0x9b, 0x89, 0x4c, 0x5a, 0xce, - 0x54, 0xbe, 0x89, 0xc1, 0x75, 0xdf, 0x9e, 0x0d, 0x7d, 0x68, 0x98, 0x93, 0xf7, 0x75, 0x87, 0x8f, - 0xc2, 0xdd, 0x71, 0x0b, 0xf2, 0x03, 0x86, 0xdd, 0xa7, 0x6a, 0xae, 0x12, 0x2f, 0xc7, 0xa9, 0xbd, - 0x3e, 0xad, 0x43, 0x49, 0xe8, 0x0b, 0x00, 0x2e, 0x62, 0x60, 0x3f, 0x3b, 0x0b, 0xf5, 0x9b, 0xa7, - 0x27, 0x2b, 0x59, 0xe1, 0x57, 0x77, 0xca, 0xc9, 0x59, 0x5f, 0x58, 0xc5, 0x2e, 0xea, 0xc2, 0xa2, - 0x70, 0x46, 0x80, 0xc0, 0x3c, 0x52, 0xa8, 0x7f, 0xc8, 0x6d, 0x2a, 0x35, 0x7d, 0x01, 0xa1, 0x3e, - 0x05, 0x55, 0xc2, 0x53, 0x4c, 0x5c, 0xf9, 0x53, 0x0c, 0xae, 0xaa, 0x96, 0x47, 0x1c, 0x93, 0xe8, - 0x47, 0x24, 0xe2, 0x88, 0x5f, 0x41, 0x56, 0xb7, 0x06, 0xc4, 0xf5, 0x6c, 0xc7, 0x55, 0xa4, 0x72, - 0x7c, 0x35, 0xb7, 0xfe, 0xf9, 0x9c, 0xab, 0x3d, 0x4f, 0xbf, 0x5a, 0xe3, 0xca, 0xa2, 0x42, 0x05, - 0x60, 0x4b, 0x7f, 0x95, 0x20, 0x23, 0xb8, 0xe8, 0x1e, 0x64, 0x58, 0x72, 0xd1, 0x73, 0xf8, 0x89, - 0x77, 0x8d, 0x9f, 0x23, 0xdd, 0xa3, 0x74, 0x66, 0x3f, 0xcd, 0xc1, 0x34, 0x13, 0x53, 0x31, 0xfa, - 0x09, 0x64, 0x58, 0x9e, 0xf5, 0x83, 0xdb, 0x58, 0x12, 0x1a, 0x3c, 0x11, 0xa3, 0x39, 0x99, 0x66, - 0xb2, 0x2a, 0x46, 0x8d, 0xf3, 0xd2, 0x25, 0xce, 0xf4, 0x6f, 0x08, 0xcf, 0xed, 0x4c, 0x27, 0xcc, - 0x6c, 0x06, 0xfd, 0x2d, 0x0d, 0x25, 0x86, 0x7c, 0xa1, 0x88, 0xf9, 0x38, 0x12, 0x31, 0xd7, 0xa6, - 0x22, 0x26, 0x30, 0x8f, 0x06, 0xcc, 0x07, 0x90, 0x1a, 0x5b, 0xc6, 0xab, 0xb1, 0x9f, 0x41, 0x41, - 0xf4, 0xfa, 0xb4, 0x99, 0x70, 0x4a, 0xcc, 0x86, 0xd3, 0x27, 0x80, 0xa8, 0x2f, 0x49, 0x7f, 0x4a, - 0x30, 0xc9, 0x04, 0x65, 0xc6, 0x69, 0xcc, 0x0d, 0xbe, 0xd4, 0x25, 0x82, 0xef, 0x11, 0xc8, 0xe4, - 0xd8, 0x73, 0xf4, 0x7e, 0x44, 0x3f, 0xcd, 0xf4, 0x97, 0x4f, 0x4f, 0x56, 0x8a, 0x2d, 0xca, 0x3b, - 0x1f, 0xa4, 0x48, 0x22, 0x3c, 0xec, 0xa2, 0xdf, 0xc0, 0x22, 0xc7, 0xc0, 0x86, 0x43, 0x06, 0x1e, - 0xeb, 0x52, 0x99, 0x72, 0x7c, 0xb5, 0xb8, 0x7e, 0x6f, 0x6e, 0x90, 0x4d, 0xb9, 0xbd, 0xda, 0x14, - 0x8a, 0x9a, 0xec, 0x43, 0x05, 0x04, 0x17, 0x3d, 0x81, 0xdc, 0xbe, 0x5f, 0xe1, 0xfb, 0x2f, 0xc9, - 0x44, 0xc9, 0xb2, 0xc2, 0x74, 0x77, 0x0e, 0xf0, 0x39, 0xbd, 0x80, 0x5f, 0x01, 0xec, 0x07, 0x2c, - 0xb4, 0x0b, 0x05, 0x47, 0xb0, 0x71, 0x7f, 0x6f, 0xa2, 0x00, 0x4b, 0x89, 0xcb, 0x83, 0xe6, 0x43, - 0x98, 0xfa, 0x04, 0x3d, 0x01, 0x30, 0x82, 0xec, 0x51, 0x72, 0xcc, 0xd0, 0x1f, 0x5f, 0x22, 0xcd, - 0x84, 0xa5, 0x21, 0x08, 0x7a, 0x06, 0xc5, 0xf0, 0x89, 0x99, 0x9a, 0x7f, 0x47, 0x53, 0x0b, 0x11, - 0x9c, 0xfa, 0x04, 0xf5, 0xe0, 0xea, 0xc0, 0x1e, 0x8e, 0x6c, 0xd7, 0xf0, 0x48, 0x34, 0x04, 0x0a, - 0x2c, 0x04, 0x2a, 0xa7, 0x27, 0x2b, 0xa8, 0x21, 0xf8, 0xe7, 0x87, 0x01, 0x1a, 0x9c, 0xe1, 0xfb, - 0x41, 0x35, 0x15, 0xbc, 0x14, 0xb1, 0x18, 0x06, 0xd5, 0x4e, 0x18, 0xbe, 0x33, 0x41, 0x15, 0x09, - 0x6d, 0x15, 0xbb, 0x95, 0x65, 0xc8, 0x06, 0x31, 0x40, 0x47, 0x84, 0xda, 0x4e, 0x43, 0x5e, 0x60, - 0x83, 0x49, 0x6b, 0xa7, 0x21, 0x4b, 0x95, 0xbf, 0x27, 0x00, 0x85, 0x9e, 0xdb, 0x1a, 0x7b, 0x3a, - 0x93, 0xac, 0x41, 0xca, 0x7f, 0x35, 0xcb, 0xe0, 0xdc, 0xfa, 0x8f, 0xde, 0xda, 0xc0, 0x42, 0x80, - 0x47, 0x0b, 0x1a, 0x57, 0x44, 0x5f, 0x46, 0x07, 0x83, 0xdc, 0xdc, 0xb1, 0xef, 0x4c, 0x08, 0x3f, - 0x5a, 0x10, 0x13, 0xc3, 0x63, 0x48, 0xba, 0x9e, 0xee, 0xf9, 0x05, 0xa0, 0xb8, 0xbe, 0x36, 0x47, - 0x7f, 0xd6, 0xf8, 0xea, 0x0e, 0x55, 0x13, 0x43, 0x22, 0xc3, 0x40, 0xcf, 0x20, 0x1b, 0x24, 0x15, - 0x9f, 0x32, 0xee, 0x5f, 0x1c, 0x30, 0xf0, 0xa0, 0xa8, 0xdb, 0x01, 0x16, 0xaa, 0x41, 0x6e, 0xc8, - 0xc5, 0xc2, 0xae, 0x53, 0xe6, 0x75, 0x0d, 0x04, 0x02, 0xab, 0x6f, 0x91, 0x27, 0x0d, 0x84, 0x92, - 0x8a, 0xd1, 0x03, 0xc8, 0x3b, 0xc4, 0x1d, 0x0f, 0x49, 0xdf, 0x1d, 0xe9, 0x96, 0x5f, 0x7d, 0x72, - 0xeb, 0x37, 0x22, 0xe6, 0xf1, 0x59, 0xbe, 0xba, 0x33, 0xd2, 0x85, 0x09, 0x39, 0x5f, 0x85, 0x52, - 0xdc, 0xca, 0x03, 0x48, 0xb2, 0x33, 0xd3, 0xe9, 0x73, 0xb7, 0xf3, 0xb8, 0xd3, 0x7d, 0xd6, 0x91, - 0x17, 0xe8, 0xfc, 0xd8, 0x6c, 0xb5, 0x5b, 0xbd, 0x56, 0xbf, 0xdb, 0x69, 0x3f, 0x97, 0x25, 0x74, - 0x13, 0xae, 0x71, 0x42, 0xad, 0xd3, 0xec, 0x3f, 0xd3, 0x54, 0xc1, 0x8a, 0x55, 0x56, 0xa3, 0x61, - 0x42, 0x07, 0xc8, 0x6e, 0xa7, 0xe5, 0x4f, 0xb8, 0xb5, 0x66, 0x53, 0x96, 0x58, 0xc0, 0x68, 0xdd, - 0x6d, 0x39, 0x56, 0xcf, 0x03, 0xe0, 0xc0, 0x43, 0x95, 0xbf, 0x5c, 0x81, 0x12, 0x6b, 0x49, 0x17, - 0xaa, 0xfd, 0x65, 0x56, 0xfb, 0xfd, 0xfe, 0x22, 0x4f, 0xd5, 0xfe, 0x18, 0x2f, 0xfb, 0xf7, 0x21, - 0x3b, 0xd2, 0x1d, 0x62, 0x79, 0xd4, 0x99, 0x89, 0xa9, 0xb1, 0x22, 0xb3, 0xcd, 0x18, 0x81, 0x78, - 0xc6, 0x17, 0x54, 0xa9, 0x52, 0xfa, 0x88, 0x38, 0x6c, 0xfe, 0xf7, 0xfd, 0x7f, 0x93, 0x0f, 0xa1, - 0x8b, 0xa1, 0x55, 0x4f, 0x7d, 0x01, 0x4d, 0x48, 0xa2, 0x0f, 0x01, 0xc6, 0xa3, 0xbe, 0xd0, 0x8b, - 0x8e, 0x48, 0xd9, 0xf1, 0x88, 0x4b, 0xa3, 0x6d, 0x58, 0x1c, 0xda, 0xd8, 0xd8, 0x37, 0x06, 0xfe, - 0x0d, 0xd3, 0x4d, 0x49, 0x49, 0xb3, 0x78, 0xfe, 0x61, 0xe4, 0x7e, 0xe8, 0x26, 0x55, 0x3d, 0x34, - 0x07, 0xd5, 0x9e, 0xd8, 0xa4, 0x38, 0x94, 0x1c, 0xd5, 0xa6, 0x4c, 0xf4, 0x10, 0xd2, 0x7e, 0x7e, - 0xf8, 0xa5, 0xfd, 0xe2, 0x99, 0xc5, 0x11, 0x85, 0x36, 0xda, 0x80, 0xa2, 0x45, 0x8e, 0xa3, 0x13, - 0x4f, 0x76, 0x2a, 0xf6, 0xf2, 0x1d, 0x72, 0x7c, 0xfe, 0xb8, 0x93, 0xb7, 0x42, 0x0e, 0x46, 0x4f, - 0xa0, 0x30, 0x72, 0x8c, 0xa1, 0xee, 0x4c, 0xfa, 0x7e, 0xba, 0xc2, 0x65, 0xd2, 0x55, 0xd4, 0x6f, - 0x0e, 0xc1, 0xb8, 0x68, 0x03, 0xfc, 0x01, 0x83, 0xb8, 0x4a, 0x8e, 0x9d, 0xf1, 0x72, 0x60, 0x42, - 0x19, 0xd5, 0xa1, 0xc0, 0x8e, 0x18, 0x4c, 0x36, 0x79, 0x76, 0xc2, 0x65, 0x7e, 0xc2, 0x1c, 0x3d, - 0xe1, 0x39, 0xd3, 0x4d, 0xce, 0x0a, 0xe8, 0x18, 0x6d, 0x02, 0x04, 0x3b, 0x2b, 0xad, 0xca, 0x6f, - 0x6b, 0x7a, 0xdb, 0x42, 0x30, 0x34, 0x49, 0x8b, 0x68, 0xa3, 0x2d, 0xc8, 0x8a, 0xb4, 0xf5, 0xcb, - 0x71, 0x6e, 0xee, 0xaa, 0x32, 0x5b, 0x44, 0x44, 0x70, 0x05, 0x08, 0xa8, 0x03, 0x49, 0x93, 0xe8, - 0x2e, 0x51, 0x4a, 0xcc, 0xaa, 0x2f, 0xe6, 0x40, 0x9d, 0x49, 0xaf, 0xea, 0xce, 0xe0, 0x90, 0x0c, - 0xf5, 0xc6, 0xa1, 0x6e, 0x1d, 0x90, 0x36, 0xd5, 0xd7, 0x7c, 0x18, 0xd4, 0x01, 0x99, 0xb9, 0x2b, - 0x5a, 0x8f, 0x64, 0xe6, 0xb1, 0x8f, 0xb8, 0xc7, 0x8a, 0xd4, 0x63, 0x73, 0x6b, 0x12, 0x8b, 0xa7, - 0xad, 0xb0, 0x2e, 0xfd, 0x02, 0x8a, 0xfb, 0xb6, 0x33, 0xd4, 0xbd, 0x20, 0x4b, 0x16, 0xc3, 0xa9, - 0xed, 0xcd, 0xc9, 0x4a, 0x61, 0x83, 0x71, 0x45, 0x66, 0x15, 0xf6, 0xa3, 0x8f, 0xe8, 0x91, 0x28, - 0xdf, 0x57, 0x58, 0xb5, 0xfd, 0xe4, 0xa2, 0xa7, 0x9b, 0xad, 0xdd, 0x1d, 0x48, 0x0d, 0x0e, 0xc9, - 0xe0, 0xa5, 0xab, 0x5c, 0x65, 0x3e, 0xff, 0xe9, 0x05, 0xa1, 0x1a, 0x54, 0x29, 0xdc, 0x19, 0x35, - 0x8e, 0x82, 0x9e, 0x40, 0xda, 0x21, 0xfe, 0x38, 0x78, 0x8d, 0x01, 0x7e, 0x76, 0x41, 0x40, 0x8d, - 0x69, 0xa9, 0xd6, 0xbe, 0x2d, 0x22, 0x95, 0xe3, 0xa0, 0x2e, 0x64, 0xf6, 0xe9, 0xba, 0x63, 0x10, - 0x57, 0xb9, 0xce, 0x30, 0x3f, 0x7d, 0x6b, 0x5a, 0x9f, 0xdd, 0xb0, 0xc4, 0x7e, 0x27, 0x40, 0x82, - 0xec, 0x66, 0x84, 0x09, 0xbd, 0xc9, 0x1b, 0xb3, 0xd9, 0x2d, 0x36, 0xac, 0xa9, 0x6d, 0x8b, 0x65, - 0x37, 0x7f, 0xc2, 0xb4, 0xca, 0x1d, 0x19, 0xe4, 0xeb, 0xfe, 0xab, 0x31, 0x71, 0x26, 0x8a, 0x12, - 0xa9, 0xc8, 0x59, 0x4a, 0x7f, 0x42, 0xc9, 0xe8, 0x33, 0xc8, 0x62, 0x32, 0x22, 0x16, 0x76, 0xbb, - 0x96, 0x72, 0x93, 0x8d, 0x19, 0x57, 0xe8, 0xec, 0xdb, 0x14, 0x44, 0x5e, 0x71, 0x43, 0x29, 0xf4, - 0x82, 0xee, 0x9f, 0xf4, 0x81, 0xe0, 0xae, 0x55, 0x9f, 0x28, 0x4b, 0xec, 0xd0, 0xf7, 0x2e, 0xec, - 0x48, 0x31, 0x53, 0x5d, 0x15, 0xe7, 0x69, 0x46, 0xd0, 0xb4, 0x29, 0x6c, 0xf4, 0x6b, 0xc8, 0x8b, - 0x90, 0xde, 0xb4, 0xf7, 0x5c, 0xe5, 0x07, 0xec, 0x5d, 0xeb, 0x17, 0x7c, 0xd7, 0x56, 0xa8, 0x2a, - 0x8a, 0x55, 0x14, 0x6d, 0xe9, 0x3b, 0x09, 0x16, 0x67, 0x52, 0x0a, 0xfd, 0x16, 0xd2, 0x96, 0x8d, - 0x23, 0x0b, 0x58, 0x8b, 0x1b, 0x9a, 0xea, 0xd8, 0xd8, 0xdf, 0xbf, 0xee, 0x1f, 0x18, 0xde, 0xe1, - 0x78, 0xaf, 0x3a, 0xb0, 0x87, 0x6b, 0x81, 0x29, 0x78, 0x6f, 0xed, 0xdc, 0xcf, 0x70, 0x55, 0x5f, - 0x4d, 0x4b, 0x51, 0x54, 0x15, 0xa3, 0x4f, 0xa1, 0x44, 0x8e, 0x47, 0x86, 0x13, 0x69, 0x2b, 0x74, - 0x4c, 0x8a, 0x73, 0x13, 0x8b, 0x21, 0x93, 0x76, 0x8d, 0xa5, 0x3f, 0x48, 0x50, 0x3a, 0x13, 0xce, - 0xb4, 0xcd, 0xb2, 0xd5, 0x7f, 0xaa, 0xcd, 0x52, 0x4a, 0xd0, 0x80, 0x63, 0x6f, 0xfd, 0xd6, 0x12, - 0x7f, 0xcf, 0x6f, 0x2d, 0x4b, 0x26, 0x40, 0x98, 0x11, 0xe8, 0x97, 0x50, 0xb0, 0x4d, 0xdc, 0x0f, - 0xbb, 0xb7, 0x14, 0xb6, 0x62, 0x5a, 0xac, 0xbb, 0x26, 0x3e, 0xd3, 0xc0, 0x73, 0x76, 0x40, 0xc2, - 0x68, 0x05, 0x32, 0x54, 0x7d, 0xc6, 0xee, 0xb4, 0x6d, 0x62, 0xba, 0xa2, 0x2d, 0x7d, 0x2b, 0x41, - 0x36, 0xfa, 0x5d, 0x2a, 0x16, 0xbc, 0xe2, 0xfc, 0x49, 0xe2, 0x1d, 0x37, 0xe2, 0xe9, 0x45, 0x30, - 0x7e, 0xf1, 0x45, 0x70, 0xe9, 0x08, 0x72, 0x91, 0x58, 0x3b, 0x3b, 0x18, 0x4a, 0xef, 0x30, 0x18, - 0x7e, 0x04, 0xa9, 0x17, 0xf6, 0x9e, 0x38, 0x40, 0xbc, 0x5e, 0xe0, 0xda, 0xc9, 0x4d, 0x7b, 0x4f, - 0x6d, 0x6a, 0xc9, 0x17, 0xf6, 0x9e, 0x8a, 0x2b, 0xb7, 0xc5, 0xf0, 0x07, 0x90, 0xda, 0xde, 0xad, - 0xb7, 0xd5, 0xc6, 0xb9, 0x83, 0xdb, 0xcf, 0x12, 0xdf, 0xfc, 0x71, 0x45, 0xda, 0x4c, 0x64, 0x90, - 0x7c, 0xa5, 0xf2, 0x9d, 0x04, 0xa8, 0xa9, 0x7b, 0x3a, 0xbd, 0xea, 0x4b, 0x4c, 0x6e, 0xb1, 0xb7, - 0xf8, 0x7b, 0xba, 0xd1, 0xc6, 0xdf, 0xa7, 0xd1, 0xfa, 0xa6, 0x56, 0xbe, 0x95, 0x00, 0x22, 0xc6, - 0x7d, 0x19, 0xfd, 0x14, 0x39, 0x7f, 0xa6, 0x38, 0x93, 0xff, 0x74, 0x9f, 0xf0, 0x3f, 0x54, 0x3e, - 0x84, 0x0c, 0xe6, 0x47, 0xe6, 0x2b, 0xc9, 0xdc, 0xe6, 0x3d, 0xe3, 0x99, 0x47, 0x0b, 0x5a, 0xa0, - 0x5c, 0x4f, 0x43, 0x72, 0x6c, 0x19, 0xb6, 0x75, 0xf7, 0x73, 0x40, 0xb3, 0x69, 0x82, 0x0a, 0x90, - 0x65, 0xff, 0x75, 0x8f, 0x60, 0x7f, 0x0a, 0xdf, 0xb5, 0x8e, 0x02, 0x82, 0x54, 0xbf, 0xf5, 0xfa, - 0x3f, 0xcb, 0x0b, 0xaf, 0x4f, 0x97, 0xa5, 0x7f, 0x9c, 0x2e, 0x4b, 0xff, 0x3c, 0x5d, 0x96, 0xfe, - 0x7d, 0xba, 0x2c, 0xfd, 0xfe, 0xbf, 0xcb, 0x0b, 0x5f, 0xa5, 0xb9, 0x01, 0xff, 0x0f, 0x00, 0x00, - 0xff, 0xff, 0x04, 0xf3, 0x7d, 0x3c, 0x0e, 0x18, 0x00, 0x00, + // 2289 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x58, 0x4b, 0x73, 0xdb, 0xd6, + 0xf5, 0x17, 0xf8, 0x04, 0x0e, 0x5f, 0xd0, 0xf5, 0x23, 0xb0, 0xfe, 0xf9, 0x4b, 0x34, 0x93, 0xb8, + 0x8a, 0x9b, 0x50, 0x8e, 0x9c, 0x76, 0x32, 0x7d, 0x64, 0xc2, 0x97, 0x6c, 0xc8, 0x14, 0x29, 0x43, + 0x94, 0x5d, 0x67, 0xda, 0x72, 0x20, 0xe2, 0x4a, 0x82, 0x0d, 0x02, 0x34, 0x00, 0x2a, 0xe2, 0x17, + 0xe8, 0x64, 0xd9, 0xe9, 0x07, 0xc8, 0x74, 0xb2, 0xed, 0xb6, 0x8b, 0x4e, 0xa7, 0x1f, 0xc0, 0xcb, + 0x2c, 0xbb, 0xd2, 0xb4, 0xca, 0x87, 0xe8, 0x8c, 0x57, 0x9d, 0x7b, 0x71, 0x2f, 0x00, 0x8a, 0xa2, + 0x4b, 0xd9, 0x0b, 0xce, 0x10, 0xe7, 0xf1, 0xc3, 0xb9, 0xe7, 0x9e, 0x27, 0xe0, 0xee, 0xc0, 0x19, + 0xbc, 0x70, 0x1d, 0x7d, 0x70, 0xbc, 0x31, 0x7a, 0x71, 0xb4, 0xe1, 0xbd, 0xb4, 0xc8, 0xef, 0x40, + 0xf7, 0xf0, 0x86, 0xe7, 0xbb, 0xe3, 0x81, 0x3f, 0x76, 0xb1, 0x51, 0x1d, 0xb9, 0x8e, 0xef, 0xa0, + 0x1b, 0xa1, 0x6c, 0xd5, 0x7b, 0x69, 0x55, 0x99, 0xdc, 0x4a, 0x79, 0x1a, 0x82, 0xfe, 0x1b, 0x1d, + 0x6c, 0x18, 0xba, 0xaf, 0x07, 0x8a, 0x2b, 0x77, 0xa6, 0x25, 0xc6, 0xbe, 0x69, 0x6d, 0x1c, 0x5b, + 0x83, 0x0d, 0xdf, 0x1c, 0x62, 0xcf, 0xd7, 0x87, 0x23, 0x26, 0xf7, 0xf1, 0x7c, 0x63, 0x46, 0xae, + 0x79, 0x62, 0x5a, 0xf8, 0x08, 0x33, 0xd1, 0xeb, 0x47, 0xce, 0x91, 0x43, 0xff, 0x6e, 0x90, 0x7f, + 0x01, 0xb5, 0xf2, 0x43, 0x12, 0xa0, 0xe1, 0x58, 0xe3, 0xa1, 0xdd, 0x9b, 0x8c, 0x30, 0x7a, 0x06, + 0x05, 0x0f, 0x0f, 0x75, 0xdb, 0x37, 0x07, 0x7d, 0x7f, 0x32, 0xc2, 0x8a, 0x50, 0x16, 0xd6, 0x8b, + 0x9b, 0xd5, 0xea, 0xa5, 0x07, 0xa9, 0x46, 0x9a, 0xd5, 0x3d, 0xa6, 0x46, 0x1e, 0xea, 0xa9, 0x57, + 0x67, 0x6b, 0x4b, 0x5a, 0xde, 0x8b, 0xd1, 0xd0, 0x0a, 0xa4, 0xbf, 0x31, 0x0d, 0xff, 0x58, 0x49, + 0x94, 0x85, 0xf5, 0x34, 0x13, 0x09, 0x48, 0xa8, 0x02, 0xd2, 0xc8, 0xc5, 0x03, 0xd3, 0x33, 0x1d, + 0x5b, 0x49, 0xc6, 0xf8, 0x11, 0x19, 0x7d, 0x0c, 0xb2, 0xee, 0xba, 0xfa, 0xa4, 0x6f, 0x98, 0x43, + 0x6c, 0x13, 0x92, 0xa7, 0xa4, 0xca, 0xc9, 0xf5, 0xb4, 0x56, 0xa2, 0xf4, 0x66, 0x48, 0x46, 0x37, + 0x21, 0x63, 0x39, 0x03, 0xdd, 0xc2, 0x4a, 0xba, 0x2c, 0xac, 0x4b, 0x1a, 0x7b, 0xaa, 0xfc, 0x28, + 0x40, 0x3e, 0x6e, 0x27, 0x12, 0x21, 0x55, 0xef, 0x76, 0xdb, 0xf2, 0x12, 0xca, 0x42, 0x52, 0xed, + 0xf4, 0x64, 0x01, 0x49, 0x90, 0xde, 0x6a, 0x77, 0x6b, 0x3d, 0x39, 0x81, 0x72, 0x90, 0x6d, 0xb6, + 0x1a, 0xea, 0x4e, 0xad, 0x2d, 0x27, 0x89, 0x68, 0xb3, 0xd6, 0x6b, 0xc9, 0x29, 0x54, 0x00, 0xa9, + 0xa7, 0xee, 0xb4, 0xf6, 0x7a, 0xb5, 0x9d, 0x5d, 0x39, 0x8d, 0xf2, 0x20, 0xaa, 0x9d, 0x5e, 0x4b, + 0x7b, 0x52, 0x6b, 0xcb, 0x19, 0x04, 0x90, 0xd9, 0xeb, 0x69, 0x6a, 0xe7, 0x81, 0x9c, 0x25, 0x50, + 0xf5, 0x67, 0xbd, 0xd6, 0x9e, 0x2c, 0xa2, 0x12, 0xe4, 0x42, 0x9d, 0xde, 0xd7, 0xb2, 0x84, 0x10, + 0x14, 0x1b, 0xdd, 0x76, 0xbb, 0xd6, 0x6b, 0x35, 0x99, 0x3c, 0x90, 0x57, 0x74, 0x6a, 0x3b, 0x2d, + 0x39, 0x47, 0xac, 0xe9, 0xaa, 0x4d, 0x39, 0x4f, 0x49, 0xfb, 0xed, 0xb6, 0x5c, 0x20, 0xff, 0xf6, + 0xf7, 0xd5, 0xa6, 0x5c, 0x24, 0xef, 0x57, 0x3b, 0xbd, 0x7e, 0x4d, 0xd3, 0x6a, 0xcf, 0x64, 0x03, + 0x95, 0x00, 0xd4, 0x4e, 0x6f, 0xf3, 0x49, 0xab, 0xd1, 0xeb, 0x6a, 0xf2, 0x2b, 0xa1, 0xf2, 0x1f, + 0x01, 0xae, 0x6d, 0x39, 0x2e, 0x36, 0x8f, 0xec, 0x47, 0x78, 0xa2, 0xe1, 0x43, 0xec, 0x62, 0x7b, + 0x80, 0x51, 0x19, 0xd2, 0xbe, 0x7e, 0x60, 0x05, 0x77, 0x5a, 0xa8, 0x03, 0x71, 0xf0, 0xeb, 0xb3, + 0xb5, 0x84, 0xda, 0xd4, 0x02, 0x06, 0xfa, 0x08, 0xd2, 0xa6, 0x6d, 0xe0, 0x53, 0x7a, 0x45, 0x85, + 0x7a, 0x89, 0x49, 0x64, 0x55, 0x42, 0x24, 0x62, 0x94, 0x8b, 0x14, 0x48, 0xd9, 0xfa, 0x10, 0xd3, + 0x8b, 0x92, 0xd8, 0x45, 0x51, 0x0a, 0x7a, 0x04, 0xe2, 0x89, 0x6e, 0x99, 0x86, 0xe9, 0x4f, 0x94, + 0x14, 0x8d, 0x9c, 0x8f, 0xe7, 0x46, 0x8e, 0xed, 0xf9, 0xae, 0x6e, 0xda, 0xfe, 0x13, 0xa6, 0xc0, + 0x80, 0x42, 0x00, 0x74, 0x0f, 0x96, 0xbd, 0x63, 0xdd, 0xc5, 0x46, 0x7f, 0xe4, 0xe2, 0x43, 0xf3, + 0xb4, 0x6f, 0x61, 0x9b, 0x5e, 0x28, 0x0f, 0x8e, 0x52, 0xc0, 0xde, 0xa5, 0xdc, 0x36, 0xb6, 0x2b, + 0x7f, 0x48, 0x80, 0x1c, 0x84, 0x64, 0x13, 0x7b, 0x03, 0xd7, 0x1c, 0xf9, 0x8e, 0x1b, 0x5a, 0x2b, + 0xcc, 0x58, 0x7b, 0x07, 0x12, 0xa6, 0xc1, 0xce, 0x7a, 0x93, 0xd0, 0xcf, 0xa9, 0x37, 0x5e, 0x9f, + 0xad, 0x89, 0x01, 0x8a, 0xda, 0xd4, 0x12, 0xa6, 0x81, 0x7e, 0x09, 0x29, 0x9a, 0x0b, 0xe4, 0xbc, + 0xb9, 0xcd, 0xdb, 0xff, 0x33, 0x17, 0xf8, 0x4b, 0x88, 0x12, 0x2a, 0x83, 0x68, 0x8f, 0x2d, 0x8b, + 0x3a, 0x9e, 0xb8, 0x44, 0xe4, 0xe7, 0xe4, 0x54, 0x74, 0x1b, 0xf2, 0x06, 0x3e, 0xd4, 0xc7, 0x96, + 0xdf, 0xc7, 0xa7, 0x23, 0x97, 0xc5, 0x6c, 0x8e, 0xd1, 0x5a, 0xa7, 0x23, 0x17, 0xbd, 0x0f, 0x99, + 0x63, 0xd3, 0x30, 0xb0, 0xad, 0x64, 0x62, 0x10, 0x8c, 0xb6, 0x9d, 0x12, 0x45, 0x59, 0xda, 0x4e, + 0x89, 0x92, 0x0c, 0xdb, 0x29, 0x31, 0x2b, 0x8b, 0x95, 0x6f, 0x13, 0x70, 0x33, 0xb0, 0x67, 0x4b, + 0x1f, 0x9a, 0xd6, 0xe4, 0x5d, 0xdd, 0x11, 0xa0, 0x30, 0x77, 0xdc, 0x86, 0xfc, 0x80, 0x62, 0xf7, + 0x89, 0x9a, 0xa7, 0x24, 0xcb, 0x49, 0x62, 0x6f, 0x40, 0xeb, 0x10, 0x12, 0xfa, 0x02, 0x80, 0x89, + 0x98, 0x46, 0x90, 0xa5, 0x85, 0xfa, 0xad, 0xf3, 0xb3, 0x35, 0x89, 0xfb, 0xd5, 0x9b, 0x72, 0xb2, + 0x14, 0x08, 0xab, 0x86, 0x87, 0xba, 0xb0, 0xcc, 0x9d, 0x11, 0x22, 0x50, 0x8f, 0x14, 0xea, 0x1f, + 0x30, 0x9b, 0x4a, 0xcd, 0x40, 0x80, 0xab, 0x4f, 0x41, 0x95, 0x8c, 0x29, 0xa6, 0x51, 0xf9, 0x4b, + 0x02, 0xae, 0xab, 0xb6, 0x8f, 0x5d, 0x0b, 0xeb, 0x27, 0x38, 0xe6, 0x88, 0xdf, 0x80, 0xa4, 0xdb, + 0x03, 0xec, 0xf9, 0x8e, 0xeb, 0x29, 0x42, 0x39, 0xb9, 0x9e, 0xdb, 0xfc, 0x7c, 0xce, 0xd5, 0x5e, + 0xa6, 0x5f, 0xad, 0x31, 0x65, 0x5e, 0xa9, 0x42, 0xb0, 0x95, 0xbf, 0x09, 0x20, 0x72, 0x2e, 0xba, + 0x07, 0x22, 0x4d, 0x2e, 0x72, 0x8e, 0x20, 0xf1, 0x6e, 0xb0, 0x73, 0x64, 0x7b, 0x84, 0x4e, 0xed, + 0x27, 0x39, 0x98, 0xa5, 0x62, 0xaa, 0x81, 0x7e, 0x06, 0x22, 0xcd, 0xb3, 0x7e, 0x78, 0x1b, 0x2b, + 0x5c, 0x83, 0x25, 0x62, 0x3c, 0x27, 0xb3, 0x54, 0x56, 0x35, 0x50, 0xe3, 0xb2, 0x74, 0x49, 0x52, + 0xfd, 0xf7, 0xb8, 0xe7, 0xf6, 0xa6, 0x13, 0x66, 0x36, 0x83, 0xfe, 0x9e, 0x85, 0x12, 0x45, 0x5e, + 0x28, 0x62, 0x3e, 0x8a, 0x45, 0xcc, 0x8d, 0xa9, 0x88, 0x09, 0xcd, 0x23, 0x01, 0xf3, 0x3e, 0x64, + 0xc6, 0xb6, 0xf9, 0x72, 0x1c, 0x64, 0x50, 0x18, 0xbd, 0x01, 0x6d, 0x26, 0x9c, 0x52, 0xb3, 0xe1, + 0xf4, 0x09, 0x20, 0xe2, 0x4b, 0xdc, 0x9f, 0x12, 0x4c, 0x53, 0x41, 0x99, 0x72, 0x1a, 0x73, 0x83, + 0x2f, 0x73, 0x85, 0xe0, 0x7b, 0x08, 0x32, 0x3e, 0xf5, 0x5d, 0xbd, 0x1f, 0xd3, 0xcf, 0x52, 0xfd, + 0xd5, 0xf3, 0xb3, 0xb5, 0x62, 0x8b, 0xf0, 0x2e, 0x07, 0x29, 0xe2, 0x18, 0xcf, 0xf0, 0xd0, 0xef, + 0x60, 0x99, 0x61, 0x18, 0xa6, 0x8b, 0x07, 0x3e, 0xed, 0x56, 0x62, 0x39, 0xb9, 0x5e, 0xdc, 0xbc, + 0x37, 0x37, 0xc8, 0xa6, 0xdc, 0x5e, 0x6d, 0x72, 0x45, 0x4d, 0x0e, 0xa0, 0x42, 0x82, 0x87, 0x1e, + 0x43, 0xee, 0x30, 0xa8, 0xf0, 0xfd, 0x17, 0x78, 0xa2, 0x48, 0xb4, 0x30, 0xdd, 0x9d, 0x03, 0x7c, + 0x49, 0x2f, 0x60, 0x57, 0x00, 0x87, 0x21, 0x0b, 0xed, 0x43, 0xc1, 0xe5, 0x6c, 0xa3, 0x7f, 0x30, + 0x51, 0x80, 0xa6, 0xc4, 0xd5, 0x41, 0xf3, 0x11, 0x4c, 0x7d, 0x82, 0x1e, 0x03, 0x98, 0x61, 0xf6, + 0x28, 0x39, 0x6a, 0xe8, 0x4f, 0xaf, 0x90, 0x66, 0xdc, 0xd2, 0x08, 0x04, 0x3d, 0x85, 0x62, 0xf4, + 0x44, 0x4d, 0xcd, 0xbf, 0xa5, 0xa9, 0x85, 0x18, 0x4e, 0x7d, 0x82, 0x7a, 0x70, 0x7d, 0xe0, 0x0c, + 0x47, 0x8e, 0x67, 0xfa, 0x38, 0x1e, 0x02, 0x05, 0x1a, 0x02, 0x95, 0xf3, 0xb3, 0x35, 0xd4, 0xe0, + 0xfc, 0xcb, 0xc3, 0x00, 0x0d, 0x2e, 0xf0, 0x83, 0xa0, 0x9a, 0x0a, 0x5e, 0x82, 0x58, 0x8c, 0x82, + 0x6a, 0x2f, 0x0a, 0xdf, 0x99, 0xa0, 0x8a, 0x85, 0xb6, 0x6a, 0x78, 0x95, 0x55, 0x90, 0xc2, 0x18, + 0x20, 0x23, 0x42, 0x6d, 0xaf, 0x21, 0x2f, 0xd1, 0xc1, 0xa4, 0xb5, 0xd7, 0x90, 0x85, 0xca, 0x3f, + 0x52, 0x80, 0x22, 0xcf, 0xed, 0x8c, 0x7d, 0x9d, 0x4a, 0xd6, 0x20, 0x13, 0xbc, 0x9a, 0x66, 0x70, + 0x6e, 0xf3, 0x27, 0x6f, 0x6c, 0x60, 0x11, 0xc0, 0xc3, 0x25, 0x8d, 0x29, 0xa2, 0x2f, 0xe3, 0x83, + 0x41, 0x6e, 0xf3, 0xce, 0x62, 0x21, 0xfc, 0x70, 0x89, 0x4f, 0x0c, 0x8f, 0x20, 0xed, 0xf9, 0xba, + 0x1f, 0x14, 0x80, 0xe2, 0xe6, 0xc6, 0x1c, 0xfd, 0x59, 0xe3, 0xab, 0x7b, 0x44, 0x8d, 0x0f, 0x8b, + 0x14, 0x03, 0x3d, 0x05, 0x29, 0x4c, 0x2a, 0x36, 0x65, 0xdc, 0x5f, 0x1c, 0x30, 0xf4, 0x20, 0xaf, + 0xdb, 0x21, 0x16, 0xaa, 0x41, 0x6e, 0xc8, 0xc4, 0xa2, 0xae, 0x53, 0x66, 0x75, 0x0d, 0x38, 0x02, + 0xad, 0x6f, 0xb1, 0x27, 0x0d, 0xb8, 0x92, 0x6a, 0xa0, 0xaf, 0x20, 0xef, 0x62, 0x6f, 0x3c, 0xc4, + 0x7d, 0x6f, 0xa4, 0xdb, 0x41, 0xf5, 0xc9, 0x6d, 0xbe, 0x17, 0x33, 0x8f, 0x0d, 0xfb, 0xd5, 0xbd, + 0x91, 0xce, 0x4d, 0xc8, 0x05, 0x2a, 0x84, 0xe2, 0x55, 0xbe, 0x82, 0x34, 0x3d, 0x33, 0x99, 0x3e, + 0xf7, 0x3b, 0x8f, 0x3a, 0xdd, 0xa7, 0x1d, 0x79, 0x89, 0xcc, 0x8f, 0xcd, 0x56, 0xbb, 0xd5, 0x6b, + 0xf5, 0xbb, 0x9d, 0xf6, 0x33, 0x59, 0x40, 0xb7, 0xe0, 0x06, 0x23, 0xd4, 0x3a, 0xcd, 0xfe, 0x53, + 0x4d, 0xe5, 0xac, 0x44, 0x65, 0x3d, 0x1e, 0x26, 0x64, 0x80, 0xec, 0x76, 0x5a, 0xc1, 0x84, 0x5b, + 0x6b, 0x36, 0x65, 0x81, 0x06, 0x8c, 0xd6, 0xdd, 0x95, 0x13, 0xf5, 0x3c, 0x80, 0x11, 0x7a, 0xa8, + 0xf2, 0xd7, 0x6b, 0x50, 0xa2, 0x2d, 0x69, 0xa1, 0xda, 0x5f, 0xa6, 0xb5, 0x3f, 0xe8, 0x2f, 0xf2, + 0x54, 0xed, 0x4f, 0xb0, 0xb2, 0x7f, 0x1f, 0xa4, 0x91, 0xee, 0x62, 0xdb, 0x27, 0xce, 0x4c, 0x4d, + 0x8d, 0x15, 0xe2, 0x2e, 0x65, 0x84, 0xe2, 0x62, 0x20, 0xa8, 0x12, 0xa5, 0xec, 0x09, 0x76, 0xe9, + 0x1e, 0x10, 0xf8, 0xff, 0x16, 0x1b, 0x42, 0x97, 0x23, 0xab, 0x9e, 0x04, 0x02, 0x1a, 0x97, 0x44, + 0x1f, 0x00, 0x8c, 0x47, 0x7d, 0xae, 0x17, 0x1f, 0x91, 0xa4, 0xf1, 0x88, 0x49, 0xa3, 0x5d, 0x58, + 0x1e, 0x3a, 0x86, 0x79, 0x68, 0x0e, 0x82, 0x1b, 0x26, 0xab, 0x94, 0x92, 0xa5, 0xf1, 0xfc, 0xff, + 0xb1, 0xfb, 0x21, 0xab, 0x56, 0xf5, 0xd8, 0x1a, 0x54, 0x7b, 0x7c, 0xd5, 0x62, 0x50, 0x72, 0x5c, + 0x9b, 0x30, 0xd1, 0x03, 0xc8, 0x06, 0xf9, 0x11, 0x94, 0xf6, 0xc5, 0x33, 0x8b, 0x21, 0x72, 0x6d, + 0xb4, 0x05, 0x45, 0x1b, 0x9f, 0xc6, 0x27, 0x1e, 0x69, 0x2a, 0xf6, 0xf2, 0x1d, 0x7c, 0x7a, 0xf9, + 0xb8, 0x93, 0xb7, 0x23, 0x8e, 0x81, 0x1e, 0x43, 0x61, 0xe4, 0x9a, 0x43, 0xdd, 0x9d, 0xf4, 0x83, + 0x74, 0x85, 0xab, 0xa4, 0x2b, 0xaf, 0xdf, 0x0c, 0x82, 0x72, 0xd1, 0x16, 0x04, 0x03, 0x06, 0xf6, + 0x94, 0x1c, 0x3d, 0xe3, 0xd5, 0xc0, 0xb8, 0x32, 0xaa, 0x43, 0x81, 0x1e, 0x31, 0x9c, 0x6c, 0xf2, + 0xf4, 0x84, 0xab, 0xec, 0x84, 0x39, 0x72, 0xc2, 0x4b, 0xa6, 0x9b, 0x9c, 0x1d, 0xd2, 0x0d, 0xb4, + 0x0d, 0x10, 0x2e, 0xb5, 0xa4, 0x2a, 0xbf, 0xa9, 0xe9, 0xed, 0x72, 0xc1, 0xc8, 0x24, 0x2d, 0xa6, + 0x8d, 0x76, 0x40, 0xe2, 0x69, 0x1b, 0x94, 0xe3, 0xdc, 0xdc, 0x55, 0x65, 0xb6, 0x88, 0xf0, 0xe0, + 0x0a, 0x11, 0x50, 0x07, 0xd2, 0x16, 0xd6, 0x3d, 0xac, 0x94, 0xa8, 0x55, 0x5f, 0xcc, 0x81, 0xba, + 0x90, 0x5e, 0xd5, 0xbd, 0xc1, 0x31, 0x1e, 0xea, 0x8d, 0x63, 0xdd, 0x3e, 0xc2, 0x6d, 0xa2, 0xaf, + 0x05, 0x30, 0xa8, 0x03, 0x32, 0x75, 0x57, 0xbc, 0x1e, 0xc9, 0xd4, 0x63, 0x1f, 0x32, 0x8f, 0x15, + 0x89, 0xc7, 0xe6, 0xd6, 0x24, 0x1a, 0x4f, 0x3b, 0x51, 0x5d, 0xfa, 0x15, 0x14, 0x0f, 0x1d, 0x77, + 0xa8, 0xfb, 0x61, 0x96, 0x2c, 0x47, 0x53, 0xdb, 0xeb, 0xb3, 0xb5, 0xc2, 0x16, 0xe5, 0xf2, 0xcc, + 0x2a, 0x1c, 0xc6, 0x1f, 0xd1, 0x43, 0x5e, 0xbe, 0xaf, 0xd1, 0x6a, 0xfb, 0xc9, 0xa2, 0xa7, 0x9b, + 0xad, 0xdd, 0x1d, 0xc8, 0x0c, 0x8e, 0xf1, 0xe0, 0x85, 0xa7, 0x5c, 0xa7, 0x3e, 0xff, 0xf9, 0x82, + 0x50, 0x0d, 0xa2, 0x14, 0xed, 0x8c, 0x1a, 0x43, 0x41, 0x8f, 0x21, 0xeb, 0xe2, 0x60, 0x1c, 0xbc, + 0x41, 0x01, 0x3f, 0x5b, 0x10, 0x50, 0xa3, 0x5a, 0xaa, 0x7d, 0xe8, 0xf0, 0x48, 0x65, 0x38, 0xa8, + 0x0b, 0xe2, 0x21, 0x59, 0x77, 0x4c, 0xec, 0x29, 0x37, 0x29, 0xe6, 0xa7, 0x6f, 0x4c, 0xeb, 0x8b, + 0x1b, 0x16, 0xdf, 0xef, 0x38, 0x48, 0x98, 0xdd, 0x94, 0x30, 0x21, 0x37, 0xf9, 0xde, 0x6c, 0x76, + 0xf3, 0x0d, 0x6b, 0x6a, 0xdb, 0xa2, 0xd9, 0xcd, 0x9e, 0x0c, 0x52, 0xe5, 0x4e, 0x4c, 0xfc, 0x4d, + 0xff, 0xe5, 0x18, 0xbb, 0x13, 0x45, 0x89, 0x55, 0x64, 0x89, 0xd0, 0x1f, 0x13, 0x32, 0xfa, 0x0c, + 0x24, 0x03, 0x8f, 0xb0, 0x6d, 0x78, 0x5d, 0x5b, 0xb9, 0x45, 0xc7, 0x8c, 0x6b, 0x64, 0xf6, 0x6d, + 0x72, 0x22, 0xab, 0xb8, 0x91, 0x14, 0x7a, 0x4e, 0xf6, 0x4f, 0xf2, 0x80, 0x8d, 0xae, 0x5d, 0x9f, + 0x28, 0x2b, 0xf4, 0xd0, 0xf7, 0x16, 0x76, 0x24, 0x9f, 0xa9, 0xae, 0xf3, 0xf3, 0x34, 0x63, 0x68, + 0xda, 0x14, 0x36, 0xfa, 0x2d, 0xe4, 0x79, 0x48, 0x6f, 0x3b, 0x07, 0x9e, 0xf2, 0x7f, 0xf4, 0x5d, + 0x9b, 0x0b, 0xbe, 0x6b, 0x27, 0x52, 0xe5, 0xc5, 0x2a, 0x8e, 0xb6, 0xf2, 0xbd, 0x00, 0xcb, 0x33, + 0x29, 0x85, 0x7e, 0x0f, 0x59, 0xdb, 0x31, 0x62, 0x0b, 0x58, 0x8b, 0x19, 0x9a, 0xe9, 0x38, 0x46, + 0xb0, 0x7f, 0xdd, 0x3f, 0x32, 0xfd, 0xe3, 0xf1, 0x41, 0x75, 0xe0, 0x0c, 0x37, 0x42, 0x53, 0x8c, + 0x83, 0x8d, 0x4b, 0xbf, 0xd3, 0x55, 0x03, 0x35, 0x2d, 0x43, 0x50, 0x55, 0x03, 0x7d, 0x0a, 0x25, + 0x7c, 0x3a, 0x32, 0xdd, 0x58, 0x5b, 0x21, 0x63, 0x52, 0x92, 0x99, 0x58, 0x8c, 0x98, 0xa4, 0x6b, + 0xac, 0xfc, 0x49, 0x80, 0xd2, 0x85, 0x70, 0x26, 0x6d, 0x96, 0xae, 0xfe, 0x53, 0x6d, 0x96, 0x50, + 0xc2, 0x06, 0x9c, 0x78, 0xe3, 0xb7, 0x96, 0xe4, 0x3b, 0x7e, 0x6b, 0x59, 0xb1, 0x00, 0xa2, 0x8c, + 0x40, 0xbf, 0x86, 0x82, 0x63, 0x19, 0xfd, 0xa8, 0x7b, 0x0b, 0x51, 0x2b, 0x26, 0xc5, 0xba, 0x6b, + 0x19, 0x17, 0x1a, 0x78, 0xce, 0x09, 0x49, 0x06, 0x5a, 0x03, 0x91, 0xa8, 0xcf, 0xd8, 0x9d, 0x75, + 0x2c, 0x83, 0xac, 0x68, 0x2b, 0xdf, 0x09, 0x20, 0xc5, 0xbf, 0x4b, 0x25, 0xc2, 0x57, 0x5c, 0x3e, + 0x49, 0xbc, 0xe5, 0x46, 0x3c, 0xbd, 0x08, 0x26, 0x17, 0x5f, 0x04, 0x57, 0x4e, 0x20, 0x17, 0x8b, + 0xb5, 0x8b, 0x83, 0xa1, 0xf0, 0x16, 0x83, 0xe1, 0x87, 0x90, 0x79, 0xee, 0x1c, 0xf0, 0x03, 0x24, + 0xeb, 0x05, 0xa6, 0x9d, 0xde, 0x76, 0x0e, 0xd4, 0xa6, 0x96, 0x7e, 0xee, 0x1c, 0xa8, 0x46, 0xe5, + 0x0e, 0x1f, 0xfe, 0x00, 0x32, 0xbb, 0xfb, 0xf5, 0xb6, 0xda, 0xb8, 0x74, 0x70, 0xfb, 0x45, 0xea, + 0xdb, 0x3f, 0xaf, 0x09, 0xdb, 0x29, 0x11, 0xc9, 0xd7, 0x2a, 0xdf, 0x0b, 0x80, 0x9a, 0xba, 0xaf, + 0x93, 0xab, 0xbe, 0xc2, 0xe4, 0x96, 0x78, 0x83, 0xbf, 0xa7, 0x1b, 0x6d, 0xf2, 0x5d, 0x1a, 0x6d, + 0x60, 0x6a, 0xe5, 0x3b, 0x01, 0x20, 0x66, 0xdc, 0x97, 0xf1, 0x4f, 0x91, 0xf3, 0x67, 0x8a, 0x0b, + 0xf9, 0x4f, 0xf6, 0x89, 0xe0, 0x43, 0xe5, 0x03, 0x10, 0x0d, 0x76, 0x64, 0xb6, 0x92, 0xcc, 0x6d, + 0xde, 0x33, 0x9e, 0x79, 0xb8, 0xa4, 0x85, 0xca, 0xf5, 0x2c, 0xa4, 0xc7, 0xb6, 0xe9, 0xd8, 0x77, + 0x3f, 0x07, 0x34, 0x9b, 0x26, 0xa8, 0x00, 0x12, 0xfd, 0xaf, 0xfb, 0xd8, 0x08, 0xa6, 0xf0, 0x7d, + 0xfb, 0x24, 0x24, 0x08, 0xf5, 0xdb, 0xaf, 0xfe, 0xbd, 0xba, 0xf4, 0xea, 0x7c, 0x55, 0xf8, 0xe1, + 0x7c, 0x55, 0xf8, 0xe7, 0xf9, 0xaa, 0xf0, 0xaf, 0xf3, 0x55, 0xe1, 0x8f, 0x3f, 0xae, 0x2e, 0x7d, + 0x9d, 0x65, 0x06, 0xfc, 0x37, 0x00, 0x00, 0xff, 0xff, 0x17, 0xf5, 0xd9, 0x38, 0x2f, 0x18, 0x00, + 0x00, } diff --git a/pkg/sql/sqlbase/structured.proto b/pkg/sql/sqlbase/structured.proto index b08bc0fe52fc..e2637dbc510a 100644 --- a/pkg/sql/sqlbase/structured.proto +++ b/pkg/sql/sqlbase/structured.proto @@ -27,7 +27,7 @@ import "gogoproto/gogo.proto"; message ColumnType { // These mirror the types supported by the sql/parser. See // sql/parser/col_types.go. - enum Kind { + enum SemanticType { BOOL = 0; INT = 1; // INT(width) FLOAT = 2; // FLOAT(precision) @@ -65,7 +65,7 @@ message ColumnType { INT2VECTOR = 200; } - optional Kind kind = 1 [(gogoproto.nullable) = false]; + optional SemanticType semantic_type = 1 [(gogoproto.nullable) = false]; // BIT, INT, FLOAT, DECIMAL, CHAR and BINARY optional int32 width = 2 [(gogoproto.nullable) = false]; // FLOAT and DECIMAL. diff --git a/pkg/sql/sqlbase/structured_test.go b/pkg/sql/sqlbase/structured_test.go index 552b4522dbc8..79585710c97a 100644 --- a/pkg/sql/sqlbase/structured_test.go +++ b/pkg/sql/sqlbase/structured_test.go @@ -840,19 +840,19 @@ func TestColumnTypeSQLString(t *testing.T) { colType ColumnType expectedSQL string }{ - {ColumnType{Kind: ColumnType_INT}, "INT"}, - {ColumnType{Kind: ColumnType_INT, Width: 2}, "BIT(2)"}, - {ColumnType{Kind: ColumnType_FLOAT}, "FLOAT"}, - {ColumnType{Kind: ColumnType_FLOAT, Precision: 3}, "FLOAT(3)"}, - {ColumnType{Kind: ColumnType_DECIMAL}, "DECIMAL"}, - {ColumnType{Kind: ColumnType_DECIMAL, Precision: 6}, "DECIMAL(6)"}, - {ColumnType{Kind: ColumnType_DECIMAL, Precision: 7, Width: 8}, "DECIMAL(7,8)"}, - {ColumnType{Kind: ColumnType_DATE}, "DATE"}, - {ColumnType{Kind: ColumnType_TIMESTAMP}, "TIMESTAMP"}, - {ColumnType{Kind: ColumnType_INTERVAL}, "INTERVAL"}, - {ColumnType{Kind: ColumnType_STRING}, "STRING"}, - {ColumnType{Kind: ColumnType_STRING, Width: 10}, "STRING(10)"}, - {ColumnType{Kind: ColumnType_BYTES}, "BYTES"}, + {ColumnType{SemanticType: ColumnType_INT}, "INT"}, + {ColumnType{SemanticType: ColumnType_INT, Width: 2}, "BIT(2)"}, + {ColumnType{SemanticType: ColumnType_FLOAT}, "FLOAT"}, + {ColumnType{SemanticType: ColumnType_FLOAT, Precision: 3}, "FLOAT(3)"}, + {ColumnType{SemanticType: ColumnType_DECIMAL}, "DECIMAL"}, + {ColumnType{SemanticType: ColumnType_DECIMAL, Precision: 6}, "DECIMAL(6)"}, + {ColumnType{SemanticType: ColumnType_DECIMAL, Precision: 7, Width: 8}, "DECIMAL(7,8)"}, + {ColumnType{SemanticType: ColumnType_DATE}, "DATE"}, + {ColumnType{SemanticType: ColumnType_TIMESTAMP}, "TIMESTAMP"}, + {ColumnType{SemanticType: ColumnType_INTERVAL}, "INTERVAL"}, + {ColumnType{SemanticType: ColumnType_STRING}, "STRING"}, + {ColumnType{SemanticType: ColumnType_STRING, Width: 10}, "STRING(10)"}, + {ColumnType{SemanticType: ColumnType_BYTES}, "BYTES"}, } for i, d := range testData { sql := d.colType.SQLString() @@ -867,20 +867,20 @@ func TestColumnValueEncodedSize(t *testing.T) { colType ColumnType size int // -1 means unbounded }{ - {ColumnType{Kind: ColumnType_BOOL}, 1}, - {ColumnType{Kind: ColumnType_INT}, 10}, - {ColumnType{Kind: ColumnType_INT, Width: 2}, 10}, - {ColumnType{Kind: ColumnType_FLOAT}, 9}, - {ColumnType{Kind: ColumnType_FLOAT, Precision: 100}, 9}, - {ColumnType{Kind: ColumnType_DECIMAL}, -1}, - {ColumnType{Kind: ColumnType_DECIMAL, Precision: 100}, 69}, - {ColumnType{Kind: ColumnType_DECIMAL, Precision: 100, Width: 100}, 69}, - {ColumnType{Kind: ColumnType_DATE}, 10}, - {ColumnType{Kind: ColumnType_TIMESTAMP}, 10}, - {ColumnType{Kind: ColumnType_INTERVAL}, 28}, - {ColumnType{Kind: ColumnType_STRING}, -1}, - {ColumnType{Kind: ColumnType_STRING, Width: 100}, 110}, - {ColumnType{Kind: ColumnType_BYTES}, -1}, + {ColumnType{SemanticType: ColumnType_BOOL}, 1}, + {ColumnType{SemanticType: ColumnType_INT}, 10}, + {ColumnType{SemanticType: ColumnType_INT, Width: 2}, 10}, + {ColumnType{SemanticType: ColumnType_FLOAT}, 9}, + {ColumnType{SemanticType: ColumnType_FLOAT, Precision: 100}, 9}, + {ColumnType{SemanticType: ColumnType_DECIMAL}, -1}, + {ColumnType{SemanticType: ColumnType_DECIMAL, Precision: 100}, 69}, + {ColumnType{SemanticType: ColumnType_DECIMAL, Precision: 100, Width: 100}, 69}, + {ColumnType{SemanticType: ColumnType_DATE}, 10}, + {ColumnType{SemanticType: ColumnType_TIMESTAMP}, 10}, + {ColumnType{SemanticType: ColumnType_INTERVAL}, 28}, + {ColumnType{SemanticType: ColumnType_STRING}, -1}, + {ColumnType{SemanticType: ColumnType_STRING, Width: 100}, 110}, + {ColumnType{SemanticType: ColumnType_BYTES}, -1}, } for i, test := range tests { testIsBounded := test.size != -1 @@ -904,7 +904,7 @@ func TestColumnValueEncodedSize(t *testing.T) { func TestFitColumnToFamily(t *testing.T) { intEncodedSize, _ := upperBoundColumnValueEncodedSize(ColumnDescriptor{ ID: 8, - Type: ColumnType{Kind: ColumnType_INT}, + Type: ColumnType{SemanticType: ColumnType_INT}, }) makeTestTableDescriptor := func(familyTypes [][]ColumnType) TableDescriptor { @@ -927,15 +927,15 @@ func TestFitColumnToFamily(t *testing.T) { emptyFamily := []ColumnType{} partiallyFullFamily := []ColumnType{ - {Kind: ColumnType_INT}, - {Kind: ColumnType_BYTES, Width: 10}, + {SemanticType: ColumnType_INT}, + {SemanticType: ColumnType_BYTES, Width: 10}, } fullFamily := []ColumnType{ - {Kind: ColumnType_BYTES, Width: FamilyHeuristicTargetBytes + 1}, + {SemanticType: ColumnType_BYTES, Width: FamilyHeuristicTargetBytes + 1}, } maxIntsInOneFamily := make([]ColumnType, FamilyHeuristicTargetBytes/intEncodedSize) for i := range maxIntsInOneFamily { - maxIntsInOneFamily[i] = ColumnType{Kind: ColumnType_INT} + maxIntsInOneFamily[i] = ColumnType{SemanticType: ColumnType_INT} } tests := []struct { @@ -945,27 +945,27 @@ func TestFitColumnToFamily(t *testing.T) { idx int // not applicable if colFits is false }{ // Bounded size column. - {colFits: true, idx: 0, newCol: ColumnType{Kind: ColumnType_BOOL}, + {colFits: true, idx: 0, newCol: ColumnType{SemanticType: ColumnType_BOOL}, existingFamilies: nil, }, - {colFits: true, idx: 0, newCol: ColumnType{Kind: ColumnType_BOOL}, + {colFits: true, idx: 0, newCol: ColumnType{SemanticType: ColumnType_BOOL}, existingFamilies: [][]ColumnType{emptyFamily}, }, - {colFits: true, idx: 0, newCol: ColumnType{Kind: ColumnType_BOOL}, + {colFits: true, idx: 0, newCol: ColumnType{SemanticType: ColumnType_BOOL}, existingFamilies: [][]ColumnType{partiallyFullFamily}, }, - {colFits: true, idx: 0, newCol: ColumnType{Kind: ColumnType_BOOL}, + {colFits: true, idx: 0, newCol: ColumnType{SemanticType: ColumnType_BOOL}, existingFamilies: [][]ColumnType{fullFamily}, }, - {colFits: true, idx: 0, newCol: ColumnType{Kind: ColumnType_BOOL}, + {colFits: true, idx: 0, newCol: ColumnType{SemanticType: ColumnType_BOOL}, existingFamilies: [][]ColumnType{fullFamily, emptyFamily}, }, // Unbounded size column. - {colFits: true, idx: 0, newCol: ColumnType{Kind: ColumnType_DECIMAL}, + {colFits: true, idx: 0, newCol: ColumnType{SemanticType: ColumnType_DECIMAL}, existingFamilies: [][]ColumnType{emptyFamily}, }, - {colFits: true, idx: 0, newCol: ColumnType{Kind: ColumnType_DECIMAL}, + {colFits: true, idx: 0, newCol: ColumnType{SemanticType: ColumnType_DECIMAL}, existingFamilies: [][]ColumnType{partiallyFullFamily}, }, } diff --git a/pkg/sql/sqlbase/system.go b/pkg/sql/sqlbase/system.go index 0f5f3d3f3818..6a314d961426 100644 --- a/pkg/sql/sqlbase/system.go +++ b/pkg/sql/sqlbase/system.go @@ -188,10 +188,10 @@ func SystemDesiredPrivileges(id ID) privilege.List { // Helpers used to make some of the TableDescriptor literals below more concise. var ( - colTypeInt = ColumnType{Kind: ColumnType_INT} - colTypeString = ColumnType{Kind: ColumnType_STRING} - colTypeBytes = ColumnType{Kind: ColumnType_BYTES} - colTypeTimestamp = ColumnType{Kind: ColumnType_TIMESTAMP} + colTypeInt = ColumnType{SemanticType: ColumnType_INT} + colTypeString = ColumnType{SemanticType: ColumnType_STRING} + colTypeBytes = ColumnType{SemanticType: ColumnType_BYTES} + colTypeTimestamp = ColumnType{SemanticType: ColumnType_TIMESTAMP} singleASC = []IndexDescriptor_Direction{IndexDescriptor_ASC} singleID1 = []ColumnID{1} ) @@ -465,7 +465,7 @@ var ( Columns: []ColumnDescriptor{ {Name: "key", ID: 1, Type: colTypeString}, {Name: "value", ID: 2, Type: colTypeBytes, Nullable: true}, - {Name: "lastUpdated", ID: 3, Type: ColumnType{Kind: ColumnType_TIMESTAMP}}, + {Name: "lastUpdated", ID: 3, Type: ColumnType{SemanticType: ColumnType_TIMESTAMP}}, }, NextColumnID: 4, Families: []ColumnFamilyDescriptor{ diff --git a/pkg/sql/sqlbase/table.go b/pkg/sql/sqlbase/table.go index 502156beb355..158b0806a8fa 100644 --- a/pkg/sql/sqlbase/table.go +++ b/pkg/sql/sqlbase/table.go @@ -90,7 +90,7 @@ func MakeColumnDefDescs( } } - // Set Type.Kind and Type.Locale. + // Set Type.SemanticType and Type.Locale. colDatumType := parser.CastTargetToDatumType(d.Type) col.Type = DatumTypeToColumnType(colDatumType) @@ -1396,7 +1396,7 @@ func CheckColumnType(col ColumnDescriptor, typ parser.Type, pmap *parser.Placeho } else if !typ.Equivalent(colTyp) { // Not a placeholder; check that the value cast has succeeded. return fmt.Errorf("value type %s doesn't match type %s of column %q", - typ, col.Type.Kind, col.Name) + typ, col.Type.SemanticType, col.Name) } return nil } @@ -1411,7 +1411,7 @@ func MarshalColumnValue(col ColumnDescriptor, val parser.Datum) (roachpb.Value, return r, nil } - switch col.Type.Kind { + switch col.Type.SemanticType { case ColumnType_BOOL: if v, ok := val.(*parser.DBool); ok { r.SetBool(bool(*v)) @@ -1485,10 +1485,10 @@ func MarshalColumnValue(col ColumnDescriptor, val parser.Datum) (roachpb.Value, return r, nil } default: - return r, errors.Errorf("unsupported column type: %s", col.Type.Kind) + return r, errors.Errorf("unsupported column type: %s", col.Type.SemanticType) } return r, fmt.Errorf("value type %s doesn't match type %s of column %q", - val.ResolvedType(), col.Type.Kind, col.Name) + val.ResolvedType(), col.Type.SemanticType, col.Name) } // UnmarshalColumnValue decodes the value from a key-value pair using the type @@ -1501,7 +1501,7 @@ func UnmarshalColumnValue( return parser.DNull, nil } - switch typ.Kind { + switch typ.SemanticType { case ColumnType_BOOL: v, err := value.GetBool() if err != nil { @@ -1592,7 +1592,7 @@ func UnmarshalColumnValue( } return a.NewDOid(parser.MakeDOid(parser.DInt(v))), nil default: - return nil, errors.Errorf("unsupported column type: %s", typ.Kind) + return nil, errors.Errorf("unsupported column type: %s", typ.SemanticType) } } @@ -1600,7 +1600,7 @@ func UnmarshalColumnValue( // bit string) and scale (for decimals) of the value fits the specified // column type. Used by INSERT and UPDATE. func CheckValueWidth(col ColumnDescriptor, val parser.Datum) error { - switch col.Type.Kind { + switch col.Type.SemanticType { case ColumnType_STRING: if v, ok := parser.AsDString(val); ok { if col.Type.Width > 0 && utf8.RuneCountInString(string(v)) > int(col.Type.Width) { @@ -1831,8 +1831,8 @@ func MakePrimaryIndexKey(desc *TableDescriptor, vals ...interface{}) (roachpb.Ke colID := index.ColumnIDs[i] for _, c := range desc.Columns { if c.ID == colID { - if t := DatumTypeToColumnType(datums[i].ResolvedType()).Kind; t != c.Type.Kind { - return nil, errors.Errorf("column %d of type %s, got value of type %s", i, c.Type.Kind, t) + if t := DatumTypeToColumnType(datums[i].ResolvedType()).SemanticType; t != c.Type.SemanticType { + return nil, errors.Errorf("column %d of type %s, got value of type %s", i, c.Type.SemanticType, t) } break } diff --git a/pkg/sql/sqlbase/table_test.go b/pkg/sql/sqlbase/table_test.go index 9eb7d1acfe65..0f9c4b6b2703 100644 --- a/pkg/sql/sqlbase/table_test.go +++ b/pkg/sql/sqlbase/table_test.go @@ -44,7 +44,7 @@ func makeTableDescForTest(test indexKeyTest) (TableDescriptor, map[ColumnID]int) columns := make([]ColumnDescriptor, len(test.primaryValues)+len(test.secondaryValues)) colMap := make(map[ColumnID]int, len(test.secondaryValues)) for i := range columns { - columns[i] = ColumnDescriptor{ID: ColumnID(i + 1), Type: ColumnType{Kind: ColumnType_INT}} + columns[i] = ColumnDescriptor{ID: ColumnID(i + 1), Type: ColumnType{SemanticType: ColumnType_INT}} colMap[columns[i].ID] = i if i < len(test.primaryValues) { primaryColumnIDs[i] = columns[i].ID @@ -169,7 +169,7 @@ func TestIndexKey(t *testing.T) { valuesLen := randutil.RandIntInRange(rng, len(t.primaryInterleaves)+1, len(t.primaryInterleaves)+10) t.primaryValues = make([]parser.Datum, valuesLen) for j := range t.primaryValues { - t.primaryValues[j] = RandDatum(rng, ColumnType{Kind: ColumnType_INT}, true) + t.primaryValues[j] = RandDatum(rng, ColumnType{SemanticType: ColumnType_INT}, true) } t.secondaryInterleaves = make([]ID, rng.Intn(10)) @@ -179,7 +179,7 @@ func TestIndexKey(t *testing.T) { valuesLen = randutil.RandIntInRange(rng, len(t.secondaryInterleaves)+1, len(t.secondaryInterleaves)+10) t.secondaryValues = make([]parser.Datum, valuesLen) for j := range t.secondaryValues { - t.secondaryValues[j] = RandDatum(rng, ColumnType{Kind: ColumnType_INT}, true) + t.secondaryValues[j] = RandDatum(rng, ColumnType{SemanticType: ColumnType_INT}, true) } tests = append(tests, t) diff --git a/pkg/sql/sqlbase/testutils.go b/pkg/sql/sqlbase/testutils.go index 9faa2e7507b4..e54b755fb7d8 100644 --- a/pkg/sql/sqlbase/testutils.go +++ b/pkg/sql/sqlbase/testutils.go @@ -67,13 +67,13 @@ func GetTableDescriptor(kvDB *client.DB, database string, table string) *TableDe // RandDatum generates a random Datum of the given type. // If null is true, the datum can be DNull. -// Note that if typ.Kind is ColumnType_NULL, the datum will always be DNull, +// Note that if typ.SemanticType is ColumnType_NULL, the datum will always be DNull, // regardless of the null flag. func RandDatum(rng *rand.Rand, typ ColumnType, null bool) parser.Datum { if null && rng.Intn(10) == 0 { return parser.DNull } - switch typ.Kind { + switch typ.SemanticType { case ColumnType_BOOL: return parser.MakeDBool(rng.Intn(2) == 1) case ColumnType_INT: @@ -149,13 +149,13 @@ func RandDatum(rng *rand.Rand, typ ColumnType, null bool) parser.Datum { } var ( - columnKinds []ColumnType_Kind - collationLocales = [...]string{"da", "de", "en"} + columnSemanticTypes []ColumnType_SemanticType + collationLocales = [...]string{"da", "de", "en"} ) func init() { - for k := range ColumnType_Kind_name { - columnKinds = append(columnKinds, ColumnType_Kind(k)) + for k := range ColumnType_SemanticType_name { + columnSemanticTypes = append(columnSemanticTypes, ColumnType_SemanticType(k)) } } @@ -164,10 +164,10 @@ func RandCollationLocale(rng *rand.Rand) *string { return &collationLocales[rng.Intn(len(collationLocales))] } -// RandColumnType returns a random ColumnType_Kind value. +// RandColumnType returns a random ColumnType_SemanticType value. func RandColumnType(rng *rand.Rand) ColumnType { - typ := ColumnType{Kind: columnKinds[rng.Intn(len(columnKinds))]} - if typ.Kind == ColumnType_COLLATEDSTRING { + typ := ColumnType{SemanticType: columnSemanticTypes[rng.Intn(len(columnSemanticTypes))]} + if typ.SemanticType == ColumnType_COLLATEDSTRING { typ.Locale = RandCollationLocale(rng) } return typ diff --git a/pkg/sql/table_test.go b/pkg/sql/table_test.go index b3936e166dbc..93e258ec6c28 100644 --- a/pkg/sql/table_test.go +++ b/pkg/sql/table_test.go @@ -57,72 +57,72 @@ func TestMakeTableDescColumns(t *testing.T) { }{ { "BIT", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT, Width: 1}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT, Width: 1}, true, }, { "BIT(3)", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT, Width: 3}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT, Width: 3}, true, }, { "BOOLEAN", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_BOOL}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BOOL}, true, }, { "INT", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, true, }, { "FLOAT(3)", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_FLOAT, Precision: 3}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_FLOAT, Precision: 3}, true, }, { "DECIMAL(6,5)", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_DECIMAL, Precision: 6, Width: 5}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_DECIMAL, Precision: 6, Width: 5}, true, }, { "DATE", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_DATE}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_DATE}, true, }, { "TIMESTAMP", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_TIMESTAMP}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_TIMESTAMP}, true, }, { "INTERVAL", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_INTERVAL}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INTERVAL}, true, }, { "CHAR", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_STRING}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_STRING}, true, }, { "TEXT", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_STRING}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_STRING}, true, }, { "BLOB", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_BYTES}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BYTES}, true, }, { "INT NOT NULL", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, false, }, { "INT NULL", - sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT}, + sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT}, true, }, }