From 8bfdceab08a7bb57336aeb29575de0dc348cca58 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Tue, 12 Nov 2019 11:15:29 -0500 Subject: [PATCH] sql: add types, datum and encoding for TimeTZ This PR adds the types and framework to support TimeTZ, adding the tree.Datum class to ensure it can be parsed and then also adds the encoding for TimeTZ to be put in the database. This is the minimum amount of work needed before any set of tests can pass - trying to keep the PRs small but in the end did not succeed due to automated tests iterating over all the types. Changes per package (and recommended review order) are as follows: **sql/types: added the new TimeTZ type and type family.** **pkg/util/timetz: introduce new TimeTZ type** Added a custom wrapper around timeofday and offset seconds, which form follows postgres' implementation of `TimeTzADT`. Note that `OffsetSecs` being negative / not what you expect is fully handled by all utility functions from this interface. **pkg/sql/sem/tree: add Datum for DTimeTZ** A wrapper around TimeTZ for the tree expressions. Also removed some magic constants I found whilst form following other things. **pkg/util/timeutil/pgdate: remove usage of testutil.IsError** This causes a circular dependency on the new timetz type, and this was a very straightforward way of going around it. **pkg/util/encoding: added encoding for DTimeTZ** Form following postgres, we store the timeofday and offset into the database as 12 bytes - an 8 and 4 byte varint. To have sort order right, and to form follow the btree encoding for TimeTZ, we use UTC time when storing it for `(En|De)codeTimeTZ[As|Des]cending`. **pkg/sql/pgwire: added pgwire for DTimeTZ** **pkg/sql/sqlbase: added necessary changes to encode TimeTZ data** **ccl/changefeedccl: added encoding for ccl** Note we must use the string type here as we cannot encode two ints nicely with avro, nor is one int enough to not lose data. **sql/parser: added parsing support for TimeTZ** I wanted to omit this in this PR but some of the automated tests iterating over every time needs this. Note there are more bits to implement which I will leave for a later PR. **pkg/cli: add ability to dump TimeTZ** **builtins: added equality support for TimeTZ** **sql/logictest/testdata/logic_test: added basic tests for TimeTZ** More to come, but here are some now since the parser is all hooked up. As this is not the fully featured commit yet, I am omitting the release note until the test suite is more fleshed out. Release note: none --- c-deps/libroach/protos/roachpb/data.pb.cc | 1 + c-deps/libroach/protos/roachpb/data.pb.h | 1 + docs/generated/sql/aggregates.md | 6 + docs/generated/sql/bnf/stmt_block.bnf | 1 + docs/generated/sql/functions.md | 14 + docs/generated/sql/operators.md | 6 + docs/generated/sql/window_functions.md | 18 + pkg/ccl/changefeedccl/avro.go | 10 + pkg/ccl/changefeedccl/avro_test.go | 1 + pkg/cli/dump.go | 2 + pkg/roachpb/data.go | 19 ++ pkg/roachpb/data.pb.go | 318 +++++++++--------- pkg/roachpb/data.proto | 1 + pkg/sql/copy_in_test.go | 17 +- pkg/sql/exec_util.go | 1 + .../logictest/testdata/logic_test/pg_catalog | 10 + pkg/sql/logictest/testdata/logic_test/timetz | 48 +++ pkg/sql/opt/optbuilder/testdata/aggregate | 1 + pkg/sql/parser/parse_test.go | 2 +- pkg/sql/parser/sql.y | 2 +- pkg/sql/pg_catalog.go | 1 + pkg/sql/pgwire/pgwire_test.go | 3 + pkg/sql/pgwire/pgwirebase/encoding.go | 14 +- pkg/sql/pgwire/types.go | 22 ++ pkg/sql/sem/builtins/pg_builtins.go | 1 + pkg/sql/sem/tree/constant.go | 1 + pkg/sql/sem/tree/datum.go | 147 +++++++- pkg/sql/sem/tree/datum_invariants_test.go | 64 ++++ pkg/sql/sem/tree/datum_test.go | 116 +++++++ pkg/sql/sem/tree/eval.go | 21 +- pkg/sql/sem/tree/expr.go | 8 +- pkg/sql/sem/tree/parse_string.go | 2 + pkg/sql/sem/tree/parse_string_test.go | 7 + pkg/sql/sem/tree/testutils.go | 2 + pkg/sql/sem/tree/type_check.go | 4 + pkg/sql/sem/tree/walk.go | 3 + pkg/sql/sqlbase/column_type_encoding.go | 37 ++ pkg/sql/sqlbase/datum_alloc.go | 13 + pkg/sql/sqlbase/table.go | 2 +- pkg/sql/sqlbase/testutils.go | 8 + pkg/sql/types/oid.go | 3 + pkg/sql/types/types.go | 36 +- pkg/sql/types/types.pb.go | 99 +++--- pkg/sql/types/types.proto | 15 +- pkg/sql/types/types_test.go | 8 + pkg/util/encoding/encoding.go | 148 +++++++- pkg/util/encoding/encoding_test.go | 155 +++++++++ pkg/util/encoding/type_string.go | 5 +- pkg/util/timetz/timetz.go | 159 +++++++++ pkg/util/timetz/timetz_test.go | 195 +++++++++++ pkg/util/timeutil/pgdate/pgdate_test.go | 10 +- 51 files changed, 1555 insertions(+), 233 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/timetz create mode 100644 pkg/util/timetz/timetz.go create mode 100644 pkg/util/timetz/timetz_test.go diff --git a/c-deps/libroach/protos/roachpb/data.pb.cc b/c-deps/libroach/protos/roachpb/data.pb.cc index 1ec8abd1d5b3..14ef5efb4f1b 100644 --- a/c-deps/libroach/protos/roachpb/data.pb.cc +++ b/c-deps/libroach/protos/roachpb/data.pb.cc @@ -458,6 +458,7 @@ bool ValueType_IsValid(int value) { case 9: case 10: case 11: + case 12: case 100: return true; default: diff --git a/c-deps/libroach/protos/roachpb/data.pb.h b/c-deps/libroach/protos/roachpb/data.pb.h index 32d981d48518..bf107c298877 100644 --- a/c-deps/libroach/protos/roachpb/data.pb.h +++ b/c-deps/libroach/protos/roachpb/data.pb.h @@ -141,6 +141,7 @@ enum ValueType { DECIMAL = 5, DELIMITED_DECIMAL = 9, DURATION = 6, + TIMETZ = 12, TUPLE = 10, BITARRAY = 11, TIMESERIES = 100, diff --git a/docs/generated/sql/aggregates.md b/docs/generated/sql/aggregates.md index f4d4520d7829..da62bd3d85be 100644 --- a/docs/generated/sql/aggregates.md +++ b/docs/generated/sql/aggregates.md @@ -29,6 +29,8 @@ array_agg(arg1: oid) → oid[]

Aggregates the selected values into an array.

+array_agg(arg1: timetz) → timetz[]

Aggregates the selected values into an array.

+
array_agg(arg1: varbit) → varbit[]

Aggregates the selected values into an array.

avg(arg1: decimal) → decimal

Calculates the average of the selected values.

@@ -89,6 +91,8 @@
max(arg1: oid) → oid

Identifies the maximum selected value.

+max(arg1: timetz) → timetz

Identifies the maximum selected value.

+
max(arg1: varbit) → varbit

Identifies the maximum selected value.

min(arg1: bool) → bool

Identifies the minimum selected value.

@@ -121,6 +125,8 @@
min(arg1: oid) → oid

Identifies the minimum selected value.

+min(arg1: timetz) → timetz

Identifies the minimum selected value.

+
min(arg1: varbit) → varbit

Identifies the minimum selected value.

sqrdiff(arg1: decimal) → decimal

Calculates the sum of squared differences from the mean of the selected values.

diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 82f3422cf390..ccedd21ad240 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -2086,6 +2086,7 @@ character_without_length ::= const_datetime ::= 'DATE' + | 'TIMETZ' | 'TIMESTAMP' opt_timezone | 'TIMESTAMPTZ' diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 7a60dbd3dd8b..ab56abb8142d 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -31,6 +31,8 @@
array_append(array: oid[], elem: oid) → oid[]

Appends elem to array, returning the result.

+array_append(array: timetz[], elem: timetz) → timetz[]

Appends elem to array, returning the result.

+
array_append(array: varbit[], elem: varbit) → varbit[]

Appends elem to array, returning the result.

array_cat(left: bool[], right: bool[]) → bool[]

Appends two arrays.

@@ -61,6 +63,8 @@
array_cat(left: oid[], right: oid[]) → oid[]

Appends two arrays.

+array_cat(left: timetz[], right: timetz[]) → timetz[]

Appends two arrays.

+
array_cat(left: varbit[], right: varbit[]) → varbit[]

Appends two arrays.

array_length(input: anyelement[], array_dimension: int) → int

Calculates the length of input on the provided array_dimension. However, because CockroachDB doesn’t yet support multi-dimensional arrays, the only supported array_dimension is 1.

@@ -95,6 +99,8 @@
array_position(array: oid[], elem: oid) → int

Return the index of the first occurrence of elem in array.

+array_position(array: timetz[], elem: timetz) → int

Return the index of the first occurrence of elem in array.

+
array_position(array: varbit[], elem: varbit) → int

Return the index of the first occurrence of elem in array.

array_positions(array: bool[], elem: bool) → int[]

Returns and array of indexes of all occurrences of elem in array.

@@ -125,6 +131,8 @@
array_positions(array: oid[], elem: oid) → int[]

Returns and array of indexes of all occurrences of elem in array.

+array_positions(array: timetz[], elem: timetz) → int[]

Returns and array of indexes of all occurrences of elem in array.

+
array_positions(array: varbit[], elem: varbit) → int[]

Returns and array of indexes of all occurrences of elem in array.

array_prepend(elem: bool, array: bool[]) → bool[]

Prepends elem to array, returning the result.

@@ -155,6 +163,8 @@
array_prepend(elem: oid, array: oid[]) → oid[]

Prepends elem to array, returning the result.

+array_prepend(elem: timetz, array: timetz[]) → timetz[]

Prepends elem to array, returning the result.

+
array_prepend(elem: varbit, array: varbit[]) → varbit[]

Prepends elem to array, returning the result.

array_remove(array: bool[], elem: bool) → bool[]

Remove from array all elements equal to elem.

@@ -185,6 +195,8 @@
array_remove(array: oid[], elem: oid) → oid[]

Remove from array all elements equal to elem.

+array_remove(array: timetz[], elem: timetz) → timetz[]

Remove from array all elements equal to elem.

+
array_remove(array: varbit[], elem: varbit) → varbit[]

Remove from array all elements equal to elem.

array_replace(array: bool[], toreplace: bool, replacewith: bool) → bool[]

Replace all occurrences of toreplace in array with replacewith.

@@ -215,6 +227,8 @@
array_replace(array: oid[], toreplace: oid, replacewith: oid) → oid[]

Replace all occurrences of toreplace in array with replacewith.

+array_replace(array: timetz[], toreplace: timetz, replacewith: timetz) → timetz[]

Replace all occurrences of toreplace in array with replacewith.

+
array_replace(array: varbit[], toreplace: varbit, replacewith: varbit) → varbit[]

Replace all occurrences of toreplace in array with replacewith.

array_to_string(input: anyelement[], delim: string) → string

Join an array into a string with a delimiter.

diff --git a/docs/generated/sql/operators.md b/docs/generated/sql/operators.md index ed515b9d2146..5f50f81e5cc6 100644 --- a/docs/generated/sql/operators.md +++ b/docs/generated/sql/operators.md @@ -167,6 +167,7 @@ timestamptz < datebool timestamptz < timestampbool timestamptz < timestamptzbool +timetz < timetzbool tuple < tuplebool uuid < uuidbool varbit < varbitbool @@ -207,6 +208,7 @@ timestamptz <= datebool timestamptz <= timestampbool timestamptz <= timestamptzbool +timetz <= timetzbool tuple <= tuplebool uuid <= uuidbool varbit <= varbitbool @@ -259,6 +261,7 @@ timestamptz = timestampbool timestamptz = timestamptzbool timestamptz = timestamptzbool +timetz = timetzbool tuple = tuplebool uuid = uuidbool uuid[] = uuid[]bool @@ -315,6 +318,7 @@ time IN tuplebool timestamp IN tuplebool timestamptz IN tuplebool +timetz IN tuplebool tuple IN tuplebool uuid IN tuplebool varbit IN tuplebool @@ -361,6 +365,7 @@ timestamptz IS NOT DISTINCT FROM timestampbool timestamptz IS NOT DISTINCT FROM timestamptzbool timestamptz IS NOT DISTINCT FROM timestamptzbool +timetz IS NOT DISTINCT FROM timetzbool tuple IS NOT DISTINCT FROM tuplebool unknown IS NOT DISTINCT FROM unknownbool uuid IS NOT DISTINCT FROM uuidbool @@ -436,6 +441,7 @@ timestamptz || timestamptztimestamptz timestamptz || timestamptztimestamptz timestamptz || timestamptztimestamptz +timetz || timetztimetz uuid || uuid[]uuid[] uuid[] || uuiduuid[] uuid[] || uuid[]uuid[] diff --git a/docs/generated/sql/window_functions.md b/docs/generated/sql/window_functions.md index fa983a63f6dd..76c476aebd31 100644 --- a/docs/generated/sql/window_functions.md +++ b/docs/generated/sql/window_functions.md @@ -35,6 +35,8 @@
first_value(val: oid) → oid

Returns val evaluated at the row that is the first row of the window frame.

+first_value(val: timetz) → timetz

Returns val evaluated at the row that is the first row of the window frame.

+
first_value(val: varbit) → varbit

Returns val evaluated at the row that is the first row of the window frame.

lag(val: bool) → bool

Returns val evaluated at the previous row within current row’s partition; if there is no such row, instead returns null.

@@ -127,6 +129,12 @@
lag(val: oid, n: int, default: oid) → oid

Returns val evaluated at the row that is n rows before the current row within its partition; if there is no such, row, instead returns default (which must be of the same type as val). Both n and default are evaluated with respect to the current row.

+lag(val: timetz) → timetz

Returns val evaluated at the previous row within current row’s partition; if there is no such row, instead returns null.

+
+lag(val: timetz, n: int) → timetz

Returns val evaluated at the row that is n rows before the current row within its partition; if there is no such row, instead returns null. n is evaluated with respect to the current row.

+
+lag(val: timetz, n: int, default: timetz) → timetz

Returns val evaluated at the row that is n rows before the current row within its partition; if there is no such, row, instead returns default (which must be of the same type as val). Both n and default are evaluated with respect to the current row.

+
lag(val: varbit) → varbit

Returns val evaluated at the previous row within current row’s partition; if there is no such row, instead returns null.

lag(val: varbit, n: int) → varbit

Returns val evaluated at the row that is n rows before the current row within its partition; if there is no such row, instead returns null. n is evaluated with respect to the current row.

@@ -163,6 +171,8 @@
last_value(val: oid) → oid

Returns val evaluated at the row that is the last row of the window frame.

+last_value(val: timetz) → timetz

Returns val evaluated at the row that is the last row of the window frame.

+
last_value(val: varbit) → varbit

Returns val evaluated at the row that is the last row of the window frame.

lead(val: bool) → bool

Returns val evaluated at the following row within current row’s partition; if there is no such row, instead returns null.

@@ -255,6 +265,12 @@
lead(val: oid, n: int, default: oid) → oid

Returns val evaluated at the row that is n rows after the current row within its partition; if there is no such, row, instead returns default (which must be of the same type as val). Both n and default are evaluated with respect to the current row.

+lead(val: timetz) → timetz

Returns val evaluated at the following row within current row’s partition; if there is no such row, instead returns null.

+
+lead(val: timetz, n: int) → timetz

Returns val evaluated at the row that is n rows after the current row within its partition; if there is no such row, instead returns null. n is evaluated with respect to the current row.

+
+lead(val: timetz, n: int, default: timetz) → timetz

Returns val evaluated at the row that is n rows after the current row within its partition; if there is no such, row, instead returns default (which must be of the same type as val). Both n and default are evaluated with respect to the current row.

+
lead(val: varbit) → varbit

Returns val evaluated at the following row within current row’s partition; if there is no such row, instead returns null.

lead(val: varbit, n: int) → varbit

Returns val evaluated at the row that is n rows after the current row within its partition; if there is no such row, instead returns null. n is evaluated with respect to the current row.

@@ -291,6 +307,8 @@
nth_value(val: oid, n: int) → oid

Returns val evaluated at the row that is the nth row of the window frame (counting from 1); null if no such row.

+nth_value(val: timetz, n: int) → timetz

Returns val evaluated at the row that is the nth row of the window frame (counting from 1); null if no such row.

+
nth_value(val: varbit, n: int) → varbit

Returns val evaluated at the row that is the nth row of the window frame (counting from 1); null if no such row.

ntile(n: int) → int

Calculates an integer ranging from 1 to n, dividing the partition as equally as possible.

diff --git a/pkg/ccl/changefeedccl/avro.go b/pkg/ccl/changefeedccl/avro.go index 79e7bf288ef9..9ea36696f38e 100644 --- a/pkg/ccl/changefeedccl/avro.go +++ b/pkg/ccl/changefeedccl/avro.go @@ -222,6 +222,16 @@ func columnDescToAvroSchema(colDesc *sqlbase.ColumnDescriptor) (*avroSchemaField micros := x.(time.Duration) / time.Microsecond return tree.MakeDTime(timeofday.TimeOfDay(micros)), nil } + case types.TimeTZFamily: + avroType = avroSchemaString + // We cannot encode this as a long, as it does not encode + // timezone correctly. + schema.encodeFn = func(d tree.Datum) (interface{}, error) { + return d.(*tree.DTimeTZ).TimeTZ.String(), nil + } + schema.decodeFn = func(x interface{}) (tree.Datum, error) { + return tree.ParseDTimeTZ(nil, x.(string)) + } case types.TimestampFamily: avroType = avroLogicalType{ SchemaType: avroSchemaLong, diff --git a/pkg/ccl/changefeedccl/avro_test.go b/pkg/ccl/changefeedccl/avro_test.go index b4b2acf44814..09e28521c9cf 100644 --- a/pkg/ccl/changefeedccl/avro_test.go +++ b/pkg/ccl/changefeedccl/avro_test.go @@ -297,6 +297,7 @@ func TestAvroSchema(t *testing.T) { `JSONB`: `["null","string"]`, `STRING`: `["null","string"]`, `TIME`: `["null",{"type":"long","logicalType":"time-micros"}]`, + `TIMETZ`: `["null","string"]`, `TIMESTAMP`: `["null",{"type":"long","logicalType":"timestamp-micros"}]`, `TIMESTAMPTZ`: `["null",{"type":"long","logicalType":"timestamp-micros"}]`, `UUID`: `["null","string"]`, diff --git a/pkg/cli/dump.go b/pkg/cli/dump.go index 7f408d2e7e88..123c5a47e14d 100644 --- a/pkg/cli/dump.go +++ b/pkg/cli/dump.go @@ -676,6 +676,8 @@ func dumpTableData(w io.Writer, conn *sqlConn, clusterTS string, bmd basicMetada case types.TimeFamily: // pq awkwardly represents TIME as a time.Time with date 0000-01-01. d = tree.MakeDTime(timeofday.FromTime(t)) + case types.TimeTZFamily: + d = tree.NewDTimeTZFromTime(t) case types.TimestampFamily: d = tree.MakeDTimestamp(t, time.Nanosecond) case types.TimestampTZFamily: diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index f4f0e74817d8..8fcb20cf4014 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/interval" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/timetz" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/pkg/errors" @@ -439,6 +440,14 @@ func (v *Value) SetTime(t time.Time) { v.setTag(ValueType_TIME) } +// SetTimeTZ encodes the specified time value into the bytes field of the +// receiver, sets the tag and clears the checksum. +func (v *Value) SetTimeTZ(t timetz.TimeTZ) { + v.ensureRawBytes(headerSize + encoding.EncodedTimeTZMaxLen) + v.RawBytes = encoding.EncodeTimeTZAscending(v.RawBytes[:headerSize], t) + v.setTag(ValueType_TIMETZ) +} + // SetDuration encodes the specified duration value into the bytes field of the // receiver, sets the tag and clears the checksum. func (v *Value) SetDuration(t duration.Duration) error { @@ -563,6 +572,16 @@ func (v Value) GetTime() (time.Time, error) { return t, err } +// GetTimeTZ decodes a time value from the bytes field of the receiver. If the +// tag is not TIMETZ an error will be returned. +func (v Value) GetTimeTZ() (timetz.TimeTZ, error) { + if tag := v.GetTag(); tag != ValueType_TIMETZ { + return timetz.TimeTZ{}, fmt.Errorf("value type is not %s: %s", ValueType_TIMETZ, tag) + } + _, t, err := encoding.DecodeTimeTZAscending(v.dataBytes()) + return t, err +} + // GetDuration decodes a duration value from the bytes field of the receiver. If // the tag is not DURATION an error will be returned. func (v Value) GetDuration() (duration.Duration, error) { diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 0c6d42d727d4..c2de628016fe 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -46,6 +46,7 @@ const ( ValueType_DECIMAL ValueType = 5 ValueType_DELIMITED_DECIMAL ValueType = 9 ValueType_DURATION ValueType = 6 + ValueType_TIMETZ ValueType = 12 // TUPLE represents a DTuple, encoded as repeated pairs of varint field number // followed by a value encoded Datum. ValueType_TUPLE ValueType = 10 @@ -64,6 +65,7 @@ var ValueType_name = map[int32]string{ 5: "DECIMAL", 9: "DELIMITED_DECIMAL", 6: "DURATION", + 12: "TIMETZ", 10: "TUPLE", 11: "BITARRAY", 100: "TIMESERIES", @@ -78,6 +80,7 @@ var ValueType_value = map[string]int32{ "DECIMAL": 5, "DELIMITED_DECIMAL": 9, "DURATION": 6, + "TIMETZ": 12, "TUPLE": 10, "BITARRAY": 11, "TIMESERIES": 100, @@ -87,7 +90,7 @@ func (x ValueType) String() string { return proto.EnumName(ValueType_name, int32(x)) } func (ValueType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{0} + return fileDescriptor_data_a728f859083e539d, []int{0} } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -111,7 +114,7 @@ func (x ReplicaChangeType) String() string { return proto.EnumName(ReplicaChangeType_name, int32(x)) } func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{1} + return fileDescriptor_data_a728f859083e539d, []int{1} } // TransactionStatus specifies possible states for a transaction. @@ -163,7 +166,7 @@ func (x TransactionStatus) String() string { return proto.EnumName(TransactionStatus_name, int32(x)) } func (TransactionStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{2} + return fileDescriptor_data_a728f859083e539d, []int{2} } // Span is a key range with an inclusive start Key and an exclusive end Key. @@ -180,7 +183,7 @@ type Span struct { func (m *Span) Reset() { *m = Span{} } func (*Span) ProtoMessage() {} func (*Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{0} + return fileDescriptor_data_a728f859083e539d, []int{0} } func (m *Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -232,7 +235,7 @@ func (m *Value) Reset() { *m = Value{} } func (m *Value) String() string { return proto.CompactTextString(m) } func (*Value) ProtoMessage() {} func (*Value) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{1} + return fileDescriptor_data_a728f859083e539d, []int{1} } func (m *Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -268,7 +271,7 @@ func (m *KeyValue) Reset() { *m = KeyValue{} } func (m *KeyValue) String() string { return proto.CompactTextString(m) } func (*KeyValue) ProtoMessage() {} func (*KeyValue) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{2} + return fileDescriptor_data_a728f859083e539d, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -306,7 +309,7 @@ func (m *StoreIdent) Reset() { *m = StoreIdent{} } func (m *StoreIdent) String() string { return proto.CompactTextString(m) } func (*StoreIdent) ProtoMessage() {} func (*StoreIdent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{3} + return fileDescriptor_data_a728f859083e539d, []int{3} } func (m *StoreIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -346,7 +349,7 @@ func (m *SplitTrigger) Reset() { *m = SplitTrigger{} } func (m *SplitTrigger) String() string { return proto.CompactTextString(m) } func (*SplitTrigger) ProtoMessage() {} func (*SplitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{4} + return fileDescriptor_data_a728f859083e539d, []int{4} } func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -394,7 +397,7 @@ func (m *MergeTrigger) Reset() { *m = MergeTrigger{} } func (m *MergeTrigger) String() string { return proto.CompactTextString(m) } func (*MergeTrigger) ProtoMessage() {} func (*MergeTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{5} + return fileDescriptor_data_a728f859083e539d, []int{5} } func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -464,7 +467,7 @@ type ChangeReplicasTrigger struct { func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } func (*ChangeReplicasTrigger) ProtoMessage() {} func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{6} + return fileDescriptor_data_a728f859083e539d, []int{6} } func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -506,7 +509,7 @@ func (m *ModifiedSpanTrigger) Reset() { *m = ModifiedSpanTrigger{} } func (m *ModifiedSpanTrigger) String() string { return proto.CompactTextString(m) } func (*ModifiedSpanTrigger) ProtoMessage() {} func (*ModifiedSpanTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{7} + return fileDescriptor_data_a728f859083e539d, []int{7} } func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -552,7 +555,7 @@ func (m *StickyBitTrigger) Reset() { *m = StickyBitTrigger{} } func (m *StickyBitTrigger) String() string { return proto.CompactTextString(m) } func (*StickyBitTrigger) ProtoMessage() {} func (*StickyBitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{8} + return fileDescriptor_data_a728f859083e539d, []int{8} } func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -591,7 +594,7 @@ func (m *InternalCommitTrigger) Reset() { *m = InternalCommitTrigger{} } func (m *InternalCommitTrigger) String() string { return proto.CompactTextString(m) } func (*InternalCommitTrigger) ProtoMessage() {} func (*InternalCommitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{9} + return fileDescriptor_data_a728f859083e539d, []int{9} } func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -660,7 +663,7 @@ func (m *ObservedTimestamp) Reset() { *m = ObservedTimestamp{} } func (m *ObservedTimestamp) String() string { return proto.CompactTextString(m) } func (*ObservedTimestamp) ProtoMessage() {} func (*ObservedTimestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{10} + return fileDescriptor_data_a728f859083e539d, []int{10} } func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -888,7 +891,7 @@ type Transaction struct { func (m *Transaction) Reset() { *m = Transaction{} } func (*Transaction) ProtoMessage() {} func (*Transaction) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{11} + return fileDescriptor_data_a728f859083e539d, []int{11} } func (m *Transaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -938,7 +941,7 @@ func (m *TransactionRecord) Reset() { *m = TransactionRecord{} } func (m *TransactionRecord) String() string { return proto.CompactTextString(m) } func (*TransactionRecord) ProtoMessage() {} func (*TransactionRecord) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{12} + return fileDescriptor_data_a728f859083e539d, []int{12} } func (m *TransactionRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -974,7 +977,7 @@ func (m *Intent) Reset() { *m = Intent{} } func (m *Intent) String() string { return proto.CompactTextString(m) } func (*Intent) ProtoMessage() {} func (*Intent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{13} + return fileDescriptor_data_a728f859083e539d, []int{13} } func (m *Intent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1011,7 +1014,7 @@ func (m *SequencedWrite) Reset() { *m = SequencedWrite{} } func (m *SequencedWrite) String() string { return proto.CompactTextString(m) } func (*SequencedWrite) ProtoMessage() {} func (*SequencedWrite) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{14} + return fileDescriptor_data_a728f859083e539d, []int{14} } func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1075,7 +1078,7 @@ type Lease struct { func (m *Lease) Reset() { *m = Lease{} } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{15} + return fileDescriptor_data_a728f859083e539d, []int{15} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1120,7 +1123,7 @@ func (m *AbortSpanEntry) Reset() { *m = AbortSpanEntry{} } func (m *AbortSpanEntry) String() string { return proto.CompactTextString(m) } func (*AbortSpanEntry) ProtoMessage() {} func (*AbortSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{16} + return fileDescriptor_data_a728f859083e539d, []int{16} } func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1183,7 +1186,7 @@ func (m *TxnCoordMeta) Reset() { *m = TxnCoordMeta{} } func (m *TxnCoordMeta) String() string { return proto.CompactTextString(m) } func (*TxnCoordMeta) ProtoMessage() {} func (*TxnCoordMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_data_1d98507eebc9a9f8, []int{17} + return fileDescriptor_data_a728f859083e539d, []int{17} } func (m *TxnCoordMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6299,140 +6302,141 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_1d98507eebc9a9f8) } - -var fileDescriptor_data_1d98507eebc9a9f8 = []byte{ - // 2110 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0xcd, 0x6f, 0xdb, 0xc8, - 0x15, 0x37, 0x4d, 0x4a, 0xa2, 0x9e, 0x3e, 0x4c, 0x4f, 0xe2, 0x44, 0x49, 0x50, 0x29, 0x55, 0x8a, - 0x36, 0x0d, 0x76, 0x65, 0x34, 0x69, 0x17, 0x68, 0xd0, 0x16, 0xd5, 0x57, 0x12, 0x29, 0x96, 0x9d, - 0x1d, 0xd1, 0x09, 0xb2, 0xdb, 0x80, 0xa5, 0xc8, 0xb1, 0xcc, 0x46, 0x22, 0xb5, 0x24, 0xe5, 0x58, - 0xfd, 0x0b, 0xf6, 0xd6, 0x3d, 0x16, 0x3d, 0x05, 0xe8, 0xad, 0x97, 0x9e, 0x8a, 0xf6, 0xd4, 0x6b, - 0x73, 0x2a, 0x72, 0xeb, 0xa2, 0x07, 0xa1, 0x55, 0x2e, 0x3d, 0xf7, 0x18, 0xa0, 0x40, 0x31, 0xc3, - 0x21, 0x45, 0xc7, 0x8a, 0x21, 0xc1, 0xbb, 0xc0, 0x5e, 0x24, 0xf2, 0xcd, 0x7b, 0xbf, 0x37, 0xf3, - 0xbe, 0x87, 0x80, 0x5c, 0x47, 0x37, 0x0e, 0x47, 0xbd, 0x6d, 0x53, 0xf7, 0xf5, 0xca, 0xc8, 0x75, - 0x7c, 0x07, 0x6d, 0x1a, 0x8e, 0xf1, 0x9c, 0xd1, 0x2b, 0x7c, 0xf5, 0xea, 0xa5, 0x90, 0x6d, 0x48, - 0x7c, 0x7d, 0xce, 0x7a, 0xb5, 0xec, 0xf9, 0x8e, 0xab, 0xf7, 0xc9, 0x36, 0xb1, 0xfb, 0x96, 0x1d, - 0xfe, 0x51, 0xbe, 0x23, 0xc3, 0xe0, 0x3c, 0x37, 0xce, 0xe2, 0xb9, 0xc3, 0x99, 0x0a, 0x63, 0xdf, - 0x1a, 0x6c, 0x1f, 0x0e, 0x8c, 0x6d, 0xdf, 0x1a, 0x12, 0xcf, 0xd7, 0x87, 0x23, 0xbe, 0x72, 0xb1, - 0xef, 0xf4, 0x1d, 0xf6, 0xb8, 0x4d, 0x9f, 0x02, 0x6a, 0xf9, 0x19, 0x48, 0xdd, 0x91, 0x6e, 0xa3, - 0x2b, 0x20, 0x3e, 0x27, 0x93, 0x82, 0x78, 0x5d, 0xb8, 0x99, 0xad, 0xa5, 0xde, 0x4e, 0x4b, 0xe2, - 0x43, 0x32, 0xc1, 0x94, 0x86, 0xae, 0x43, 0x8a, 0xd8, 0xa6, 0x46, 0x97, 0xa5, 0x93, 0xcb, 0x49, - 0x62, 0x9b, 0x0f, 0xc9, 0xe4, 0x6e, 0xf6, 0xb7, 0x2f, 0x4b, 0x6b, 0x7f, 0x79, 0x59, 0x12, 0xfe, - 0xf3, 0xb2, 0x24, 0xb4, 0x25, 0x59, 0x50, 0xd6, 0xdb, 0x92, 0xbc, 0xae, 0x88, 0xe5, 0x21, 0x24, - 0x1e, 0xeb, 0x83, 0x31, 0x41, 0xd7, 0x20, 0xed, 0xea, 0x2f, 0xb4, 0xde, 0xc4, 0x27, 0x5e, 0x41, - 0xa0, 0x30, 0x58, 0x76, 0xf5, 0x17, 0x35, 0xfa, 0x8e, 0xaa, 0x90, 0x8e, 0x76, 0x5b, 0x58, 0xbf, - 0x2e, 0xdc, 0xcc, 0xdc, 0xfe, 0x56, 0x65, 0x6e, 0x3c, 0x7a, 0xa4, 0xca, 0xe1, 0xc0, 0xa8, 0xa8, - 0x21, 0x53, 0x4d, 0x7a, 0x35, 0x2d, 0xad, 0xe1, 0xb9, 0xd4, 0x5d, 0x89, 0xaa, 0x2e, 0x7f, 0x0a, - 0xf2, 0x43, 0x32, 0x09, 0x34, 0xf2, 0x13, 0x09, 0x0b, 0x4e, 0xf4, 0x43, 0x48, 0x1c, 0x51, 0x1e, - 0xae, 0xab, 0x50, 0x39, 0xe5, 0xa8, 0x0a, 0xc3, 0xe0, 0x6a, 0x02, 0xe6, 0xf2, 0x3f, 0x04, 0x80, - 0xae, 0xef, 0xb8, 0xa4, 0x65, 0x12, 0xdb, 0x47, 0x7d, 0x00, 0x63, 0x30, 0xf6, 0x7c, 0xe2, 0x6a, - 0x96, 0xc9, 0xd5, 0x3c, 0xa0, 0xfc, 0xff, 0x9c, 0x96, 0xee, 0xf4, 0x2d, 0xff, 0x70, 0xdc, 0xab, - 0x18, 0xce, 0x70, 0x3b, 0xc2, 0x36, 0x7b, 0xf3, 0xe7, 0xed, 0xd1, 0xf3, 0xfe, 0x36, 0x73, 0xd5, - 0x78, 0x6c, 0x99, 0x95, 0xfd, 0xfd, 0x56, 0x63, 0x36, 0x2d, 0xa5, 0xeb, 0x01, 0x60, 0xab, 0x81, - 0xd3, 0x1c, 0xbb, 0x65, 0xa2, 0x0f, 0x21, 0x65, 0x3b, 0x26, 0xa1, 0x5a, 0xe8, 0x7e, 0x13, 0xb5, - 0x8b, 0xb3, 0x69, 0x29, 0xb9, 0xeb, 0x98, 0xa4, 0xd5, 0x78, 0x1b, 0x3d, 0xe1, 0x24, 0x65, 0x6a, - 0x99, 0xe8, 0x07, 0x20, 0xd3, 0x40, 0x61, 0xfc, 0x22, 0xe3, 0xbf, 0x34, 0x9b, 0x96, 0x52, 0xc1, - 0xce, 0xa9, 0x40, 0xf8, 0x88, 0x53, 0x5e, 0x70, 0x9a, 0xf2, 0x1f, 0x04, 0xc8, 0x76, 0x47, 0x03, - 0xcb, 0x57, 0x5d, 0xab, 0xdf, 0x27, 0x2e, 0x6a, 0x42, 0x7a, 0x40, 0x0e, 0x7c, 0xcd, 0x24, 0x9e, - 0xc1, 0x8e, 0x96, 0xb9, 0x5d, 0x5e, 0x60, 0x24, 0xac, 0xdb, 0x7d, 0xd2, 0x20, 0x9e, 0xe1, 0x5a, - 0x23, 0xdf, 0x71, 0xb9, 0xb9, 0x64, 0x2a, 0x4a, 0xa9, 0xe8, 0x3e, 0x80, 0x6b, 0xf5, 0x0f, 0x39, - 0xce, 0xfa, 0x8a, 0x38, 0x69, 0x26, 0x4b, 0xc9, 0x81, 0x77, 0xdb, 0x92, 0x2c, 0x2a, 0x52, 0x79, - 0xb6, 0x0e, 0xd9, 0x0e, 0x71, 0xfb, 0xe4, 0x1b, 0xba, 0x59, 0x64, 0x83, 0x12, 0x00, 0xd1, 0xbc, - 0xd4, 0x3c, 0x5f, 0xf7, 0x3d, 0x96, 0x38, 0x99, 0xdb, 0x1f, 0xc4, 0xe0, 0x78, 0x32, 0x57, 0x82, - 0x2c, 0xae, 0x84, 0xc9, 0x5c, 0xe9, 0x3c, 0xae, 0xd7, 0xbb, 0x54, 0xa6, 0x76, 0x89, 0x02, 0xcf, - 0xa6, 0xa5, 0x3c, 0xa6, 0x68, 0x11, 0x1d, 0xe7, 0x19, 0x7a, 0xe7, 0xc8, 0x30, 0xd8, 0x3b, 0xba, - 0x07, 0xd9, 0x03, 0x97, 0x90, 0x5f, 0x13, 0xaa, 0xcb, 0xf5, 0x0b, 0x89, 0xe5, 0x13, 0x28, 0x13, - 0x08, 0x76, 0xa9, 0xdc, 0x09, 0x23, 0xff, 0x31, 0x01, 0x5b, 0xf5, 0x43, 0x7a, 0x52, 0x4c, 0x46, - 0x03, 0xcb, 0xd0, 0xbd, 0xd0, 0xda, 0x9f, 0xc0, 0x25, 0x93, 0x8c, 0x5c, 0x62, 0xe8, 0x3e, 0x31, - 0x35, 0x83, 0xf1, 0x68, 0xfe, 0x64, 0x44, 0x98, 0xe9, 0xf3, 0xb7, 0xbf, 0xb3, 0xc8, 0x64, 0x01, - 0x46, 0x00, 0xa8, 0x4e, 0x46, 0x04, 0x5f, 0x9c, 0x63, 0xcc, 0xa9, 0xe8, 0x29, 0xa0, 0x18, 0xb6, - 0x1b, 0x48, 0x71, 0x57, 0x9c, 0x81, 0x7b, 0xca, 0x19, 0x9b, 0x73, 0x14, 0xce, 0x82, 0x7e, 0x05, - 0xd7, 0x62, 0xd0, 0xe3, 0x91, 0x19, 0x57, 0xe1, 0x15, 0xc4, 0xeb, 0xe2, 0x8a, 0x3a, 0xae, 0xcc, - 0xe1, 0xf6, 0x03, 0xb4, 0xd0, 0x52, 0x88, 0xc0, 0xd5, 0x98, 0x2e, 0x9b, 0x1c, 0xfb, 0xa1, 0x22, - 0x9a, 0x93, 0x12, 0xcb, 0xc9, 0x9b, 0xb3, 0x69, 0xe9, 0x72, 0x23, 0xe2, 0xda, 0x25, 0xc7, 0x3e, - 0x97, 0x67, 0x39, 0x9a, 0x8e, 0x5e, 0xf0, 0x65, 0x73, 0x21, 0x97, 0x89, 0x3e, 0x02, 0x89, 0x85, - 0x6a, 0x62, 0xd9, 0x50, 0xc5, 0x8c, 0x1f, 0xf5, 0xe0, 0xb2, 0x65, 0xfb, 0xc4, 0xb5, 0xf5, 0x81, - 0xa6, 0x9b, 0x66, 0xdc, 0x0c, 0xc9, 0x95, 0xcd, 0xb0, 0x15, 0x42, 0x55, 0x29, 0x52, 0x64, 0x82, - 0x03, 0xb8, 0x12, 0xe9, 0x70, 0xc9, 0xd0, 0x39, 0x8a, 0x6b, 0x49, 0xad, 0xac, 0x25, 0xda, 0x30, - 0x0e, 0xb0, 0x42, 0x3d, 0x77, 0x65, 0xda, 0x79, 0x58, 0xe9, 0xff, 0x42, 0x80, 0x0b, 0x1d, 0xc7, - 0xb4, 0x0e, 0x2c, 0x62, 0xd2, 0x8e, 0x16, 0xc6, 0xeb, 0x07, 0x80, 0xbc, 0x89, 0xe7, 0x93, 0xa1, - 0x66, 0x38, 0xf6, 0x81, 0xd5, 0xd7, 0xbc, 0x91, 0x6e, 0xb3, 0x58, 0x95, 0xb1, 0x12, 0xac, 0xd4, - 0xd9, 0x02, 0x6b, 0x83, 0x4d, 0x40, 0xac, 0xd6, 0x0e, 0xac, 0x23, 0x62, 0x13, 0xcf, 0x0b, 0xb8, - 0x83, 0x08, 0xbc, 0xbc, 0x60, 0xc3, 0x54, 0x08, 0x2b, 0x54, 0x64, 0x87, 0x4b, 0x50, 0x0a, 0xef, - 0x46, 0xbf, 0x00, 0xa5, 0xeb, 0x5b, 0xc6, 0xf3, 0x49, 0x6d, 0x5e, 0x59, 0x6b, 0x00, 0x1e, 0xa3, - 0x69, 0x3d, 0xcb, 0xe7, 0xd5, 0x6a, 0xb9, 0x5e, 0xe7, 0x85, 0x50, 0x1c, 0xfd, 0xaf, 0x22, 0x6c, - 0xb5, 0xb8, 0x59, 0xea, 0xce, 0x70, 0x38, 0xd7, 0xd1, 0x80, 0x9c, 0x47, 0xab, 0xb9, 0xe6, 0x07, - 0x04, 0xae, 0xa6, 0xb4, 0x70, 0xff, 0xf3, 0xaa, 0x8f, 0xb3, 0x5e, 0xbc, 0x07, 0x34, 0x20, 0x37, - 0xa4, 0x65, 0x36, 0x42, 0x59, 0x7f, 0x2f, 0x4a, 0xbc, 0x1c, 0xe3, 0xec, 0x30, 0x5e, 0x9c, 0x7f, - 0x09, 0x97, 0x79, 0x8d, 0x08, 0xdd, 0x1f, 0xe1, 0x89, 0x0c, 0xef, 0xe6, 0x02, 0xbc, 0x85, 0x95, - 0x07, 0x6f, 0x19, 0xef, 0x29, 0x48, 0x5b, 0x43, 0xee, 0x77, 0xe6, 0xad, 0x08, 0x3f, 0xa8, 0xb9, - 0xdf, 0x5d, 0xb4, 0xdf, 0xd3, 0x71, 0x82, 0x2f, 0x0c, 0x17, 0x04, 0xcf, 0xc7, 0x80, 0xe6, 0xde, - 0x8a, 0x80, 0x83, 0x84, 0xbb, 0xb1, 0xc8, 0x9c, 0xef, 0xb8, 0x1b, 0x2b, 0xde, 0x3b, 0x94, 0xbb, - 0xf2, 0xe7, 0x7c, 0x4e, 0x2a, 0xff, 0x46, 0x80, 0xcd, 0xbd, 0x9e, 0x47, 0xdc, 0x23, 0x62, 0x46, - 0xde, 0x8e, 0x77, 0x7b, 0x61, 0x89, 0x6e, 0xff, 0x15, 0x8c, 0x4e, 0x72, 0x38, 0xb9, 0x95, 0xff, - 0x94, 0x82, 0x8c, 0xea, 0xea, 0xb6, 0xa7, 0x1b, 0xbe, 0xe5, 0xd8, 0xe8, 0x01, 0x48, 0x74, 0x4e, - 0xe5, 0xf1, 0x73, 0x6b, 0x89, 0xee, 0xa5, 0x1e, 0xdb, 0x1d, 0xe2, 0xeb, 0x35, 0x99, 0x2a, 0x79, - 0x3d, 0x2d, 0x09, 0x98, 0x21, 0x20, 0x04, 0x92, 0xad, 0x0f, 0x83, 0x81, 0x2b, 0x8d, 0xd9, 0x33, - 0xfa, 0x09, 0x24, 0x69, 0x73, 0x1c, 0x07, 0xdd, 0x71, 0x71, 0xe7, 0x88, 0xed, 0xa6, 0xcb, 0x78, - 0x31, 0x97, 0x41, 0x6d, 0xc8, 0x0f, 0x74, 0xcf, 0xd7, 0x0e, 0x89, 0xee, 0xfa, 0x3d, 0xa2, 0xaf, - 0xd4, 0xf7, 0x72, 0x54, 0xf4, 0x41, 0x28, 0x49, 0xb1, 0x1c, 0xd7, 0xea, 0x6b, 0x73, 0x4b, 0x26, - 0x57, 0xc0, 0xa2, 0xa2, 0x73, 0xff, 0x3d, 0x80, 0xdc, 0x50, 0x3f, 0x8e, 0x41, 0xa5, 0x96, 0x87, - 0xca, 0x0e, 0xf5, 0xe3, 0x39, 0xd2, 0xa7, 0x70, 0xc1, 0xe1, 0xe1, 0x31, 0x87, 0xf3, 0x0a, 0xf2, - 0x7b, 0xab, 0xe7, 0xa9, 0x60, 0xe2, 0xb0, 0xc8, 0x79, 0x77, 0xc1, 0x43, 0x3f, 0x87, 0x2c, 0xad, - 0xa9, 0xb6, 0xcf, 0x72, 0xc6, 0x2b, 0x64, 0x18, 0xea, 0xfb, 0x4a, 0x5c, 0x38, 0x2e, 0x04, 0x22, - 0x94, 0xe2, 0xa1, 0x32, 0xe4, 0x5e, 0xb8, 0x96, 0x4f, 0x34, 0xdf, 0x71, 0x34, 0x67, 0x60, 0x16, - 0xb2, 0xac, 0xa6, 0x66, 0x18, 0x51, 0x75, 0x9c, 0xbd, 0x81, 0x89, 0x9e, 0x41, 0x21, 0xd6, 0x09, - 0x87, 0x96, 0x1d, 0xb3, 0x4b, 0x7e, 0x79, 0xbb, 0xc4, 0x26, 0x8e, 0x8e, 0x65, 0xcf, 0x2d, 0xa4, - 0xc2, 0x05, 0x97, 0x1c, 0xb8, 0xc4, 0x3b, 0x8c, 0x9b, 0xa8, 0xb0, 0xb1, 0x3c, 0x32, 0x8a, 0xe4, - 0xe7, 0xa8, 0x3f, 0x85, 0x6b, 0x27, 0xa3, 0x41, 0x7b, 0xa1, 0x7b, 0x5a, 0x68, 0xc3, 0x82, 0xc2, - 0x8e, 0x59, 0x38, 0xe1, 0xf5, 0x27, 0xba, 0x17, 0x1a, 0x1f, 0x7d, 0x0c, 0x8a, 0x65, 0x6b, 0x07, - 0x03, 0x36, 0x02, 0x32, 0x63, 0x78, 0x85, 0x4d, 0x66, 0xdd, 0x6f, 0x2f, 0xb2, 0x2e, 0xf9, 0x6c, - 0x4c, 0x6c, 0x83, 0x98, 0x4f, 0x28, 0x27, 0xdf, 0x55, 0xde, 0xb2, 0xef, 0x31, 0x79, 0x46, 0xf4, - 0x4e, 0xdd, 0xaf, 0x44, 0x45, 0x6a, 0x4b, 0x72, 0x5a, 0x81, 0xb6, 0x24, 0xe7, 0x94, 0x7c, 0xf9, - 0xef, 0x22, 0x6c, 0xc6, 0x32, 0x05, 0x13, 0xc3, 0x71, 0xcd, 0xaf, 0x30, 0x7b, 0xbf, 0x39, 0x99, - 0x7a, 0xfe, 0xb0, 0xfd, 0x1a, 0xdc, 0x23, 0xc7, 0x5c, 0xb3, 0xae, 0x88, 0x91, 0x83, 0x92, 0x4a, - 0xaa, 0x2d, 0xc9, 0x29, 0x45, 0x6e, 0x4b, 0xb2, 0xac, 0xa4, 0x23, 0x97, 0x81, 0x92, 0x69, 0x4b, - 0x72, 0x56, 0xc9, 0x05, 0xee, 0x6b, 0x4b, 0x72, 0x5e, 0xd9, 0x68, 0x4b, 0xf2, 0x86, 0xa2, 0xb4, - 0x25, 0x59, 0x51, 0x36, 0xcb, 0x7f, 0x13, 0x20, 0xd9, 0x62, 0x9b, 0x46, 0x3f, 0x02, 0x29, 0x9a, - 0x58, 0xce, 0x38, 0x69, 0xcc, 0x65, 0x94, 0x1d, 0xd5, 0x40, 0xf4, 0x8f, 0xc3, 0xc9, 0x65, 0x15, - 0xdf, 0x07, 0x47, 0xa4, 0xc2, 0x31, 0xb7, 0x8b, 0xab, 0xbb, 0x9d, 0x4f, 0x29, 0xbf, 0x13, 0x20, - 0x7f, 0xd2, 0x88, 0x67, 0x5d, 0xcc, 0x0d, 0x90, 0x3d, 0xce, 0xcc, 0xef, 0xba, 0xf7, 0xdf, 0x4e, - 0x4b, 0xf5, 0xa5, 0x6e, 0xd3, 0xef, 0xf9, 0x3a, 0x42, 0x0f, 0xd6, 0x25, 0x9f, 0xe1, 0x08, 0x38, - 0xd6, 0xef, 0xfe, 0x27, 0x42, 0x62, 0x87, 0xe8, 0x1e, 0x41, 0x3f, 0x86, 0x44, 0x70, 0x79, 0x5a, - 0x61, 0x22, 0x0b, 0x24, 0xd0, 0x33, 0x00, 0x72, 0x3c, 0xb2, 0x5c, 0x9d, 0xda, 0x60, 0xb9, 0x16, - 0x5c, 0xfc, 0xef, 0xb4, 0x74, 0x35, 0x76, 0x92, 0xbb, 0x65, 0x57, 0xb7, 0x4d, 0x7b, 0x3c, 0x18, - 0xe8, 0xbd, 0x01, 0x29, 0xe3, 0x18, 0x20, 0x6a, 0x40, 0x2a, 0xbc, 0x08, 0x89, 0x2b, 0x5f, 0x84, - 0x42, 0x51, 0x34, 0x86, 0xd8, 0x35, 0x22, 0xb8, 0x27, 0xd2, 0x5f, 0xcf, 0x0a, 0xaf, 0xa6, 0xe7, - 0xdc, 0xf1, 0xd6, 0x1c, 0x9d, 0x5d, 0x26, 0xbb, 0x0c, 0x1b, 0xed, 0x42, 0x66, 0xe4, 0x3a, 0x23, - 0xc7, 0xa3, 0xf5, 0xd9, 0x5b, 0x2e, 0xef, 0xf3, 0xb3, 0x69, 0x09, 0x1e, 0x71, 0x29, 0xb5, 0x8b, - 0x21, 0x44, 0x50, 0x3d, 0x74, 0x11, 0x12, 0x64, 0xe4, 0x18, 0x87, 0xac, 0x3f, 0x8b, 0x38, 0x78, - 0x41, 0x1f, 0xc6, 0xa2, 0x86, 0x76, 0x5b, 0xb1, 0xb6, 0xf9, 0x76, 0x5a, 0xca, 0x31, 0xcf, 0x86, - 0xb1, 0x17, 0xf7, 0x7f, 0x58, 0x4d, 0xcb, 0x33, 0x01, 0xf2, 0xd5, 0x9e, 0xe3, 0xb2, 0xd2, 0xd0, - 0xb4, 0x7d, 0x77, 0x72, 0x56, 0x70, 0x9e, 0x7f, 0xd4, 0x42, 0x87, 0x20, 0x8f, 0x5c, 0xcb, 0x71, - 0x2d, 0x7f, 0xc2, 0xbf, 0xcd, 0xec, 0xbc, 0x9d, 0x96, 0x1e, 0x9c, 0x37, 0xbe, 0x1f, 0x71, 0x4c, - 0x1c, 0xa1, 0xc7, 0x83, 0x7c, 0x1d, 0xb2, 0xea, 0xb1, 0x5d, 0x77, 0x1c, 0xd7, 0xa4, 0x19, 0x8e, - 0x3e, 0x0a, 0x4a, 0x43, 0x10, 0xe9, 0xc5, 0xb3, 0x73, 0x3a, 0x5e, 0x0e, 0x6e, 0x40, 0xce, 0x70, - 0x86, 0x43, 0xdd, 0x36, 0x35, 0xc3, 0x19, 0xdb, 0x7e, 0x70, 0x02, 0x9c, 0xe5, 0xc4, 0x3a, 0xa5, - 0xa1, 0x1a, 0xe4, 0x78, 0x4b, 0xd5, 0x5c, 0xa2, 0x9b, 0x34, 0xbc, 0x96, 0xa8, 0xd0, 0x59, 0x2e, - 0x83, 0xa9, 0x08, 0x6a, 0x40, 0x3e, 0xc4, 0xe0, 0x05, 0x3a, 0xb1, 0x0c, 0x48, 0xa8, 0x38, 0xa8, - 0xca, 0xe8, 0x7b, 0xb0, 0x11, 0xa2, 0x58, 0xf6, 0x91, 0x3e, 0xb0, 0x4c, 0x16, 0x1c, 0x32, 0x0e, - 0xc1, 0x5b, 0x01, 0x75, 0x61, 0x47, 0x90, 0xcf, 0xd5, 0x11, 0xa2, 0x3e, 0x90, 0x54, 0x52, 0xb7, - 0xfe, 0x2c, 0x40, 0x9a, 0x7d, 0x4d, 0x64, 0x9f, 0x38, 0x32, 0x90, 0xda, 0xdf, 0x7d, 0xb8, 0xbb, - 0xf7, 0x64, 0x57, 0x59, 0x43, 0x29, 0x10, 0x5b, 0xbb, 0xaa, 0x22, 0xa0, 0x34, 0x24, 0xee, 0xed, - 0xec, 0x55, 0x55, 0x65, 0x9d, 0x3e, 0xd6, 0x9e, 0xaa, 0xcd, 0xae, 0x22, 0xa2, 0x0b, 0xb0, 0xd1, - 0x68, 0xee, 0xb4, 0x3a, 0x2d, 0xb5, 0xd9, 0xd0, 0x02, 0xa2, 0x8c, 0x64, 0x90, 0xd4, 0x56, 0xa7, - 0xa9, 0x48, 0x14, 0xaa, 0xd1, 0xac, 0xb7, 0x3a, 0xd5, 0x1d, 0x25, 0x81, 0xb6, 0x60, 0x73, 0xce, - 0x1b, 0x92, 0xd3, 0x28, 0x0b, 0x72, 0x63, 0x1f, 0x57, 0xd5, 0xd6, 0xde, 0xae, 0x92, 0xa4, 0xd8, - 0xea, 0xfe, 0xa3, 0x9d, 0xa6, 0x02, 0x74, 0xa1, 0xd6, 0x52, 0xab, 0x18, 0x57, 0x9f, 0x2a, 0x19, - 0x94, 0x07, 0xa0, 0xa0, 0xdd, 0x26, 0x6e, 0x35, 0xbb, 0x8a, 0x59, 0xa6, 0x5d, 0x2b, 0x75, 0xeb, - 0x67, 0xb0, 0x79, 0xea, 0xcb, 0x0d, 0xda, 0x80, 0x4c, 0xb5, 0xd1, 0xd0, 0x70, 0xf3, 0xd1, 0x4e, - 0xab, 0x5e, 0x55, 0xd6, 0x10, 0x82, 0x3c, 0x6e, 0x76, 0xf6, 0x1e, 0x37, 0x23, 0x9a, 0x70, 0x55, - 0xfa, 0xfc, 0xf7, 0xc5, 0xb5, 0x5b, 0x7b, 0x27, 0xa6, 0x92, 0xa0, 0x3d, 0xd0, 0x5d, 0x3f, 0x6a, - 0xee, 0x36, 0x5a, 0xbb, 0xf7, 0x95, 0x35, 0xfa, 0xd2, 0x55, 0xab, 0xf7, 0xe9, 0x8b, 0x88, 0x72, - 0x90, 0xae, 0xef, 0x75, 0x3a, 0x2d, 0x55, 0x6d, 0x36, 0x14, 0x81, 0xae, 0x55, 0x6b, 0x7b, 0x98, - 0xbe, 0xac, 0x07, 0x80, 0xb5, 0xef, 0xbf, 0xfa, 0x77, 0x71, 0xed, 0xd5, 0xac, 0x28, 0xbc, 0x9e, - 0x15, 0x85, 0x2f, 0x67, 0x45, 0xe1, 0x5f, 0xb3, 0xa2, 0xf0, 0xc5, 0x9b, 0xe2, 0xda, 0xeb, 0x37, - 0xc5, 0xb5, 0x2f, 0xdf, 0x14, 0xd7, 0x3e, 0x49, 0x71, 0x37, 0xf5, 0x92, 0xec, 0xf3, 0xf6, 0x9d, - 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0xd3, 0xba, 0x56, 0x1f, 0x98, 0x17, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_a728f859083e539d) } + +var fileDescriptor_data_a728f859083e539d = []byte{ + // 2120 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0xcf, 0x6f, 0xdb, 0xc8, + 0xf5, 0x37, 0x4d, 0x4a, 0xa2, 0x9e, 0x7e, 0x98, 0x9e, 0xc4, 0x89, 0x92, 0xe0, 0x2b, 0xe5, 0xab, + 0x14, 0x6d, 0x1a, 0xec, 0xca, 0x68, 0xd2, 0x2e, 0xd0, 0xa0, 0x2d, 0xaa, 0x5f, 0x49, 0xa4, 0x58, + 0x76, 0x76, 0x44, 0x27, 0x48, 0xb6, 0x01, 0x4b, 0x91, 0x63, 0x99, 0x8d, 0x44, 0x6a, 0x49, 0xca, + 0xb1, 0xfa, 0x17, 0xec, 0xad, 0x7b, 0x2c, 0x7a, 0x0a, 0xd0, 0x5b, 0x2f, 0x3d, 0x15, 0xe8, 0xa9, + 0x97, 0x1e, 0x9a, 0x53, 0x91, 0x5b, 0x17, 0x3d, 0x08, 0xad, 0x72, 0xe9, 0xb9, 0xc7, 0x00, 0x05, + 0x8a, 0x19, 0x0e, 0x29, 0x3a, 0x56, 0x0c, 0x09, 0xde, 0x02, 0x7b, 0x91, 0xc8, 0x37, 0xef, 0x7d, + 0xde, 0xcc, 0xfb, 0x3d, 0x04, 0xe4, 0x3a, 0xba, 0x71, 0x38, 0xea, 0x6d, 0x9b, 0xba, 0xaf, 0x57, + 0x46, 0xae, 0xe3, 0x3b, 0x68, 0xd3, 0x70, 0x8c, 0x17, 0x8c, 0x5e, 0xe1, 0xab, 0x57, 0x2f, 0x85, + 0x6c, 0x43, 0xe2, 0xeb, 0x73, 0xd6, 0xab, 0x65, 0xcf, 0x77, 0x5c, 0xbd, 0x4f, 0xb6, 0x89, 0xdd, + 0xb7, 0xec, 0xf0, 0x8f, 0xf2, 0x1d, 0x19, 0x06, 0xe7, 0xb9, 0x71, 0x16, 0xcf, 0x1d, 0xce, 0x54, + 0x18, 0xfb, 0xd6, 0x60, 0xfb, 0x70, 0x60, 0x6c, 0xfb, 0xd6, 0x90, 0x78, 0xbe, 0x3e, 0x1c, 0xf1, + 0x95, 0x8b, 0x7d, 0xa7, 0xef, 0xb0, 0xc7, 0x6d, 0xfa, 0x14, 0x50, 0xcb, 0xcf, 0x41, 0xea, 0x8e, + 0x74, 0x1b, 0x5d, 0x01, 0xf1, 0x05, 0x99, 0x14, 0xc4, 0xeb, 0xc2, 0xcd, 0x6c, 0x2d, 0xf5, 0x6e, + 0x5a, 0x12, 0x1f, 0x92, 0x09, 0xa6, 0x34, 0x74, 0x1d, 0x52, 0xc4, 0x36, 0x35, 0xba, 0x2c, 0x9d, + 0x5c, 0x4e, 0x12, 0xdb, 0x7c, 0x48, 0x26, 0x77, 0xb3, 0xbf, 0x7e, 0x55, 0x5a, 0xfb, 0xe3, 0xab, + 0x92, 0xf0, 0xaf, 0x57, 0x25, 0xa1, 0x2d, 0xc9, 0x82, 0xb2, 0xde, 0x96, 0xe4, 0x75, 0x45, 0x2c, + 0x0f, 0x21, 0xf1, 0x58, 0x1f, 0x8c, 0x09, 0xba, 0x06, 0x69, 0x57, 0x7f, 0xa9, 0xf5, 0x26, 0x3e, + 0xf1, 0x0a, 0x02, 0x85, 0xc1, 0xb2, 0xab, 0xbf, 0xac, 0xd1, 0x77, 0x54, 0x85, 0x74, 0xb4, 0xdb, + 0xc2, 0xfa, 0x75, 0xe1, 0x66, 0xe6, 0xf6, 0xff, 0x55, 0xe6, 0xc6, 0xa3, 0x47, 0xaa, 0x1c, 0x0e, + 0x8c, 0x8a, 0x1a, 0x32, 0xd5, 0xa4, 0xd7, 0xd3, 0xd2, 0x1a, 0x9e, 0x4b, 0xdd, 0x95, 0xa8, 0xea, + 0xf2, 0x67, 0x20, 0x3f, 0x24, 0x93, 0x40, 0x23, 0x3f, 0x91, 0xb0, 0xe0, 0x44, 0xdf, 0x87, 0xc4, + 0x11, 0xe5, 0xe1, 0xba, 0x0a, 0x95, 0x53, 0x8e, 0xaa, 0x30, 0x0c, 0xae, 0x26, 0x60, 0x2e, 0xff, + 0x4d, 0x00, 0xe8, 0xfa, 0x8e, 0x4b, 0x5a, 0x26, 0xb1, 0x7d, 0xd4, 0x07, 0x30, 0x06, 0x63, 0xcf, + 0x27, 0xae, 0x66, 0x99, 0x5c, 0xcd, 0x03, 0xca, 0xff, 0xf7, 0x69, 0xe9, 0x4e, 0xdf, 0xf2, 0x0f, + 0xc7, 0xbd, 0x8a, 0xe1, 0x0c, 0xb7, 0x23, 0x6c, 0xb3, 0x37, 0x7f, 0xde, 0x1e, 0xbd, 0xe8, 0x6f, + 0x33, 0x57, 0x8d, 0xc7, 0x96, 0x59, 0xd9, 0xdf, 0x6f, 0x35, 0x66, 0xd3, 0x52, 0xba, 0x1e, 0x00, + 0xb6, 0x1a, 0x38, 0xcd, 0xb1, 0x5b, 0x26, 0xfa, 0x18, 0x52, 0xb6, 0x63, 0x12, 0xaa, 0x85, 0xee, + 0x37, 0x51, 0xbb, 0x38, 0x9b, 0x96, 0x92, 0xbb, 0x8e, 0x49, 0x5a, 0x8d, 0x77, 0xd1, 0x13, 0x4e, + 0x52, 0xa6, 0x96, 0x89, 0xbe, 0x07, 0x32, 0x0d, 0x14, 0xc6, 0x2f, 0x32, 0xfe, 0x4b, 0xb3, 0x69, + 0x29, 0x15, 0xec, 0x9c, 0x0a, 0x84, 0x8f, 0x38, 0xe5, 0x05, 0xa7, 0x29, 0xff, 0x4e, 0x80, 0x6c, + 0x77, 0x34, 0xb0, 0x7c, 0xd5, 0xb5, 0xfa, 0x7d, 0xe2, 0xa2, 0x26, 0xa4, 0x07, 0xe4, 0xc0, 0xd7, + 0x4c, 0xe2, 0x19, 0xec, 0x68, 0x99, 0xdb, 0xe5, 0x05, 0x46, 0xc2, 0xba, 0xdd, 0x27, 0x0d, 0xe2, + 0x19, 0xae, 0x35, 0xf2, 0x1d, 0x97, 0x9b, 0x4b, 0xa6, 0xa2, 0x94, 0x8a, 0xee, 0x03, 0xb8, 0x56, + 0xff, 0x90, 0xe3, 0xac, 0xaf, 0x88, 0x93, 0x66, 0xb2, 0x94, 0x1c, 0x78, 0xb7, 0x2d, 0xc9, 0xa2, + 0x22, 0x95, 0x67, 0xeb, 0x90, 0xed, 0x10, 0xb7, 0x4f, 0xbe, 0xa1, 0x9b, 0x45, 0x36, 0x28, 0x01, + 0x10, 0xcd, 0x4b, 0xcd, 0xf3, 0x75, 0xdf, 0x63, 0x89, 0x93, 0xb9, 0xfd, 0x51, 0x0c, 0x8e, 0x27, + 0x73, 0x25, 0xc8, 0xe2, 0x4a, 0x98, 0xcc, 0x95, 0xce, 0xe3, 0x7a, 0xbd, 0x4b, 0x65, 0x6a, 0x97, + 0x28, 0xf0, 0x6c, 0x5a, 0xca, 0x63, 0x8a, 0x16, 0xd1, 0x71, 0x9e, 0xa1, 0x77, 0x8e, 0x0c, 0x83, + 0xbd, 0xa3, 0x7b, 0x90, 0x3d, 0x70, 0x09, 0xf9, 0x25, 0xa1, 0xba, 0x5c, 0xbf, 0x90, 0x58, 0x3e, + 0x81, 0x32, 0x81, 0x60, 0x97, 0xca, 0x9d, 0x30, 0xf2, 0xef, 0x13, 0xb0, 0x55, 0x3f, 0xa4, 0x27, + 0xc5, 0x64, 0x34, 0xb0, 0x0c, 0xdd, 0x0b, 0xad, 0xfd, 0x0c, 0x2e, 0x99, 0x64, 0xe4, 0x12, 0x43, + 0xf7, 0x89, 0xa9, 0x19, 0x8c, 0x47, 0xf3, 0x27, 0x23, 0xc2, 0x4c, 0x9f, 0xbf, 0xfd, 0xad, 0x45, + 0x26, 0x0b, 0x30, 0x02, 0x40, 0x75, 0x32, 0x22, 0xf8, 0xe2, 0x1c, 0x63, 0x4e, 0x45, 0x4f, 0x01, + 0xc5, 0xb0, 0xdd, 0x40, 0x8a, 0xbb, 0xe2, 0x0c, 0xdc, 0x53, 0xce, 0xd8, 0x9c, 0xa3, 0x70, 0x16, + 0xf4, 0x0b, 0xb8, 0x16, 0x83, 0x1e, 0x8f, 0xcc, 0xb8, 0x0a, 0xaf, 0x20, 0x5e, 0x17, 0x57, 0xd4, + 0x71, 0x65, 0x0e, 0xb7, 0x1f, 0xa0, 0x85, 0x96, 0x42, 0x04, 0xae, 0xc6, 0x74, 0xd9, 0xe4, 0xd8, + 0x0f, 0x15, 0xd1, 0x9c, 0x94, 0x58, 0x4e, 0xde, 0x9c, 0x4d, 0x4b, 0x97, 0x1b, 0x11, 0xd7, 0x2e, + 0x39, 0xf6, 0xb9, 0x3c, 0xcb, 0xd1, 0x74, 0xf4, 0x82, 0x2f, 0x9b, 0x0b, 0xb9, 0x4c, 0xf4, 0x09, + 0x48, 0x2c, 0x54, 0x13, 0xcb, 0x86, 0x2a, 0x66, 0xfc, 0xa8, 0x07, 0x97, 0x2d, 0xdb, 0x27, 0xae, + 0xad, 0x0f, 0x34, 0xdd, 0x34, 0xe3, 0x66, 0x48, 0xae, 0x6c, 0x86, 0xad, 0x10, 0xaa, 0x4a, 0x91, + 0x22, 0x13, 0x1c, 0xc0, 0x95, 0x48, 0x87, 0x4b, 0x86, 0xce, 0x51, 0x5c, 0x4b, 0x6a, 0x65, 0x2d, + 0xd1, 0x86, 0x71, 0x80, 0x15, 0xea, 0xb9, 0x2b, 0xd3, 0xce, 0xc3, 0x4a, 0xff, 0x97, 0x02, 0x5c, + 0xe8, 0x38, 0xa6, 0x75, 0x60, 0x11, 0x93, 0x76, 0xb4, 0x30, 0x5e, 0x3f, 0x02, 0xe4, 0x4d, 0x3c, + 0x9f, 0x0c, 0x35, 0xc3, 0xb1, 0x0f, 0xac, 0xbe, 0xe6, 0x8d, 0x74, 0x9b, 0xc5, 0xaa, 0x8c, 0x95, + 0x60, 0xa5, 0xce, 0x16, 0x58, 0x1b, 0x6c, 0x02, 0x62, 0xb5, 0x76, 0x60, 0x1d, 0x11, 0x9b, 0x78, + 0x5e, 0xc0, 0x1d, 0x44, 0xe0, 0xe5, 0x05, 0x1b, 0xa6, 0x42, 0x58, 0xa1, 0x22, 0x3b, 0x5c, 0x82, + 0x52, 0x78, 0x37, 0xfa, 0x19, 0x28, 0x5d, 0xdf, 0x32, 0x5e, 0x4c, 0x6a, 0xf3, 0xca, 0x5a, 0x03, + 0xf0, 0x18, 0x4d, 0xeb, 0x59, 0x3e, 0xaf, 0x56, 0xcb, 0xf5, 0x3a, 0x2f, 0x84, 0xe2, 0xe8, 0x7f, + 0x12, 0x61, 0xab, 0xc5, 0xcd, 0x52, 0x77, 0x86, 0xc3, 0xb9, 0x8e, 0x06, 0xe4, 0x3c, 0x5a, 0xcd, + 0x35, 0x3f, 0x20, 0x70, 0x35, 0xa5, 0x85, 0xfb, 0x9f, 0x57, 0x7d, 0x9c, 0xf5, 0xe2, 0x3d, 0xa0, + 0x01, 0xb9, 0x21, 0x2d, 0xb3, 0x11, 0xca, 0xfa, 0x07, 0x51, 0xe2, 0xe5, 0x18, 0x67, 0x87, 0xf1, + 0xe2, 0xfc, 0x73, 0xb8, 0xcc, 0x6b, 0x44, 0xe8, 0xfe, 0x08, 0x4f, 0x64, 0x78, 0x37, 0x17, 0xe0, + 0x2d, 0xac, 0x3c, 0x78, 0xcb, 0xf8, 0x40, 0x41, 0xda, 0x1a, 0x72, 0xbf, 0x33, 0x6f, 0x45, 0xf8, + 0x41, 0xcd, 0xfd, 0xf6, 0xa2, 0xfd, 0x9e, 0x8e, 0x13, 0x7c, 0x61, 0xb8, 0x20, 0x78, 0x3e, 0x05, + 0x34, 0xf7, 0x56, 0x04, 0x1c, 0x24, 0xdc, 0x8d, 0x45, 0xe6, 0x7c, 0xcf, 0xdd, 0x58, 0xf1, 0xde, + 0xa3, 0xdc, 0x95, 0xbf, 0xe0, 0x73, 0x52, 0xf9, 0x57, 0x02, 0x6c, 0xee, 0xf5, 0x3c, 0xe2, 0x1e, + 0x11, 0x33, 0xf2, 0x76, 0xbc, 0xdb, 0x0b, 0x4b, 0x74, 0xfb, 0xaf, 0x61, 0x74, 0x92, 0xc3, 0xc9, + 0xad, 0xfc, 0x87, 0x14, 0x64, 0x54, 0x57, 0xb7, 0x3d, 0xdd, 0xf0, 0x2d, 0xc7, 0x46, 0x0f, 0x40, + 0xa2, 0x73, 0x2a, 0x8f, 0x9f, 0x5b, 0x4b, 0x74, 0x2f, 0xf5, 0xd8, 0xee, 0x10, 0x5f, 0xaf, 0xc9, + 0x54, 0xc9, 0x9b, 0x69, 0x49, 0xc0, 0x0c, 0x01, 0x21, 0x90, 0x6c, 0x7d, 0x18, 0x0c, 0x5c, 0x69, + 0xcc, 0x9e, 0xd1, 0x8f, 0x20, 0x49, 0x9b, 0xe3, 0x38, 0xe8, 0x8e, 0x8b, 0x3b, 0x47, 0x6c, 0x37, + 0x5d, 0xc6, 0x8b, 0xb9, 0x0c, 0x6a, 0x43, 0x7e, 0xa0, 0x7b, 0xbe, 0x76, 0x48, 0x74, 0xd7, 0xef, + 0x11, 0x7d, 0xa5, 0xbe, 0x97, 0xa3, 0xa2, 0x0f, 0x42, 0x49, 0x8a, 0xe5, 0xb8, 0x56, 0x5f, 0x9b, + 0x5b, 0x32, 0xb9, 0x02, 0x16, 0x15, 0x9d, 0xfb, 0xef, 0x01, 0xe4, 0x86, 0xfa, 0x71, 0x0c, 0x2a, + 0xb5, 0x3c, 0x54, 0x76, 0xa8, 0x1f, 0xcf, 0x91, 0x3e, 0x83, 0x0b, 0x0e, 0x0f, 0x8f, 0x39, 0x9c, + 0x57, 0x90, 0x3f, 0x58, 0x3d, 0x4f, 0x05, 0x13, 0x87, 0x45, 0xce, 0xfb, 0x0b, 0x1e, 0xfa, 0x29, + 0x64, 0x69, 0x4d, 0xb5, 0x7d, 0x96, 0x33, 0x5e, 0x21, 0xc3, 0x50, 0x3f, 0x54, 0xe2, 0xc2, 0x71, + 0x21, 0x10, 0xa1, 0x14, 0x0f, 0x95, 0x21, 0xf7, 0xd2, 0xb5, 0x7c, 0xa2, 0xf9, 0x8e, 0xa3, 0x39, + 0x03, 0xb3, 0x90, 0x65, 0x35, 0x35, 0xc3, 0x88, 0xaa, 0xe3, 0xec, 0x0d, 0x4c, 0xf4, 0x1c, 0x0a, + 0xb1, 0x4e, 0x38, 0xb4, 0xec, 0x98, 0x5d, 0xf2, 0xcb, 0xdb, 0x25, 0x36, 0x71, 0x74, 0x2c, 0x7b, + 0x6e, 0x21, 0x15, 0x2e, 0xb8, 0xe4, 0xc0, 0x25, 0xde, 0x61, 0xdc, 0x44, 0x85, 0x8d, 0xe5, 0x91, + 0x51, 0x24, 0x3f, 0x47, 0xfd, 0x31, 0x5c, 0x3b, 0x19, 0x0d, 0xda, 0x4b, 0xdd, 0xd3, 0x42, 0x1b, + 0x16, 0x14, 0x76, 0xcc, 0xc2, 0x09, 0xaf, 0x3f, 0xd1, 0xbd, 0xd0, 0xf8, 0xe8, 0x53, 0x50, 0x2c, + 0x5b, 0x3b, 0x18, 0xb0, 0x11, 0x90, 0x19, 0xc3, 0x2b, 0x6c, 0x32, 0xeb, 0xfe, 0xff, 0x22, 0xeb, + 0x92, 0xcf, 0xc7, 0xc4, 0x36, 0x88, 0xf9, 0x84, 0x72, 0xf2, 0x5d, 0xe5, 0x2d, 0xfb, 0x1e, 0x93, + 0x67, 0x44, 0xef, 0xd4, 0xfd, 0x4a, 0x54, 0xa4, 0xb6, 0x24, 0xa7, 0x15, 0x68, 0x4b, 0x72, 0x4e, + 0xc9, 0x97, 0xff, 0x2a, 0xc2, 0x66, 0x2c, 0x53, 0x30, 0x31, 0x1c, 0xd7, 0xfc, 0x1a, 0xb3, 0xf7, + 0x9b, 0x93, 0xa9, 0xe7, 0x0f, 0xdb, 0xff, 0x81, 0x7b, 0xe4, 0x98, 0x6b, 0xd6, 0x15, 0x31, 0x72, + 0x50, 0x52, 0x49, 0xb5, 0x25, 0x39, 0xa5, 0xc8, 0x6d, 0x49, 0x96, 0x95, 0x74, 0xe4, 0x32, 0x50, + 0x32, 0x6d, 0x49, 0xce, 0x2a, 0xb9, 0xc0, 0x7d, 0x6d, 0x49, 0xce, 0x2b, 0x1b, 0x6d, 0x49, 0xde, + 0x50, 0x94, 0xb6, 0x24, 0x2b, 0xca, 0x66, 0xf9, 0x2f, 0x02, 0x24, 0x5b, 0x6c, 0xd3, 0xe8, 0x07, + 0x20, 0x45, 0x13, 0xcb, 0x19, 0x27, 0x8d, 0xb9, 0x8c, 0xb2, 0xa3, 0x1a, 0x88, 0xfe, 0x71, 0x38, + 0xb9, 0xac, 0xe2, 0xfb, 0xe0, 0x88, 0x54, 0x38, 0xe6, 0x76, 0x71, 0x75, 0xb7, 0xf3, 0x29, 0xe5, + 0x37, 0x02, 0xe4, 0x4f, 0x1a, 0xf1, 0xac, 0x8b, 0xb9, 0x01, 0xb2, 0xc7, 0x99, 0xf9, 0x5d, 0xf7, + 0xfe, 0xbb, 0x69, 0xa9, 0xbe, 0xd4, 0x6d, 0xfa, 0x03, 0x5f, 0x47, 0xe8, 0xc1, 0xba, 0xe4, 0x73, + 0x1c, 0x01, 0xc7, 0xfa, 0xdd, 0x7f, 0x44, 0x48, 0xec, 0x10, 0xdd, 0x23, 0xe8, 0x87, 0x90, 0x08, + 0x2e, 0x4f, 0x2b, 0x4c, 0x64, 0x81, 0x04, 0x7a, 0x0e, 0x40, 0x8e, 0x47, 0x96, 0xab, 0x53, 0x1b, + 0x2c, 0xd7, 0x82, 0x8b, 0xff, 0x9e, 0x96, 0xae, 0xc6, 0x4e, 0x72, 0xb7, 0xec, 0xea, 0xb6, 0x69, + 0x8f, 0x07, 0x03, 0xbd, 0x37, 0x20, 0x65, 0x1c, 0x03, 0x44, 0x0d, 0x48, 0x85, 0x17, 0x21, 0x71, + 0xe5, 0x8b, 0x50, 0x28, 0x8a, 0xc6, 0x10, 0xbb, 0x46, 0x04, 0xf7, 0x44, 0xfa, 0xeb, 0x59, 0xe1, + 0xd5, 0xf4, 0x9c, 0x3b, 0xde, 0x9a, 0xa3, 0xb3, 0xcb, 0x64, 0x97, 0x61, 0xa3, 0x5d, 0xc8, 0x8c, + 0x5c, 0x67, 0xe4, 0x78, 0xb4, 0x3e, 0x7b, 0xcb, 0xe5, 0x7d, 0x7e, 0x36, 0x2d, 0xc1, 0x23, 0x2e, + 0xa5, 0x76, 0x31, 0x84, 0x08, 0xaa, 0x87, 0x2e, 0x42, 0x82, 0x8c, 0x1c, 0xe3, 0x90, 0xf5, 0x67, + 0x11, 0x07, 0x2f, 0xe8, 0xe3, 0x58, 0xd4, 0xd0, 0x6e, 0x2b, 0xd6, 0x36, 0xdf, 0x4d, 0x4b, 0x39, + 0xe6, 0xd9, 0x30, 0xf6, 0xe2, 0xfe, 0x0f, 0xab, 0x69, 0x79, 0x26, 0x40, 0xbe, 0xda, 0x73, 0x5c, + 0x56, 0x1a, 0x9a, 0xb6, 0xef, 0x4e, 0xce, 0x0a, 0xce, 0xf3, 0x8f, 0x5a, 0xe8, 0x10, 0xe4, 0x91, + 0x6b, 0x39, 0xae, 0xe5, 0x4f, 0xf8, 0xb7, 0x99, 0x9d, 0x77, 0xd3, 0xd2, 0x83, 0xf3, 0xc6, 0xf7, + 0x23, 0x8e, 0x89, 0x23, 0xf4, 0x78, 0x90, 0xaf, 0x43, 0x56, 0x3d, 0xb6, 0xeb, 0x8e, 0xe3, 0x9a, + 0x34, 0xc3, 0xd1, 0x27, 0x41, 0x69, 0x08, 0x22, 0xbd, 0x78, 0x76, 0x4e, 0xc7, 0xcb, 0xc1, 0x0d, + 0xc8, 0x19, 0xce, 0x70, 0xa8, 0xdb, 0xa6, 0x66, 0x38, 0x63, 0xdb, 0x0f, 0x4e, 0x80, 0xb3, 0x9c, + 0x58, 0xa7, 0x34, 0x54, 0x83, 0x1c, 0x6f, 0xa9, 0x9a, 0x4b, 0x74, 0x93, 0x86, 0xd7, 0x12, 0x15, + 0x3a, 0xcb, 0x65, 0x30, 0x15, 0x41, 0x0d, 0xc8, 0x87, 0x18, 0xbc, 0x40, 0x27, 0x96, 0x01, 0x09, + 0x15, 0x07, 0x55, 0x19, 0x7d, 0x07, 0x36, 0x42, 0x14, 0xcb, 0x3e, 0xd2, 0x07, 0x96, 0xc9, 0x82, + 0x43, 0xc6, 0x21, 0x78, 0x2b, 0xa0, 0x2e, 0xec, 0x08, 0xf2, 0xb9, 0x3a, 0x42, 0xd4, 0x07, 0x92, + 0x4a, 0xea, 0xd6, 0x9f, 0x05, 0x48, 0xb3, 0xaf, 0x89, 0xec, 0x13, 0x47, 0x06, 0x52, 0xfb, 0xbb, + 0x0f, 0x77, 0xf7, 0x9e, 0xec, 0x2a, 0x6b, 0x28, 0x05, 0x62, 0x6b, 0x57, 0x55, 0x04, 0x94, 0x86, + 0xc4, 0xbd, 0x9d, 0xbd, 0xaa, 0xaa, 0xac, 0xd3, 0xc7, 0xda, 0x53, 0xb5, 0xd9, 0x55, 0x44, 0x74, + 0x01, 0x36, 0x1a, 0xcd, 0x9d, 0x56, 0xa7, 0xa5, 0x36, 0x1b, 0x5a, 0x40, 0x94, 0x91, 0x0c, 0x92, + 0xda, 0xea, 0x34, 0x15, 0x89, 0x42, 0x35, 0x9a, 0xf5, 0x56, 0xa7, 0xba, 0xa3, 0x24, 0xd0, 0x16, + 0x6c, 0xce, 0x79, 0x43, 0x72, 0x1a, 0x65, 0x41, 0x6e, 0xec, 0xe3, 0xaa, 0xda, 0xda, 0xdb, 0x55, + 0x92, 0x08, 0x20, 0x49, 0x65, 0xd5, 0x67, 0x4a, 0x96, 0xea, 0x51, 0xf7, 0x1f, 0xed, 0x34, 0x15, + 0xa0, 0x4c, 0xb5, 0x96, 0x5a, 0xc5, 0xb8, 0xfa, 0x54, 0xc9, 0xa0, 0x3c, 0x00, 0x65, 0xea, 0x36, + 0x71, 0xab, 0xd9, 0x55, 0xcc, 0x32, 0xed, 0x60, 0xa9, 0x5b, 0x3f, 0x81, 0xcd, 0x53, 0x5f, 0x71, + 0xd0, 0x06, 0x64, 0xaa, 0x8d, 0x86, 0x86, 0x9b, 0x8f, 0x76, 0x5a, 0xf5, 0xaa, 0xb2, 0x86, 0x10, + 0xe4, 0x71, 0xb3, 0xb3, 0xf7, 0xb8, 0x19, 0xd1, 0x84, 0xab, 0xd2, 0x17, 0xbf, 0x2d, 0xae, 0xdd, + 0xda, 0x3b, 0x31, 0xa1, 0x04, 0xad, 0x82, 0x9e, 0xe0, 0x51, 0x73, 0xb7, 0xd1, 0xda, 0xbd, 0xaf, + 0xac, 0xd1, 0x97, 0xae, 0x5a, 0xbd, 0x4f, 0x5f, 0x44, 0x94, 0x83, 0x74, 0x7d, 0xaf, 0xd3, 0x69, + 0xa9, 0x6a, 0xb3, 0xa1, 0x08, 0x74, 0xad, 0x5a, 0xdb, 0xc3, 0xf4, 0x65, 0x3d, 0x00, 0xac, 0x7d, + 0xf7, 0xf5, 0x3f, 0x8b, 0x6b, 0xaf, 0x67, 0x45, 0xe1, 0xcd, 0xac, 0x28, 0x7c, 0x35, 0x2b, 0x0a, + 0xff, 0x98, 0x15, 0x85, 0x2f, 0xdf, 0x16, 0xd7, 0xde, 0xbc, 0x2d, 0xae, 0x7d, 0xf5, 0xb6, 0xb8, + 0xf6, 0x2c, 0xc5, 0x5d, 0xd6, 0x4b, 0xb2, 0x4f, 0xdd, 0x77, 0xfe, 0x1b, 0x00, 0x00, 0xff, 0xff, + 0x18, 0x78, 0x14, 0x05, 0xa4, 0x17, 0x00, 0x00, } diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 97dfb04b3c2f..19ecc29cfa97 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -53,6 +53,7 @@ enum ValueType { DECIMAL = 5; DELIMITED_DECIMAL = 9; DURATION = 6; + TIMETZ = 12; // TUPLE represents a DTuple, encoded as repeated pairs of varint field number // followed by a value encoded Datum. diff --git a/pkg/sql/copy_in_test.go b/pkg/sql/copy_in_test.go index 466cbf940a92..f714c3da5f6e 100644 --- a/pkg/sql/copy_in_test.go +++ b/pkg/sql/copy_in_test.go @@ -50,6 +50,7 @@ func TestCopyNullInfNaN(t *testing.T) { b BYTES NULL, d DATE NULL, t TIME NULL, + ttz TIME NULL, ts TIMESTAMP NULL, n INTERVAL NULL, o BOOL NULL, @@ -68,17 +69,17 @@ func TestCopyNullInfNaN(t *testing.T) { } stmt, err := txn.Prepare(pq.CopyIn( - "t", "i", "f", "s", "b", "d", "t", + "t", "i", "f", "s", "b", "d", "t", "ttz", "ts", "n", "o", "e", "u", "ip", "tz")) if err != nil { t.Fatal(err) } input := [][]interface{}{ - {nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil}, - {nil, math.Inf(1), nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil}, - {nil, math.Inf(-1), nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil}, - {nil, math.NaN(), nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil}, + {nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil}, + {nil, math.Inf(1), nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil}, + {nil, math.Inf(-1), nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil}, + {nil, math.NaN(), nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil}, } for _, in := range input { @@ -146,6 +147,7 @@ func TestCopyRandom(t *testing.T) { f FLOAT, e DECIMAL, t TIME, + ttz TIMETZ, ts TIMESTAMP, s STRING, b BYTES, @@ -163,7 +165,7 @@ func TestCopyRandom(t *testing.T) { t.Fatal(err) } - stmt, err := txn.Prepare(pq.CopyInSchema("d", "t", "id", "n", "o", "i", "f", "e", "t", "ts", "s", "b", "u", "ip", "tz")) + stmt, err := txn.Prepare(pq.CopyInSchema("d", "t", "id", "n", "o", "i", "f", "e", "t", "ttz", "ts", "s", "b", "u", "ip", "tz")) if err != nil { t.Fatal(err) } @@ -177,6 +179,7 @@ func TestCopyRandom(t *testing.T) { types.Float, types.Decimal, types.Time, + types.TimeTZ, types.Timestamp, types.String, types.Bytes, @@ -248,6 +251,8 @@ func TestCopyRandom(t *testing.T) { var dt tree.NodeFormatter if typs[i].Family() == types.TimeFamily { dt = tree.MakeDTime(timeofday.FromTime(d)) + } else if typs[i].Family() == types.TimeTZFamily { + dt = tree.NewDTimeTZFromTime(d) } else { dt = tree.MakeDTimestamp(d, time.Microsecond) } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index ef65ecb8e8f5..ae57d7b68e88 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -769,6 +769,7 @@ func checkResultType(typ *types.T) error { case types.DateFamily: case types.TimestampFamily: case types.TimeFamily: + case types.TimeTZFamily: case types.TimestampTZFamily: case types.IntervalFamily: case types.JsonFamily: diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 23bb34e9219d..c3a6d9266139 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -915,6 +915,8 @@ oid typname typnamespace typowner typlen typbyval typtype 1186 interval 1307062959 NULL 24 true b 1187 _interval 1307062959 NULL -1 false b 1231 _numeric 1307062959 NULL -1 false b +1266 timetz 1307062959 NULL 16 true b +1270 _timetz 1307062959 NULL -1 false b 1560 bit 1307062959 NULL -1 false b 1561 _bit 1307062959 NULL -1 false b 1562 varbit 1307062959 NULL -1 false b @@ -989,6 +991,8 @@ oid typname typcategory typispreferred typisdefined typdelim typrel 1186 interval T false true , 0 0 1187 1187 _interval A false true , 0 1186 0 1231 _numeric A false true , 0 1700 0 +1266 timetz D false true , 0 0 1270 +1270 _timetz A false true , 0 1266 0 1560 bit V false true , 0 0 1561 1561 _bit A false true , 0 1560 0 1562 varbit V false true , 0 0 1563 @@ -1063,6 +1067,8 @@ oid typname typinput typoutput typreceive typsend 1186 interval interval_in interval_out interval_recv interval_send 0 0 0 1187 _interval array_in array_out array_recv array_send 0 0 0 1231 _numeric array_in array_out array_recv array_send 0 0 0 +1266 timetz timetz_in timetz_out timetz_recv timetz_send 0 0 0 +1270 _timetz array_in array_out array_recv array_send 0 0 0 1560 bit bit_in bit_out bit_recv bit_send 0 0 0 1561 _bit array_in array_out array_recv array_send 0 0 0 1562 varbit varbit_in varbit_out varbit_recv varbit_send 0 0 0 @@ -1137,6 +1143,8 @@ oid typname typalign typstorage typnotnull typbasetype typtypmod 1186 interval NULL NULL false 0 -1 1187 _interval NULL NULL false 0 -1 1231 _numeric NULL NULL false 0 -1 +1266 timetz NULL NULL false 0 -1 +1270 _timetz NULL NULL false 0 -1 1560 bit NULL NULL false 0 -1 1561 _bit NULL NULL false 0 -1 1562 varbit NULL NULL false 0 -1 @@ -1211,6 +1219,8 @@ oid typname typndims typcollation typdefaultbin typdefault typacl 1186 interval 0 0 NULL NULL NULL 1187 _interval 0 0 NULL NULL NULL 1231 _numeric 0 0 NULL NULL NULL +1266 timetz 0 0 NULL NULL NULL +1270 _timetz 0 0 NULL NULL NULL 1560 bit 0 0 NULL NULL NULL 1561 _bit 0 0 NULL NULL NULL 1562 varbit 0 0 NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/timetz b/pkg/sql/logictest/testdata/logic_test/timetz new file mode 100644 index 000000000000..0b8387f56a37 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/timetz @@ -0,0 +1,48 @@ +# NOTE: we cast timetz to strings here to avoid confusion with lib/pq +# printing out timetz values with dates. + +statement ok +CREATE TABLE timetz_test (a TIMETZ PRIMARY KEY, b TIMETZ, c INT) + +statement ok +INSERT INTO timetz_test VALUES + ('11:00:00+0', '12:00:00+1', 1), + ('12:00:00+1', '10:00:00-1', 2), + ('13:00:00+2', '15:00:00-6', 3), + ('14:00:00+3', '24:00:00-1559', 4), + ('15:00:00+3', '15:00:00+3', 5) + +# NOTE: lib/pq parses '24:00:00-1559' incorrectly, but the data +# structure underneath should still be correct. +query B +SELECT '24:00:00-1559'::timetz > '23:59:59-1559'::timetz; +---- +true + +query TTI +SELECT a::string, b::string, c FROM timetz_test ORDER BY a, c ASC +---- +11:00:00+00:00:00 12:00:00+01:00:00 1 +12:00:00+01:00:00 10:00:00-01:00:00 2 +13:00:00+02:00:00 15:00:00-06:00:00 3 +14:00:00+03:00:00 24:00:00-15:59:00 4 +15:00:00+03:00:00 15:00:00+03:00:00 5 + +query TTI +SELECT a::string, b::string, c FROM timetz_test ORDER BY a DESC, c ASC +---- +15:00:00+03:00:00 15:00:00+03:00:00 5 +14:00:00+03:00:00 24:00:00-15:59:00 4 +13:00:00+02:00:00 15:00:00-06:00:00 3 +12:00:00+01:00:00 10:00:00-01:00:00 2 +11:00:00+00:00:00 12:00:00+01:00:00 1 + +query TTI +SELECT a::string, b::string, c FROM timetz_test WHERE a > b ORDER BY c +---- +11:00:00+00:00:00 12:00:00+01:00:00 1 + +query TTI +SELECT a::string, b::string, c FROM timetz_test WHERE a = b ORDER BY c +---- +15:00:00+03:00:00 15:00:00+03:00:00 5 diff --git a/pkg/sql/opt/optbuilder/testdata/aggregate b/pkg/sql/opt/optbuilder/testdata/aggregate index 6ca4ff06a372..eb6fca32d42c 100644 --- a/pkg/sql/opt/optbuilder/testdata/aggregate +++ b/pkg/sql/opt/optbuilder/testdata/aggregate @@ -227,6 +227,7 @@ array_agg(oid) -> oid[] array_agg(uuid) -> uuid[] array_agg(inet) -> inet[] array_agg(time) -> time[] +array_agg(timetz) -> timetz[] array_agg(varbit) -> varbit[] array_agg(bool) -> bool[] diff --git a/pkg/sql/parser/parse_test.go b/pkg/sql/parser/parse_test.go index ddd5636ea19f..dfc8eb1e745d 100644 --- a/pkg/sql/parser/parse_test.go +++ b/pkg/sql/parser/parse_test.go @@ -111,6 +111,7 @@ func TestParse(t *testing.T) { {`CREATE TABLE a (b FLOAT8)`}, {`CREATE TABLE a (b SERIAL8)`}, {`CREATE TABLE a (b TIME)`}, + {`CREATE TABLE a (b TIMETZ)`}, {`CREATE TABLE a (b UUID)`}, {`CREATE TABLE a (b INET)`}, {`CREATE TABLE a (b "char")`}, @@ -3149,7 +3150,6 @@ func TestUnimplementedSyntax(t *testing.T) { {`CREATE TABLE a(b TSVECTOR)`, 7821, `tsvector`}, {`CREATE TABLE a(b TXID_SNAPSHOT)`, 0, `txid_snapshot`}, {`CREATE TABLE a(b XML)`, 0, `xml`}, - {`CREATE TABLE a(b TIMETZ)`, 26097, `type`}, {`INSERT INTO a VALUES (1) ON CONFLICT (x) WHERE x > 3 DO NOTHING`, 32557, ``}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 0c783152a1fb..da7f6de7330d 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -7301,7 +7301,7 @@ const_datetime: } $$.val = types.MakeTime(prec) } -| TIMETZ { return unimplementedWithIssueDetail(sqllex, 26097, "type") } +| TIMETZ { $$.val = types.TimeTZ } | TIMETZ '(' ICONST ')' { return unimplementedWithIssueDetail(sqllex, 26097, "type with precision") } | TIMESTAMP opt_timezone { diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index e20f237b6fc4..5f5448a901ad 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -2643,6 +2643,7 @@ var datumToTypeCategory = map[types.Family]*tree.DString{ types.BytesFamily: typCategoryUserDefined, types.DateFamily: typCategoryDateTime, types.TimeFamily: typCategoryDateTime, + types.TimeTZFamily: typCategoryDateTime, types.FloatFamily: typCategoryNumeric, types.IntFamily: typCategoryNumeric, types.IntervalFamily: typCategoryTimespan, diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go index f8783f0962ac..1e5dfa9ebf2c 100644 --- a/pkg/sql/pgwire/pgwire_test.go +++ b/pkg/sql/pgwire/pgwire_test.go @@ -928,6 +928,9 @@ func TestPGPreparedQuery(t *testing.T) { {"SELECT $1::TIME", []preparedQueryTest{ baseTest.SetArgs("12:00:00").Results("0000-01-01T12:00:00Z"), }}, + {"SELECT $1::TIMETZ", []preparedQueryTest{ + baseTest.SetArgs("12:00:00+0330").Results("0000-01-01T12:00:00+03:30"), + }}, {"SELECT $1:::FLOAT[]", []preparedQueryTest{ baseTest.SetArgs("{}").Results("{}"), baseTest.SetArgs("{1.0,2.0,3.0}").Results("{1.0,2.0,3.0}"), diff --git a/pkg/sql/pgwire/pgwirebase/encoding.go b/pkg/sql/pgwire/pgwirebase/encoding.go index 7b6f6d5f2528..ac2ceb0a5df2 100644 --- a/pkg/sql/pgwire/pgwirebase/encoding.go +++ b/pkg/sql/pgwire/pgwirebase/encoding.go @@ -286,7 +286,12 @@ func DecodeOidDatum( return nil, pgerror.Newf(pgcode.Syntax, "could not parse string %q as time", b) } return d, nil - + case oid.T_timetz: + d, err := tree.ParseDTimeTZ(ctx, string(b)) + if err != nil { + return nil, pgerror.Newf(pgcode.Syntax, "could not parse string %q as timetz", b) + } + return d, nil case oid.T_interval: d, err := tree.ParseDInterval(string(b)) if err != nil { @@ -529,6 +534,13 @@ func DecodeOidDatum( } i := int64(binary.BigEndian.Uint64(b)) return tree.MakeDTime(timeofday.TimeOfDay(i)), nil + case oid.T_timetz: + if len(b) < 12 { + return nil, pgerror.Newf(pgcode.Syntax, "timetz requires 12 bytes for binary format") + } + timeOfDayMicros := int64(binary.BigEndian.Uint64(b)) + offsetSecs := int32(binary.BigEndian.Uint32(b)) + return tree.NewDTimeTZFromOffset(timeofday.TimeOfDay(timeOfDayMicros), offsetSecs), nil case oid.T_interval: if len(b) < 16 { return nil, pgerror.Newf(pgcode.Syntax, "interval requires 16 bytes for binary format") diff --git a/pkg/sql/pgwire/types.go b/pkg/sql/pgwire/types.go index 5de55fab9315..4d4a70634d3b 100644 --- a/pkg/sql/pgwire/types.go +++ b/pkg/sql/pgwire/types.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeofday" + "github.com/cockroachdb/cockroach/pkg/util/timetz" "github.com/cockroachdb/errors" "github.com/lib/pq/oid" ) @@ -154,6 +155,12 @@ func (b *writeBuffer) writeTextDatum( b.putInt32(int32(len(s))) b.write(s) + case *tree.DTimeTZ: + // Start at offset 4 because `putInt32` clobbers the first 4 bytes. + s := formatTimeTZ(v.TimeTZ, b.putbuf[4:4]) + b.putInt32(int32(len(s))) + b.write(s) + case *tree.DTimestamp: // Start at offset 4 because `putInt32` clobbers the first 4 bytes. s := formatTs(v.Time, nil, b.putbuf[4:4]) @@ -419,6 +426,11 @@ func (b *writeBuffer) writeBinaryDatum( b.putInt32(8) b.putInt64(int64(*v)) + case *tree.DTimeTZ: + b.putInt32(12) + b.putInt64(int64(v.TimeOfDay)) + b.putInt32(v.OffsetSecs) + case *tree.DInterval: b.putInt32(16) b.putInt64(v.Nanos() / int64(time.Microsecond/time.Nanosecond)) @@ -477,6 +489,7 @@ func (b *writeBuffer) writeBinaryDatum( const ( pgTimeFormat = "15:04:05.999999" + pgTimeTZFormat = pgTimeFormat + "-07:00" pgDateFormat = "2006-01-02" pgTimeStampFormatNoOffset = pgDateFormat + " " + pgTimeFormat pgTimeStampFormat = pgTimeStampFormatNoOffset + "-07:00" @@ -489,6 +502,15 @@ func formatTime(t timeofday.TimeOfDay, tmp []byte) []byte { return t.ToTime().AppendFormat(tmp, pgTimeFormat) } +// formatTimeTZ formats t into a format lib/pq understands, appending to the +// provided tmp buffer and reallocating if needed. The function will then return +// the resulting buffer. +// Note it does not understand the "second" component of the offset as lib/pq +// cannot parse it. +func formatTimeTZ(t timetz.TimeTZ, tmp []byte) []byte { + return t.ToTime().AppendFormat(tmp, pgTimeTZFormat) +} + func formatTs(t time.Time, offset *time.Location, tmp []byte) (b []byte) { var format string if offset != nil { diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index 58d82618dafe..c6a9f18ef3d4 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -62,6 +62,7 @@ var typeBuiltinsHaveUnderscore = map[oid.Oid]struct{}{ types.AnyArray.Oid(): {}, types.Date.Oid(): {}, types.Time.Oid(): {}, + types.TimeTZ.Oid(): {}, types.Decimal.Oid(): {}, types.Interval.Oid(): {}, types.Jsonb.Oid(): {}, diff --git a/pkg/sql/sem/tree/constant.go b/pkg/sql/sem/tree/constant.go index 229f4b0b3171..64dcca98519e 100644 --- a/pkg/sql/sem/tree/constant.go +++ b/pkg/sql/sem/tree/constant.go @@ -454,6 +454,7 @@ var ( types.Date, types.StringArray, types.Time, + types.TimeTZ, types.Timestamp, types.TimestampTZ, types.Interval, diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index aae335bba73a..f198c79e89f3 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/stringencoding" "github.com/cockroachdb/cockroach/pkg/util/timeofday" + "github.com/cockroachdb/cockroach/pkg/util/timetz" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" "github.com/cockroachdb/cockroach/pkg/util/uint128" @@ -63,8 +64,8 @@ var ( // DZero is the zero-valued integer Datum. DZero = NewDInt(0) - // DTimeRegex is a compiled regex for parsing the 24:00 time value - DTimeRegex = regexp.MustCompile("^24:00($|(:00$)|(:00.0+$))") + // DTimeMaxTimeRegex is a compiled regex for parsing the 24:00 time value. + DTimeMaxTimeRegex = regexp.MustCompile("^24:00($|(:00$)|(:00.0+$))") ) // Datum represents a SQL value. @@ -1866,11 +1867,11 @@ func ParseDTime(ctx ParseTimeContext, s string) (*DTime, error) { // special case on 24:00 and 24:00:00 as the parser // does not handle these correctly. - if DTimeRegex.MatchString(s) { + if DTimeMaxTimeRegex.MatchString(s) { return MakeDTime(timeofday.Time2400), nil } - t, err := pgdate.ParseTime(now, 0 /* mode */, s) + t, err := pgdate.ParseTime(now, pgdate.ParseModeYMD, s) if err != nil { // Build our own error message to avoid exposing the dummy date. return nil, makeParseError(s, types.Time, nil) @@ -1948,6 +1949,118 @@ func (d *DTime) Size() uintptr { return unsafe.Sizeof(*d) } +// DTimeTZ is the time with time zone Datum. +type DTimeTZ struct { + timetz.TimeTZ +} + +var ( + dMinTimeTZ = NewDTimeTZFromOffset(timeofday.Min, timetz.MinTimeTZOffsetSecs) + dMaxTimeTZ = NewDTimeTZFromOffset(timeofday.Time2400, timetz.MaxTimeTZOffsetSecs) +) + +// NewDTimeTZ creates a DTimeTZ from a timetz.TimeTZ. +func NewDTimeTZ(t timetz.TimeTZ) *DTimeTZ { + return &DTimeTZ{t} +} + +// NewDTimeTZFromTime creates a DTimeTZ from time.Time. +func NewDTimeTZFromTime(t time.Time) *DTimeTZ { + return &DTimeTZ{timetz.MakeTimeTZFromTime(t)} +} + +// NewDTimeTZFromOffset creates a DTimeTZ from a TimeOfDay and offset. +func NewDTimeTZFromOffset(t timeofday.TimeOfDay, offsetSecs int32) *DTimeTZ { + return &DTimeTZ{timetz.MakeTimeTZ(t, offsetSecs)} +} + +// NewDTimeTZFromLocation creates a DTimeTZ from a TimeOfDay and time.Location. +func NewDTimeTZFromLocation(t timeofday.TimeOfDay, loc *time.Location) *DTimeTZ { + return &DTimeTZ{timetz.MakeTimeTZFromLocation(t, loc)} +} + +// ParseDTimeTZ parses and returns the *DTime Datum value represented by the +// provided string, or an error if parsing is unsuccessful. +func ParseDTimeTZ(ctx ParseTimeContext, s string) (*DTimeTZ, error) { + now := relativeParseTime(ctx) + d, err := timetz.ParseTimeTZ(now, s) + if err != nil { + return nil, err + } + return NewDTimeTZ(d), nil +} + +// ResolvedType implements the TypedExpr interface. +func (*DTimeTZ) ResolvedType() *types.T { + return types.TimeTZ +} + +// Compare implements the Datum interface. +func (d *DTimeTZ) Compare(ctx *EvalContext, other Datum) int { + if other == DNull { + // NULL is less than any non-NULL value. + return 1 + } + return compareTimestamps(ctx, d, other) +} + +// Prev implements the Datum interface. +func (d *DTimeTZ) Prev(ctx *EvalContext) (Datum, bool) { + if d.IsMin(ctx) { + return nil, false + } + return NewDTimeTZFromOffset(d.TimeOfDay-1, d.OffsetSecs), true +} + +// Next implements the Datum interface. +func (d *DTimeTZ) Next(ctx *EvalContext) (Datum, bool) { + if d.IsMax(ctx) { + return nil, false + } + return NewDTimeTZFromOffset(d.TimeOfDay+1, d.OffsetSecs), true +} + +// IsMax implements the Datum interface. +func (d *DTimeTZ) IsMax(_ *EvalContext) bool { + return d.TimeOfDay == dMaxTimeTZ.TimeOfDay && d.OffsetSecs == timetz.MaxTimeTZOffsetSecs +} + +// IsMin implements the Datum interface. +func (d *DTimeTZ) IsMin(_ *EvalContext) bool { + return d.TimeOfDay == dMinTimeTZ.TimeOfDay && d.OffsetSecs == timetz.MinTimeTZOffsetSecs +} + +// Max implements the Datum interface. +func (d *DTimeTZ) Max(_ *EvalContext) (Datum, bool) { + return dMaxTimeTZ, true +} + +// Min implements the Datum interface. +func (d *DTimeTZ) Min(_ *EvalContext) (Datum, bool) { + return dMinTimeTZ, true +} + +// AmbiguousFormat implements the Datum interface. +func (*DTimeTZ) AmbiguousFormat() bool { return true } + +// Format implements the NodeFormatter interface. +func (d *DTimeTZ) Format(ctx *FmtCtx) { + f := ctx.flags + bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings)) + if !bareStrings { + ctx.WriteByte('\'') + } + ctx.WriteString(d.TimeTZ.String()) + if !bareStrings { + ctx.WriteByte('\'') + } +} + +// Size implements the Datum interface. +func (d *DTimeTZ) Size() uintptr { + return unsafe.Sizeof(*d) +} + // DTimestamp is the timestamp Datum. type DTimestamp struct { time.Time @@ -1968,7 +2081,7 @@ const ( // the provided string in UTC, or an error if parsing is unsuccessful. func ParseDTimestamp(ctx ParseTimeContext, s string, precision time.Duration) (*DTimestamp, error) { now := relativeParseTime(ctx) - t, err := pgdate.ParseTimestamp(now, 0 /* mode */, s) + t, err := pgdate.ParseTimestamp(now, pgdate.ParseModeYMD, s) if err != nil { return nil, err } @@ -2022,6 +2135,8 @@ func timeFromDatum(ctx *EvalContext, d Datum) (time.Time, bool) { return t.Time, true case *DTime: return timeofday.TimeOfDay(*t).ToTime(), true + case *DTimeTZ: + return t.ToTime(), true default: return time.Time{}, false } @@ -2039,6 +2154,23 @@ func compareTimestamps(ctx *EvalContext, l Datum, r Datum) int { if rTime.Before(lTime) { return 1 } + // If either side is a TimeTZ, then we must compare timezones before + // when comparing. + // This is a special quirk of TimeTZ and does not apply to TimestampTZ. + lOffset := int32(0) + rOffset := int32(0) + if _, ok := l.(*DTimeTZ); ok { + lOffset = l.(*DTimeTZ).OffsetSecs + } + if _, ok := r.(*DTimeTZ); ok { + rOffset = r.(*DTimeTZ).OffsetSecs + } + if lOffset > rOffset { + return 1 + } + if lOffset < rOffset { + return -1 + } return 0 } @@ -2133,7 +2265,7 @@ func ParseDTimestampTZ( ctx ParseTimeContext, s string, precision time.Duration, ) (*DTimestampTZ, error) { now := relativeParseTime(ctx) - t, err := pgdate.ParseTimestamp(now, 0 /* mode */, s) + t, err := pgdate.ParseTimestamp(now, pgdate.ParseModeYMD, s) if err != nil { return nil, err } @@ -2573,7 +2705,7 @@ func AsJSON(d Datum) (json.JSON, error) { case *DTimestamp: // This is RFC3339Nano, but without the TZ fields. return json.FromString(t.UTC().Format("2006-01-02T15:04:05.999999999")), nil - case *DDate, *DUuid, *DOid, *DInterval, *DBytes, *DIPAddr, *DTime, *DBitArray: + case *DDate, *DUuid, *DOid, *DInterval, *DBytes, *DIPAddr, *DTime, *DTimeTZ, *DBitArray: return json.FromString(AsStringWithFlags(t, FmtBareStrings)), nil default: if d == DNull { @@ -3719,6 +3851,7 @@ var baseDatumTypeSizes = map[types.Family]struct { types.BytesFamily: {unsafe.Sizeof(DBytes("")), variableSize}, types.DateFamily: {unsafe.Sizeof(DDate{}), fixedSize}, types.TimeFamily: {unsafe.Sizeof(DTime(0)), fixedSize}, + types.TimeTZFamily: {unsafe.Sizeof(DTimeTZ{}), fixedSize}, types.TimestampFamily: {unsafe.Sizeof(DTimestamp{}), fixedSize}, types.TimestampTZFamily: {unsafe.Sizeof(DTimestampTZ{}), fixedSize}, types.IntervalFamily: {unsafe.Sizeof(DInterval{}), fixedSize}, diff --git a/pkg/sql/sem/tree/datum_invariants_test.go b/pkg/sql/sem/tree/datum_invariants_test.go index bea6efe9d2a0..3f2a7eb2dac7 100644 --- a/pkg/sql/sem/tree/datum_invariants_test.go +++ b/pkg/sql/sem/tree/datum_invariants_test.go @@ -11,10 +11,13 @@ package tree import ( + "fmt" "testing" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/timeofday" + "github.com/stretchr/testify/assert" ) func TestAllTypesCastableToString(t *testing.T) { @@ -34,3 +37,64 @@ func TestAllTypesCastableFromString(t *testing.T) { } } } + +func TestCompareTimestamps(t *testing.T) { + defer leaktest.AfterTest(t)() + + pacificTimeZone := int32(7 * 60 * 60) + sydneyTimeZone := int32(-10 * 60 * 60) + + testCases := []struct { + desc string + left Datum + right Datum + expected int + }{ + { + desc: "same DTime are equal", + left: MakeDTime(timeofday.New(12, 0, 0, 0)), + right: MakeDTime(timeofday.New(12, 0, 0, 0)), + expected: 0, + }, + { + desc: "same DTimeTZ are equal", + left: NewDTimeTZFromOffset(timeofday.New(22, 0, 0, 0), sydneyTimeZone), + right: NewDTimeTZFromOffset(timeofday.New(22, 0, 0, 0), sydneyTimeZone), + expected: 0, + }, + { + desc: "equal wall clock time for DTime and DTimeTZ, with TimeTZ ahead", + left: MakeDTime(timeofday.New(12, 0, 0, 0)), + right: NewDTimeTZFromOffset(timeofday.New(22, 0, 0, 0), sydneyTimeZone), + expected: 1, + }, + { + desc: "equal wall clock time for DTime and DTimeTZ, with TimeTZ behind", + left: MakeDTime(timeofday.New(12, 0, 0, 0)), + right: NewDTimeTZFromOffset(timeofday.New(5, 0, 0, 0), pacificTimeZone), + expected: -1, + }, + { + desc: "equal wall clock time for DTime and DTimeTZ, with TimeTZ ahead", + left: NewDTimeTZFromOffset(timeofday.New(22, 0, 0, 0), sydneyTimeZone), + right: NewDTimeTZFromOffset(timeofday.New(5, 0, 0, 0), pacificTimeZone), + expected: -1, + }, + { + desc: "wall clock time different for DTimeTZ and DTimeTZ", + left: NewDTimeTZFromOffset(timeofday.New(23, 0, 0, 0), sydneyTimeZone), + right: NewDTimeTZFromOffset(timeofday.New(5, 0, 0, 0), pacificTimeZone), + expected: 1, + }, + } + + for _, tc := range testCases { + t.Run( + fmt.Sprintf("%s cmp %s", tc.left.String(), tc.right.String()), + func(t *testing.T) { + assert.Equal(nil, tc.expected, compareTimestamps(nil, tc.left, tc.right)) + assert.Equal(nil, -tc.expected, compareTimestamps(nil, tc.right, tc.left)) + }, + ) + } +} diff --git a/pkg/sql/sem/tree/datum_test.go b/pkg/sql/sem/tree/datum_test.go index 3dde45cd72dd..ebe0b8d7e9bd 100644 --- a/pkg/sql/sem/tree/datum_test.go +++ b/pkg/sql/sem/tree/datum_test.go @@ -24,6 +24,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/timeofday" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func prepareExpr(t *testing.T, datumExpr string) tree.TypedExpr { @@ -587,6 +589,120 @@ func TestMakeDJSON(t *testing.T) { } } +func TestDTimeTZ(t *testing.T) { + defer leaktest.AfterTest(t)() + + maxTime, err := tree.ParseDTimeTZ(nil, "24:00:00-1559") + require.NoError(t, err) + minTime, err := tree.ParseDTimeTZ(nil, "00:00:00+1559") + require.NoError(t, err) + + // These are all the same UTC time equivalents. + utcTime, err := tree.ParseDTimeTZ(nil, "11:14:15+0") + require.NoError(t, err) + sydneyTime, err := tree.ParseDTimeTZ(nil, "21:14:15+10") + require.NoError(t, err) + + // No daylight savings in Hawaii! + hawaiiZone, err := time.LoadLocation("Pacific/Honolulu") + require.NoError(t, err) + hawaiiTime := tree.NewDTimeTZFromLocation(timeofday.New(1, 14, 15, 0), hawaiiZone) + + weirdTimeZone := tree.NewDTimeTZFromOffset(timeofday.New(10, 0, 0, 0), -((5 * 60 * 60) + 30*60 + 15)) + + testCases := []struct { + t *tree.DTimeTZ + largerThan []tree.Datum + smallerThan []tree.Datum + equalTo []tree.Datum + isMax bool + isMin bool + }{ + { + t: weirdTimeZone, + largerThan: []tree.Datum{minTime, tree.DNull}, + smallerThan: []tree.Datum{maxTime}, + equalTo: []tree.Datum{weirdTimeZone}, + isMax: false, + isMin: false, + }, + { + t: utcTime, + largerThan: []tree.Datum{minTime, sydneyTime, tree.DNull}, + smallerThan: []tree.Datum{maxTime, hawaiiTime}, + equalTo: []tree.Datum{utcTime}, + isMax: false, + isMin: false, + }, + { + t: sydneyTime, + largerThan: []tree.Datum{minTime, tree.DNull}, + smallerThan: []tree.Datum{maxTime, utcTime, hawaiiTime}, + equalTo: []tree.Datum{sydneyTime}, + isMax: false, + isMin: false, + }, + { + t: hawaiiTime, + largerThan: []tree.Datum{minTime, utcTime, sydneyTime, tree.DNull}, + smallerThan: []tree.Datum{maxTime}, + equalTo: []tree.Datum{hawaiiTime}, + isMax: false, + isMin: false, + }, + { + t: minTime, + largerThan: []tree.Datum{tree.DNull}, + smallerThan: []tree.Datum{maxTime, utcTime, sydneyTime, hawaiiTime}, + equalTo: []tree.Datum{minTime}, + isMax: false, + isMin: true, + }, + { + t: maxTime, + largerThan: []tree.Datum{minTime, utcTime, sydneyTime, hawaiiTime, tree.DNull}, + smallerThan: []tree.Datum{}, + equalTo: []tree.Datum{maxTime}, + isMax: true, + isMin: false, + }, + } + for i, tc := range testCases { + t.Run(fmt.Sprintf("#%d %s", i, tc.t.String()), func(t *testing.T) { + var largerThan []tree.Datum + prev, ok := tc.t.Prev(nil) + if !tc.isMin { + assert.True(t, ok) + largerThan = append(largerThan, prev) + } else { + assert.False(t, ok) + } + for _, largerThan := range append(largerThan, tc.largerThan...) { + assert.Equal(t, 1, tc.t.Compare(nil, largerThan), "%s > %s", tc.t.String(), largerThan.String()) + } + + var smallerThan []tree.Datum + next, ok := tc.t.Next(nil) + if !tc.isMax { + assert.True(t, ok) + smallerThan = append(smallerThan, next) + } else { + assert.False(t, ok) + } + for _, smallerThan := range append(smallerThan, tc.smallerThan...) { + assert.Equal(t, -1, tc.t.Compare(nil, smallerThan), "%s < %s", tc.t.String(), smallerThan.String()) + } + + for _, equalTo := range tc.equalTo { + assert.Equal(t, 0, tc.t.Compare(nil, equalTo), "%s = %s", tc.t.String(), equalTo.String()) + } + + assert.Equal(t, tc.isMax, tc.t.IsMax(nil)) + assert.Equal(t, tc.isMin, tc.t.IsMin(nil)) + }) + } +} + func TestIsDistinctFrom(t *testing.T) { defer leaktest.AfterTest(t)() testData := []struct { diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index b8bc75b6ade6..9b8095233839 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -1776,6 +1776,7 @@ var CmpOps = cmpOpFixups(map[ComparisonOperator]cmpOpOverload{ makeEqFn(types.Oid, types.Oid), makeEqFn(types.String, types.String), makeEqFn(types.Time, types.Time), + makeEqFn(types.TimeTZ, types.TimeTZ), makeEqFn(types.Timestamp, types.Timestamp), makeEqFn(types.TimestampTZ, types.TimestampTZ), makeEqFn(types.Uuid, types.Uuid), @@ -1819,6 +1820,7 @@ var CmpOps = cmpOpFixups(map[ComparisonOperator]cmpOpOverload{ makeLtFn(types.Oid, types.Oid), makeLtFn(types.String, types.String), makeLtFn(types.Time, types.Time), + makeLtFn(types.TimeTZ, types.TimeTZ), makeLtFn(types.Timestamp, types.Timestamp), makeLtFn(types.TimestampTZ, types.TimestampTZ), makeLtFn(types.Uuid, types.Uuid), @@ -1862,6 +1864,7 @@ var CmpOps = cmpOpFixups(map[ComparisonOperator]cmpOpOverload{ makeLeFn(types.Oid, types.Oid), makeLeFn(types.String, types.String), makeLeFn(types.Time, types.Time), + makeLeFn(types.TimeTZ, types.TimeTZ), makeLeFn(types.Timestamp, types.Timestamp), makeLeFn(types.TimestampTZ, types.TimestampTZ), makeLeFn(types.Uuid, types.Uuid), @@ -1914,6 +1917,7 @@ var CmpOps = cmpOpFixups(map[ComparisonOperator]cmpOpOverload{ makeIsFn(types.Oid, types.Oid), makeIsFn(types.String, types.String), makeIsFn(types.Time, types.Time), + makeIsFn(types.TimeTZ, types.TimeTZ), makeIsFn(types.Timestamp, types.Timestamp), makeIsFn(types.TimestampTZ, types.TimestampTZ), makeIsFn(types.Uuid, types.Uuid), @@ -1962,6 +1966,7 @@ var CmpOps = cmpOpFixups(map[ComparisonOperator]cmpOpOverload{ makeEvalTupleIn(types.Oid), makeEvalTupleIn(types.String), makeEvalTupleIn(types.Time), + makeEvalTupleIn(types.TimeTZ), makeEvalTupleIn(types.Timestamp), makeEvalTupleIn(types.TimestampTZ), makeEvalTupleIn(types.Uuid), @@ -3329,7 +3334,7 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { ctx.SessionData.DataConversion.GetFloatPrec(), 64) case *DBool, *DInt, *DDecimal: s = d.String() - case *DTimestamp, *DTimestampTZ, *DDate, *DTime: + case *DTimestamp, *DTimestampTZ, *DDate, *DTime, *DTimeTZ: s = AsStringWithFlags(d, FmtBareStrings) case *DTuple: s = AsStringWithFlags(d, FmtPgwireText) @@ -3446,6 +3451,15 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { return MakeDTime(timeofday.Min.Add(d.Duration)), nil } + case types.TimeTZFamily: + switch d := d.(type) { + case *DString: + return ParseDTimeTZ(ctx, string(*d)) + case *DCollatedString: + return ParseDTimeTZ(ctx, d.Contents) + // TODO(otan#26097): expand for other valid types. + } + case types.TimestampFamily: // TODO(knz): Timestamp from float, decimal. switch d := d.(type) { @@ -4178,6 +4192,11 @@ func (t *DTime) Eval(_ *EvalContext) (Datum, error) { return t, nil } +// Eval implements the TypedExpr interface. +func (t *DTimeTZ) Eval(_ *EvalContext) (Datum, error) { + return t, nil +} + // Eval implements the TypedExpr interface. func (t *DFloat) Eval(_ *EvalContext) (Datum, error) { return t, nil diff --git a/pkg/sql/sem/tree/expr.go b/pkg/sql/sem/tree/expr.go index e05da26dc246..c79fab2e6706 100644 --- a/pkg/sql/sem/tree/expr.go +++ b/pkg/sql/sem/tree/expr.go @@ -1528,11 +1528,12 @@ var ( stringCastTypes = annotateCast(types.String, []*types.T{types.Unknown, types.Bool, types.Int, types.Float, types.Decimal, types.String, types.AnyCollatedString, types.VarBit, types.AnyArray, types.AnyTuple, - types.Bytes, types.Timestamp, types.TimestampTZ, types.Interval, types.Uuid, types.Date, types.Time, types.Oid, types.INet, types.Jsonb}) + types.Bytes, types.Timestamp, types.TimestampTZ, types.Interval, types.Uuid, types.Date, types.Time, types.TimeTZ, types.Oid, types.INet, types.Jsonb}) bytesCastTypes = annotateCast(types.Bytes, []*types.T{types.Unknown, types.String, types.AnyCollatedString, types.Bytes, types.Uuid}) dateCastTypes = annotateCast(types.Date, []*types.T{types.Unknown, types.String, types.AnyCollatedString, types.Date, types.Timestamp, types.TimestampTZ, types.Int}) - timeCastTypes = annotateCast(types.Time, []*types.T{types.Unknown, types.String, types.AnyCollatedString, types.Time, + timeCastTypes = annotateCast(types.Time, []*types.T{types.Unknown, types.String, types.AnyCollatedString, types.Time, types.TimeTZ, types.Timestamp, types.TimestampTZ, types.Interval}) + timeTZCastTypes = annotateCast(types.TimeTZ, []*types.T{types.Unknown, types.String, types.AnyCollatedString, types.Time, types.TimeTZ, types.TimestampTZ}) timestampCastTypes = annotateCast(types.Timestamp, []*types.T{types.Unknown, types.String, types.AnyCollatedString, types.Date, types.Timestamp, types.TimestampTZ, types.Int}) intervalCastTypes = annotateCast(types.Interval, []*types.T{types.Unknown, types.String, types.AnyCollatedString, types.Int, types.Time, types.Interval, types.Float, types.Decimal}) oidCastTypes = annotateCast(types.Oid, []*types.T{types.Unknown, types.String, types.AnyCollatedString, types.Int, types.Oid}) @@ -1563,6 +1564,8 @@ func validCastTypes(t *types.T) []castInfo { return dateCastTypes case types.TimeFamily: return timeCastTypes + case types.TimeTZFamily: + return timeTZCastTypes case types.TimestampFamily, types.TimestampTZFamily: return timestampCastTypes case types.IntervalFamily: @@ -1736,6 +1739,7 @@ func (node *DBool) String() string { return AsString(node) } func (node *DBytes) String() string { return AsString(node) } func (node *DDate) String() string { return AsString(node) } func (node *DTime) String() string { return AsString(node) } +func (node *DTimeTZ) String() string { return AsString(node) } func (node *DDecimal) String() string { return AsString(node) } func (node *DFloat) String() string { return AsString(node) } func (node *DInt) String() string { return AsString(node) } diff --git a/pkg/sql/sem/tree/parse_string.go b/pkg/sql/sem/tree/parse_string.go index 91e178981159..18108730bd84 100644 --- a/pkg/sql/sem/tree/parse_string.go +++ b/pkg/sql/sem/tree/parse_string.go @@ -82,6 +82,8 @@ func parseStringAs(t *types.T, s string, ctx ParseTimeContext) (Datum, error) { return NewDString(s), nil case types.TimeFamily: return ParseDTime(ctx, s) + case types.TimeTZFamily: + return ParseDTimeTZ(ctx, s) case types.TimestampFamily: return ParseDTimestamp(ctx, s, time.Microsecond) case types.TimestampTZFamily: diff --git a/pkg/sql/sem/tree/parse_string_test.go b/pkg/sql/sem/tree/parse_string_test.go index 0904b6ce5d7f..871612ec9e5b 100644 --- a/pkg/sql/sem/tree/parse_string_test.go +++ b/pkg/sql/sem/tree/parse_string_test.go @@ -104,6 +104,13 @@ func TestParseDatumStringAs(t *testing.T) { "01:02:03", "02:03:04.123456", }, + types.TimeTZ: { + "01:02:03+00:00:00", + "01:02:03+11:00:00", + "01:02:03+11:00:00", + "01:02:03-11:00:00", + "02:03:04.123456+11:00:00", + }, types.Timestamp: { "2001-01-01 01:02:03+00:00", "2001-01-01 02:03:04.123456+00:00", diff --git a/pkg/sql/sem/tree/testutils.go b/pkg/sql/sem/tree/testutils.go index 5e860d1e91cf..317f8bf4e089 100644 --- a/pkg/sql/sem/tree/testutils.go +++ b/pkg/sql/sem/tree/testutils.go @@ -59,6 +59,8 @@ func SampleDatum(t *types.T) Datum { return NewDDate(pgdate.MakeCompatibleDateFromDisk(123123)) case types.TimeFamily: return MakeDTime(timeofday.FromInt(789)) + case types.TimeTZFamily: + return NewDTimeTZFromOffset(timeofday.FromInt(345), 5*60*60 /* OffsetSecs */) case types.TimestampFamily: return MakeDTimestamp(timeutil.Unix(123, 123), time.Second) case types.TimestampTZFamily: diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index 4ec39d47016f..de27ce7e06fa 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -1373,6 +1373,10 @@ func (d *DDate) TypeCheck(_ *SemaContext, _ *types.T) (TypedExpr, error) { retur // identity function for Datum. func (d *DTime) TypeCheck(_ *SemaContext, _ *types.T) (TypedExpr, error) { return d, nil } +// TypeCheck implements the Expr interface. It is implemented as an idempotent +// identity function for Datum. +func (d *DTimeTZ) TypeCheck(_ *SemaContext, _ *types.T) (TypedExpr, error) { return d, nil } + // TypeCheck implements the Expr interface. It is implemented as an idempotent // identity function for Datum. func (d *DTimestamp) TypeCheck(_ *SemaContext, _ *types.T) (TypedExpr, error) { return d, nil } diff --git a/pkg/sql/sem/tree/walk.go b/pkg/sql/sem/tree/walk.go index d00a7f0f468f..7c02cfbf17fa 100644 --- a/pkg/sql/sem/tree/walk.go +++ b/pkg/sql/sem/tree/walk.go @@ -619,6 +619,9 @@ func (expr *DDate) Walk(_ Visitor) Expr { return expr } // Walk implements the Expr interface. func (expr *DTime) Walk(_ Visitor) Expr { return expr } +// Walk implements the Expr interface. +func (expr *DTimeTZ) Walk(_ Visitor) Expr { return expr } + // Walk implements the Expr interface. func (expr *DFloat) Walk(_ Visitor) Expr { return expr } diff --git a/pkg/sql/sqlbase/column_type_encoding.go b/pkg/sql/sqlbase/column_type_encoding.go index 35092c8edc7b..afc6d69ccc33 100644 --- a/pkg/sql/sqlbase/column_type_encoding.go +++ b/pkg/sql/sqlbase/column_type_encoding.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/cockroach/pkg/util/json" + "github.com/cockroachdb/cockroach/pkg/util/timetz" "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -111,6 +112,11 @@ func EncodeTableKey(b []byte, val tree.Datum, dir encoding.Direction) ([]byte, e return encoding.EncodeTimeAscending(b, t.Time), nil } return encoding.EncodeTimeDescending(b, t.Time), nil + case *tree.DTimeTZ: + if dir == encoding.Ascending { + return encoding.EncodeTimeTZAscending(b, t.TimeTZ), nil + } + return encoding.EncodeTimeTZDescending(b, t.TimeTZ), nil case *tree.DInterval: if dir == encoding.Ascending { return encoding.EncodeDurationAscending(b, t.Duration) @@ -266,6 +272,14 @@ func DecodeTableKey( rkey, t, err = encoding.DecodeVarintDescending(key) } return a.NewDTime(tree.DTime(t)), rkey, err + case types.TimeTZFamily: + var t timetz.TimeTZ + if dir == encoding.Ascending { + rkey, t, err = encoding.DecodeTimeTZAscending(key) + } else { + rkey, t, err = encoding.DecodeTimeTZDescending(key) + } + return a.NewDTimeTZ(tree.DTimeTZ{TimeTZ: t}), rkey, err case types.TimestampFamily: var t time.Time if dir == encoding.Ascending { @@ -364,6 +378,8 @@ func EncodeTableValue( return encoding.EncodeIntValue(appendTo, uint32(colID), t.UnixEpochDaysWithOrig()), nil case *tree.DTime: return encoding.EncodeIntValue(appendTo, uint32(colID), int64(*t)), nil + case *tree.DTimeTZ: + return encoding.EncodeTimeTZValue(appendTo, uint32(colID), t.TimeTZ), nil case *tree.DTimestamp: return encoding.EncodeTimeValue(appendTo, uint32(colID), t.Time), nil case *tree.DTimestampTZ: @@ -480,6 +496,12 @@ func decodeUntaggedDatum(a *DatumAlloc, t *types.T, buf []byte) (tree.Datum, []b return nil, b, err } return a.NewDTime(tree.DTime(data)), b, nil + case types.TimeTZFamily: + b, data, err := encoding.DecodeUntaggedTimeTZValue(buf) + if err != nil { + return nil, b, err + } + return a.NewDTimeTZ(tree.DTimeTZ{TimeTZ: data}), b, nil case types.TimestampFamily: b, data, err := encoding.DecodeUntaggedTimeValue(buf) if err != nil { @@ -610,6 +632,11 @@ func MarshalColumnValue(col *ColumnDescriptor, val tree.Datum) (roachpb.Value, e r.SetInt(int64(*v)) return r, nil } + case types.TimeTZFamily: + if v, ok := val.(*tree.DTimeTZ); ok { + r.SetTimeTZ(v.TimeTZ) + return r, nil + } case types.TimestampFamily: if v, ok := val.(*tree.DTimestamp); ok { r.SetTime(v.Time) @@ -751,6 +778,12 @@ func UnmarshalColumnValue(a *DatumAlloc, typ *types.T, value roachpb.Value) (tre return nil, err } return a.NewDTime(tree.DTime(v)), nil + case types.TimeTZFamily: + v, err := value.GetTimeTZ() + if err != nil { + return nil, err + } + return a.NewDTimeTZ(tree.DTimeTZ{TimeTZ: v}), nil case types.TimestampFamily: v, err := value.GetTime() if err != nil { @@ -1063,6 +1096,8 @@ func datumTypeToArrayElementEncodingType(t *types.T) (encoding.Type, error) { // persisted with incorrect elementType values. case types.DateFamily, types.TimeFamily: return encoding.Int, nil + case types.TimeTZFamily: + return encoding.TimeTZ, nil case types.IntervalFamily: return encoding.Duration, nil case types.BoolFamily: @@ -1118,6 +1153,8 @@ func encodeArrayElement(b []byte, d tree.Datum) ([]byte, error) { return encoding.EncodeUntaggedIntValue(b, t.UnixEpochDaysWithOrig()), nil case *tree.DTime: return encoding.EncodeUntaggedIntValue(b, int64(*t)), nil + case *tree.DTimeTZ: + return encoding.EncodeUntaggedTimeTZValue(b, t.TimeTZ), nil case *tree.DTimestamp: return encoding.EncodeUntaggedTimeValue(b, t.Time), nil case *tree.DTimestampTZ: diff --git a/pkg/sql/sqlbase/datum_alloc.go b/pkg/sql/sqlbase/datum_alloc.go index 01ae98fac108..4d7c1e6d113b 100644 --- a/pkg/sql/sqlbase/datum_alloc.go +++ b/pkg/sql/sqlbase/datum_alloc.go @@ -24,6 +24,7 @@ type DatumAlloc struct { ddecimalAlloc []tree.DDecimal ddateAlloc []tree.DDate dtimeAlloc []tree.DTime + dtimetzAlloc []tree.DTimeTZ dtimestampAlloc []tree.DTimestamp dtimestampTzAlloc []tree.DTimestampTZ dintervalAlloc []tree.DInterval @@ -155,6 +156,18 @@ func (a *DatumAlloc) NewDTime(v tree.DTime) *tree.DTime { return r } +// NewDTimeTZ allocates a DTimeTZ. +func (a *DatumAlloc) NewDTimeTZ(v tree.DTimeTZ) *tree.DTimeTZ { + buf := &a.dtimetzAlloc + if len(*buf) == 0 { + *buf = make([]tree.DTimeTZ, datumAllocSize) + } + r := &(*buf)[0] + *r = v + *buf = (*buf)[1:] + return r +} + // NewDTimestamp allocates a DTimestamp. func (a *DatumAlloc) NewDTimestamp(v tree.DTimestamp) *tree.DTimestamp { buf := &a.dtimestampAlloc diff --git a/pkg/sql/sqlbase/table.go b/pkg/sql/sqlbase/table.go index 2400464bb97a..5ed05d17bde1 100644 --- a/pkg/sql/sqlbase/table.go +++ b/pkg/sql/sqlbase/table.go @@ -100,7 +100,7 @@ func ValidateColumnDefType(t *types.T) error { case types.BitFamily, types.IntFamily, types.FloatFamily, types.BoolFamily, types.BytesFamily, types.DateFamily, types.INetFamily, types.IntervalFamily, types.JsonFamily, types.OidFamily, types.TimeFamily, - types.TimestampFamily, types.TimestampTZFamily, types.UuidFamily: + types.TimestampFamily, types.TimestampTZFamily, types.UuidFamily, types.TimeTZFamily: // These types are OK. default: diff --git a/pkg/sql/sqlbase/testutils.go b/pkg/sql/sqlbase/testutils.go index 69440ccb5bdb..394ecc913c64 100644 --- a/pkg/sql/sqlbase/testutils.go +++ b/pkg/sql/sqlbase/testutils.go @@ -159,6 +159,14 @@ func RandDatumWithNullChance(rng *rand.Rand, typ *types.T, nullChance int) tree. return tree.NewDDate(d) case types.TimeFamily: return tree.MakeDTime(timeofday.Random(rng)) + case types.TimeTZFamily: + return tree.NewDTimeTZFromOffset( + timeofday.Random(rng), + // We cannot randomize seconds, because lib/pq does NOT print the + // second offsets making some tests break when comparing + // results in == results out using string comparison. + (rng.Int31n(28*60+59)-(14*60+59))*60, + ) case types.TimestampFamily: return tree.MakeDTimestamp(timeutil.Unix(rng.Int63n(1000000), rng.Int63n(1000000)), time.Microsecond) case types.IntervalFamily: diff --git a/pkg/sql/types/oid.go b/pkg/sql/types/oid.go index f23a034d8250..fac28ffe3be6 100644 --- a/pkg/sql/types/oid.go +++ b/pkg/sql/types/oid.go @@ -82,6 +82,7 @@ var OidToType = map[oid.Oid]*T{ oid.T_regtype: RegType, oid.T_text: String, oid.T_time: Time, + oid.T_timetz: TimeTZ, oid.T_timestamp: Timestamp, oid.T_timestamptz: TimestampTZ, oid.T_unknown: Unknown, @@ -120,6 +121,7 @@ var oidToArrayOid = map[oid.Oid]oid.Oid{ oid.T_regtype: oid.T__regtype, oid.T_text: oid.T__text, oid.T_time: oid.T__time, + oid.T_timetz: oid.T__timetz, oid.T_timestamp: oid.T__timestamp, oid.T_timestamptz: oid.T__timestamptz, oid.T_uuid: oid.T__uuid, @@ -148,6 +150,7 @@ var familyToOid = map[Family]oid.Oid{ ArrayFamily: oid.T_anyarray, INetFamily: oid.T_inet, TimeFamily: oid.T_time, + TimeTZFamily: oid.T_timetz, JsonFamily: oid.T_jsonb, TupleFamily: oid.T_record, BitFamily: oid.T_bit, diff --git a/pkg/sql/types/types.go b/pkg/sql/types/types.go index 3e47ee230f14..211dfe4908cf 100644 --- a/pkg/sql/types/types.go +++ b/pkg/sql/types/types.go @@ -79,6 +79,7 @@ import ( // | UUID | UUID | T_uuid | 0 | 0 | // | INET | INET | T_inet | 0 | 0 | // | TIME | TIME | T_time | 0 | 0 | +// | TIMETZ | TIMETZ | T_timetz | 0 | 0 | // | JSON | JSONB | T_jsonb | 0 | 0 | // | JSONB | JSONB | T_jsonb | 0 | 0 | // | | | | | | @@ -247,6 +248,14 @@ var ( Time = &T{InternalType: InternalType{ Family: TimeFamily, Oid: oid.T_time, Locale: &emptyLocale}} + // TimeTZ is the type specifying hour, minute, second and timezone with + // no date component. By default, it has microsecond precision. + // For example: + // + // HH:MM:SS.ssssss+-ZZ:ZZ + TimeTZ = &T{InternalType: InternalType{ + Family: TimeTZFamily, Oid: oid.T_timetz, Locale: &emptyLocale}} + // Timestamp is the type of a value specifying year, month, day, hour, minute, // and second, but with no associated timezone. By default, it has microsecond // precision. For example: @@ -310,6 +319,7 @@ var ( Uuid, INet, Time, + TimeTZ, Jsonb, VarBit, } @@ -465,7 +475,7 @@ func MakeScalar(family Family, o oid.Oid, precision, width int32, locale string) panic(errors.AssertionFailedf("negative precision is not allowed")) } switch family { - case DecimalFamily, TimeFamily, TimestampFamily, TimestampTZFamily: + case DecimalFamily, TimeFamily, TimestampFamily, TimestampTZFamily, TimeTZFamily: default: if precision != 0 { panic(errors.AssertionFailedf("type %s cannot have precision", family)) @@ -641,6 +651,19 @@ func MakeTime(precision int32) *T { Family: TimeFamily, Oid: oid.T_time, Precision: precision, Locale: &emptyLocale}} } +// MakeTimeTZ constructs a new instance of a TIME type (oid = T_timetz) that has at +// most the given number of fractional second digits. +func MakeTimeTZ(precision int32) *T { + if precision == 0 { + return TimeTZ + } + if precision != 6 { + panic(errors.AssertionFailedf("precision %d is not currently supported", precision)) + } + return &T{InternalType: InternalType{ + Family: TimeTZFamily, Oid: oid.T_timetz, Precision: precision, Locale: &emptyLocale}} +} + // MakeTimestamp constructs a new instance of a TIMESTAMP type that has at most // the given number of fractional second digits. func MakeTimestamp(precision int32) *T { @@ -885,6 +908,8 @@ func (t *T) Name() string { return "timestamp" case TimestampTZFamily: return "timestamptz" + case TimeTZFamily: + return "timetz" case TupleFamily: // Tuple types are currently anonymous, with no name. return "" @@ -1077,6 +1102,11 @@ func (t *T) SQLStandardNameWithTypmod(haveTypmod bool, typmod int) string { return "time without time zone" } return fmt.Sprintf("time(%d) without time zone", typmod) + case TimeTZFamily: + if !haveTypmod || typmod <= 0 { + return "time with time zone" + } + return fmt.Sprintf("time(%d) with time zone", typmod) case TimestampFamily: if !haveTypmod || typmod <= 0 { return "timestamp without time zone" @@ -1160,7 +1190,7 @@ func (t *T) SQLString() string { case JsonFamily: // Only binary JSON is currently supported. return "JSONB" - case TimeFamily, TimestampFamily, TimestampTZFamily: + case TimeFamily, TimeTZFamily, TimestampFamily, TimestampTZFamily: if t.Precision() > 0 { return fmt.Sprintf("%s(%d)", strings.ToUpper(t.Name()), t.Precision()) } @@ -1730,6 +1760,8 @@ func IsDateTimeType(t *T) bool { return true case TimeFamily: return true + case TimeTZFamily: + return true case TimestampFamily: return true case TimestampTZFamily: diff --git a/pkg/sql/types/types.pb.go b/pkg/sql/types/types.pb.go index 50467bb658a0..d12fd79b5688 100644 --- a/pkg/sql/types/types.pb.go +++ b/pkg/sql/types/types.pb.go @@ -262,6 +262,19 @@ const ( // JSONB // JsonFamily Family = 18 + // TimeTZFamily is the family of date types that store only hour/minute/second + // and timestamp components, with no date component. Seconds can have + // varying precision (defaults to microsecond precision). Currently, only + // microsecond precision is supported. + // + // Canonical: types.TimeTZ + // Oid : T_timetz + // Precision: fractional seconds (3 = ms, 0,6 = us, 9 = ns, etc.) + // + // Examples: + // TIMETZ + // + TimeTZFamily Family = 19 // TupleFamily is a family of non-scalar structural types that describes the // fields of a row or record. The fields can be of any type, including nested // tuple and array types. Fields can also have optional labels. Currently, @@ -325,6 +338,7 @@ var Family_name = map[int32]string{ 16: "INetFamily", 17: "TimeFamily", 18: "JsonFamily", + 19: "TimeTZFamily", 20: "TupleFamily", 21: "BitFamily", 100: "AnyFamily", @@ -348,6 +362,7 @@ var Family_value = map[string]int32{ "INetFamily": 16, "TimeFamily": 17, "JsonFamily": 18, + "TimeTZFamily": 19, "TupleFamily": 20, "BitFamily": 21, "AnyFamily": 100, @@ -370,7 +385,7 @@ func (x *Family) UnmarshalJSON(data []byte) error { return nil } func (Family) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_types_0cfd95d968502097, []int{0} + return fileDescriptor_types_6d5b2964b54f3cfa, []int{0} } // InternalType is the protobuf encoding for SQL types. It is always wrapped by @@ -424,7 +439,7 @@ func (m *InternalType) Reset() { *m = InternalType{} } func (m *InternalType) String() string { return proto.CompactTextString(m) } func (*InternalType) ProtoMessage() {} func (*InternalType) Descriptor() ([]byte, []int) { - return fileDescriptor_types_0cfd95d968502097, []int{0} + return fileDescriptor_types_6d5b2964b54f3cfa, []int{0} } func (m *InternalType) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1071,48 +1086,48 @@ var ( ErrIntOverflowTypes = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/types/types.proto", fileDescriptor_types_0cfd95d968502097) } +func init() { proto.RegisterFile("sql/types/types.proto", fileDescriptor_types_6d5b2964b54f3cfa) } -var fileDescriptor_types_0cfd95d968502097 = []byte{ - // 639 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0xcf, 0x4e, 0xdb, 0x4a, - 0x14, 0xc6, 0x33, 0xf8, 0x0f, 0xf1, 0xc4, 0x49, 0xcc, 0x00, 0xf7, 0xfa, 0xb2, 0x70, 0x7c, 0xb3, +var fileDescriptor_types_6d5b2964b54f3cfa = []byte{ + // 640 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0xc1, 0x4e, 0xdb, 0x4a, + 0x14, 0x86, 0x3d, 0x38, 0x36, 0xf1, 0xc4, 0x49, 0xcc, 0x00, 0xf7, 0xfa, 0xb2, 0x70, 0x7c, 0xb3, 0xc1, 0xf7, 0xaa, 0x72, 0xaa, 0x2e, 0x2b, 0x75, 0x81, 0xa1, 0x54, 0x54, 0x55, 0x91, 0xdc, 0xb0, - 0x61, 0x13, 0x39, 0xf6, 0x34, 0x8c, 0x18, 0x7b, 0x8c, 0x3d, 0x80, 0xb2, 0xec, 0xae, 0xcb, 0x3e, - 0x40, 0x77, 0x7d, 0x99, 0xb0, 0x63, 0x89, 0xba, 0x40, 0x6d, 0x78, 0x91, 0x6a, 0xec, 0x71, 0x4a, - 0x17, 0x95, 0xba, 0x89, 0xf2, 0xfd, 0xce, 0x39, 0xdf, 0xf9, 0x4e, 0x32, 0x70, 0xbb, 0xbc, 0xa0, - 0x23, 0x3e, 0xcf, 0x71, 0x59, 0x7f, 0xfa, 0x79, 0xc1, 0x38, 0x43, 0x7f, 0xc7, 0x2c, 0x3e, 0x2f, - 0x58, 0x14, 0x9f, 0xf9, 0xe5, 0x05, 0xf5, 0x4b, 0x9c, 0xfa, 0x55, 0x79, 0x67, 0x6b, 0xc6, 0x66, - 0xac, 0xea, 0x19, 0x89, 0x6f, 0x75, 0xfb, 0xf0, 0x83, 0x0a, 0xcd, 0xa3, 0x8c, 0xe3, 0x22, 0x8b, + 0x61, 0x13, 0x39, 0xf6, 0x34, 0x8c, 0x18, 0x7b, 0x8c, 0x3d, 0x80, 0xb2, 0xec, 0xae, 0xcb, 0xbe, + 0x43, 0x37, 0x7d, 0x94, 0xb0, 0x63, 0x89, 0xba, 0x40, 0x6d, 0x78, 0x91, 0x6a, 0xec, 0x71, 0x4a, + 0x17, 0x95, 0xba, 0x89, 0xf2, 0x7f, 0xe7, 0x9c, 0xff, 0xfc, 0x27, 0x19, 0xb8, 0x5d, 0x5e, 0xd0, + 0x11, 0x9f, 0xe7, 0xb8, 0xac, 0x3f, 0xfd, 0xbc, 0x60, 0x9c, 0xa1, 0xbf, 0x63, 0x16, 0x9f, 0x17, + 0x2c, 0x8a, 0xcf, 0xfc, 0xf2, 0x82, 0xfa, 0x25, 0x4e, 0xfd, 0xaa, 0xbc, 0xb3, 0x35, 0x63, 0x33, + 0x56, 0xf5, 0x8c, 0xc4, 0xb7, 0xba, 0x7d, 0xf8, 0xa1, 0x05, 0xcd, 0xa3, 0x8c, 0xe3, 0x22, 0x8b, 0xe8, 0x78, 0x9e, 0x63, 0xf4, 0x02, 0xea, 0xef, 0xa3, 0x94, 0xd0, 0xb9, 0x0d, 0x5c, 0xe0, 0xf5, - 0x9e, 0x0d, 0xfc, 0xdf, 0x18, 0xfa, 0x87, 0x55, 0x5b, 0xa0, 0x2e, 0xee, 0x07, 0xad, 0x50, 0x0e, + 0x9e, 0x0d, 0xfc, 0xdf, 0x18, 0xfa, 0x87, 0x55, 0x5b, 0xd0, 0x5a, 0xdc, 0x0f, 0x94, 0x50, 0x0e, 0xa1, 0x1d, 0xa8, 0x5d, 0x93, 0x84, 0x9f, 0xd9, 0x6b, 0x2e, 0xf0, 0x34, 0x59, 0xac, 0x11, 0x1a, - 0x42, 0x23, 0x2f, 0x70, 0x4c, 0x4a, 0xc2, 0x32, 0x5b, 0x79, 0x54, 0xff, 0x89, 0xd1, 0x7f, 0xd0, - 0x8a, 0x8a, 0x22, 0x9a, 0x4f, 0x12, 0x92, 0xe2, 0x4c, 0xa0, 0xd2, 0x56, 0x5d, 0xc5, 0xd3, 0xc2, - 0x7e, 0xc5, 0x0f, 0x56, 0x18, 0xfd, 0x05, 0x75, 0xca, 0xe2, 0x88, 0x62, 0x5b, 0x73, 0x81, 0x67, - 0x84, 0x52, 0xa1, 0x5d, 0x68, 0x5e, 0x91, 0x92, 0x4c, 0x29, 0x9e, 0x88, 0xa0, 0xb6, 0xfe, 0x68, - 0x53, 0x47, 0x56, 0xaa, 0x53, 0x5f, 0xc1, 0xda, 0x73, 0x82, 0x29, 0x4e, 0xeb, 0xde, 0xf5, 0x3f, - 0xba, 0x39, 0xec, 0x56, 0x73, 0x2f, 0x29, 0x4e, 0x2b, 0xa3, 0xa7, 0xb0, 0xc7, 0x2f, 0x73, 0x8a, - 0x27, 0x31, 0xcb, 0x38, 0xce, 0x78, 0x69, 0xb7, 0x5d, 0xc5, 0x33, 0x03, 0x43, 0xec, 0xfc, 0x7a, - 0x3f, 0x00, 0xe3, 0xb0, 0x5b, 0x35, 0xec, 0xcb, 0x3a, 0xfa, 0x17, 0x9a, 0xf5, 0x04, 0x8d, 0xa6, - 0x98, 0x96, 0xb6, 0xe1, 0x2a, 0x9e, 0x11, 0x76, 0x2a, 0xf6, 0xa6, 0x42, 0xe8, 0x39, 0x54, 0x18, - 0x49, 0x6c, 0xe8, 0x02, 0xaf, 0x1b, 0x78, 0xd2, 0xe9, 0x9f, 0x19, 0xe1, 0x67, 0x97, 0x53, 0x3f, - 0x66, 0xe9, 0x88, 0x92, 0xe9, 0x28, 0xbf, 0x18, 0x31, 0x92, 0xf8, 0xc7, 0x24, 0x59, 0xde, 0x0f, - 0x94, 0x63, 0x92, 0x84, 0x62, 0x08, 0x3d, 0x81, 0xbd, 0xfa, 0xb2, 0x55, 0xa0, 0x8e, 0x0b, 0x3c, - 0x33, 0xd0, 0x64, 0x98, 0xaa, 0xd8, 0x84, 0xf9, 0xff, 0xb3, 0x02, 0xf5, 0xfa, 0x30, 0xd4, 0x83, - 0x30, 0x60, 0x8c, 0xd6, 0xca, 0x6a, 0xa1, 0x2e, 0x34, 0x8e, 0x32, 0x2e, 0x25, 0x40, 0x7d, 0xd8, - 0x39, 0xa4, 0x2c, 0x6a, 0xc0, 0x1a, 0xda, 0x80, 0xdd, 0x03, 0x1c, 0x93, 0x34, 0x6a, 0x46, 0x14, - 0x61, 0x71, 0x10, 0x71, 0x2c, 0xb5, 0x8a, 0x36, 0x61, 0x7f, 0x4c, 0x52, 0x5c, 0xf2, 0x28, 0xcd, - 0x25, 0xd4, 0x10, 0x82, 0xbd, 0xea, 0xd5, 0x5d, 0xad, 0x06, 0x75, 0x64, 0x41, 0xf3, 0x1d, 0x2f, - 0x48, 0x36, 0x93, 0x64, 0x5d, 0xac, 0x0b, 0xe6, 0x1c, 0x97, 0x12, 0xb4, 0xd1, 0x36, 0xdc, 0x58, - 0x79, 0x8d, 0x4f, 0x25, 0x36, 0x90, 0x0d, 0xb7, 0xf6, 0x19, 0xa5, 0x11, 0xc7, 0xc9, 0x2f, 0x0e, - 0x50, 0xe4, 0x3f, 0x26, 0x89, 0x94, 0xa6, 0x88, 0x7b, 0x92, 0x9d, 0x67, 0xec, 0x3a, 0x93, 0xa8, - 0x2b, 0xe2, 0x9e, 0x5c, 0xae, 0x5a, 0x7a, 0x62, 0xe7, 0x9e, 0xf8, 0x75, 0x24, 0xe8, 0x8b, 0x86, - 0xa3, 0xb7, 0xb8, 0x39, 0xd9, 0x12, 0x5a, 0x64, 0x90, 0x7a, 0x43, 0xe8, 0xd7, 0x25, 0x6b, 0x0c, - 0x91, 0x30, 0x18, 0x8b, 0xbf, 0x51, 0x82, 0x2d, 0x91, 0x21, 0x20, 0xcd, 0xfc, 0xb6, 0x90, 0x7b, - 0x59, 0x63, 0x9f, 0xec, 0xa8, 0x1f, 0xbf, 0x38, 0xad, 0xa1, 0xda, 0xee, 0x58, 0x9d, 0xa1, 0xda, - 0xde, 0xb4, 0x36, 0x87, 0x7a, 0x7b, 0x01, 0xac, 0x05, 0x18, 0xea, 0xed, 0x1b, 0x60, 0xdd, 0x80, - 0x60, 0x77, 0xf1, 0xdd, 0x69, 0x2d, 0x96, 0x0e, 0xb8, 0x5d, 0x3a, 0xe0, 0x6e, 0xe9, 0x80, 0x6f, - 0x4b, 0x07, 0x7c, 0x7a, 0x70, 0x5a, 0xb7, 0x0f, 0x4e, 0xeb, 0xee, 0xc1, 0x69, 0x9d, 0x6a, 0xd5, - 0xe3, 0xfc, 0x11, 0x00, 0x00, 0xff, 0xff, 0xc5, 0x43, 0xc5, 0xfc, 0x12, 0x04, 0x00, 0x00, + 0x42, 0x23, 0x2f, 0x70, 0x4c, 0x4a, 0xc2, 0x32, 0x5b, 0x7d, 0x54, 0xff, 0x89, 0xd1, 0x7f, 0xd0, + 0x8a, 0x8a, 0x22, 0x9a, 0x4f, 0x12, 0x92, 0xe2, 0x4c, 0xa0, 0xd2, 0x6e, 0xb9, 0xaa, 0xa7, 0x85, + 0xfd, 0x8a, 0x1f, 0xac, 0x30, 0xfa, 0x0b, 0xea, 0x94, 0xc5, 0x11, 0xc5, 0xb6, 0xe6, 0x02, 0xcf, + 0x08, 0xa5, 0x42, 0xbb, 0xd0, 0xbc, 0x22, 0x25, 0x99, 0x52, 0x3c, 0x11, 0x41, 0x6d, 0xfd, 0xd1, + 0xa6, 0x8e, 0xac, 0x54, 0xa7, 0xbe, 0x82, 0xb5, 0xe7, 0x04, 0x53, 0x9c, 0xd6, 0xbd, 0xeb, 0x7f, + 0x74, 0x73, 0xd8, 0xad, 0xe6, 0x5e, 0x52, 0x9c, 0x56, 0x46, 0x4f, 0x61, 0x8f, 0x5f, 0xe6, 0x14, + 0x4f, 0x62, 0x96, 0x71, 0x9c, 0xf1, 0xd2, 0x6e, 0xbb, 0xaa, 0x67, 0x06, 0x86, 0xd8, 0xf9, 0xf5, + 0x7e, 0x00, 0xc6, 0x61, 0xb7, 0x6a, 0xd8, 0x97, 0x75, 0xf4, 0x2f, 0x34, 0xeb, 0x09, 0x1a, 0x4d, + 0x31, 0x2d, 0x6d, 0xc3, 0x55, 0x3d, 0x23, 0xec, 0x54, 0xec, 0x4d, 0x85, 0xd0, 0x73, 0xa8, 0x32, + 0x92, 0xd8, 0xd0, 0x05, 0x5e, 0x37, 0xf0, 0xa4, 0xd3, 0x3f, 0x33, 0xc2, 0xcf, 0x2e, 0xa7, 0x7e, + 0xcc, 0xd2, 0x11, 0x25, 0xd3, 0x51, 0x7e, 0x31, 0x62, 0x24, 0xf1, 0x8f, 0x49, 0xb2, 0xbc, 0x1f, + 0xa8, 0xc7, 0x24, 0x09, 0xc5, 0x10, 0x7a, 0x02, 0x7b, 0xf5, 0x65, 0xab, 0x40, 0x1d, 0x17, 0x78, + 0x66, 0xa0, 0xc9, 0x30, 0x55, 0xb1, 0x09, 0xf3, 0xff, 0x17, 0x15, 0xea, 0xf5, 0x61, 0xa8, 0x07, + 0x61, 0xc0, 0x18, 0xad, 0x95, 0xa5, 0xa0, 0x2e, 0x34, 0x8e, 0x32, 0x2e, 0x25, 0x40, 0x7d, 0xd8, + 0x39, 0xa4, 0x2c, 0x6a, 0xc0, 0x1a, 0xda, 0x80, 0xdd, 0x03, 0x1c, 0x93, 0x34, 0x6a, 0x46, 0x54, + 0x61, 0x71, 0x10, 0x71, 0x2c, 0x75, 0x0b, 0x6d, 0xc2, 0xfe, 0x98, 0xa4, 0xb8, 0xe4, 0x51, 0x9a, + 0x4b, 0xa8, 0x21, 0x04, 0x7b, 0xd5, 0xab, 0xbb, 0x5a, 0x0d, 0xea, 0xc8, 0x82, 0xe6, 0x3b, 0x5e, + 0x90, 0x6c, 0x26, 0xc9, 0xba, 0x58, 0x17, 0xcc, 0x39, 0x2e, 0x25, 0x68, 0xa3, 0x6d, 0xb8, 0xb1, + 0xf2, 0x1a, 0x9f, 0x4a, 0x6c, 0x20, 0x1b, 0x6e, 0xed, 0x33, 0x4a, 0x23, 0x8e, 0x93, 0x5f, 0x1c, + 0xa0, 0xc8, 0x7f, 0x4c, 0x12, 0x29, 0x4d, 0x11, 0xf7, 0x24, 0x3b, 0xcf, 0xd8, 0x75, 0x26, 0x51, + 0x57, 0xc4, 0x3d, 0xb9, 0x5c, 0xb5, 0xf4, 0xc4, 0xce, 0x3d, 0xf1, 0xeb, 0x48, 0xd0, 0x17, 0x0d, + 0x47, 0x6f, 0x71, 0x73, 0xb2, 0x25, 0xb4, 0xc8, 0x20, 0xf5, 0x86, 0xd0, 0xaf, 0x4b, 0xd6, 0x18, + 0x22, 0x71, 0x86, 0xa8, 0xaf, 0xe2, 0x6d, 0x0a, 0xcb, 0xb1, 0xf8, 0x63, 0x25, 0xd8, 0x12, 0xa9, + 0x02, 0xd2, 0x38, 0x6e, 0x0b, 0xb9, 0x97, 0x35, 0x0b, 0x93, 0x9d, 0xd6, 0xc7, 0xcf, 0x8e, 0x32, + 0x6c, 0xb5, 0x3b, 0x56, 0x67, 0xa8, 0xb7, 0x17, 0xc0, 0x5a, 0x80, 0xa1, 0xde, 0xbe, 0x01, 0xd6, + 0x0d, 0x08, 0x76, 0x17, 0xdf, 0x1d, 0x65, 0xb1, 0x74, 0xc0, 0xed, 0xd2, 0x01, 0x77, 0x4b, 0x07, + 0x7c, 0x5b, 0x3a, 0xe0, 0xd3, 0x83, 0xa3, 0xdc, 0x3e, 0x38, 0xca, 0xdd, 0x83, 0xa3, 0x9c, 0x6a, + 0xd5, 0x43, 0xfd, 0x11, 0x00, 0x00, 0xff, 0xff, 0x1e, 0xdb, 0x25, 0xf4, 0x1e, 0x04, 0x00, 0x00, } diff --git a/pkg/sql/types/types.proto b/pkg/sql/types/types.proto index fa86bb21f302..d65d9a15911d 100644 --- a/pkg/sql/types/types.proto +++ b/pkg/sql/types/types.proto @@ -276,8 +276,19 @@ enum Family { // JsonFamily = 18; - // Reserved for TIMETZ if/when fully implemented. See #26097. - reserved 19; + // TimeTZFamily is the family of date types that store only hour/minute/second + // and timestamp components, with no date component. Seconds can have + // varying precision (defaults to microsecond precision). Currently, only + // microsecond precision is supported. + // + // Canonical: types.TimeTZ + // Oid : T_timetz + // Precision: fractional seconds (3 = ms, 0,6 = us, 9 = ns, etc.) + // + // Examples: + // TIMETZ + // + TimeTZFamily = 19; // TupleFamily is a family of non-scalar structural types that describes the // fields of a row or record. The fields can be of any type, including nested diff --git a/pkg/sql/types/types_test.go b/pkg/sql/types/types_test.go index 6fb695cca153..dbfeb4114299 100644 --- a/pkg/sql/types/types_test.go +++ b/pkg/sql/types/types_test.go @@ -221,6 +221,14 @@ func TestTypes(t *testing.T) { Family: TimeFamily, Oid: oid.T_time, Precision: 6, Locale: &emptyLocale}}}, {MakeTime(6), MakeScalar(TimeFamily, oid.T_time, 6, 0, emptyLocale)}, + // TIMETZ + {MakeTimeTZ(0), TimeTZ}, + {MakeTimeTZ(0), &T{InternalType: InternalType{ + Family: TimeTZFamily, Oid: oid.T_timetz, Locale: &emptyLocale}}}, + {MakeTimeTZ(6), &T{InternalType: InternalType{ + Family: TimeTZFamily, Oid: oid.T_timetz, Precision: 6, Locale: &emptyLocale}}}, + {MakeTimeTZ(6), MakeScalar(TimeTZFamily, oid.T_timetz, 6, 0, emptyLocale)}, + // TIMESTAMP {MakeTimestamp(0), Timestamp}, {MakeTimestamp(0), &T{InternalType: InternalType{ diff --git a/pkg/util/encoding/encoding.go b/pkg/util/encoding/encoding.go index 3251c6f4895b..2eeb333e5edd 100644 --- a/pkg/util/encoding/encoding.go +++ b/pkg/util/encoding/encoding.go @@ -28,6 +28,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/bitarray" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/ipaddr" + "github.com/cockroachdb/cockroach/pkg/util/timeofday" + "github.com/cockroachdb/cockroach/pkg/util/timetz" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/pkg/errors" @@ -77,6 +79,8 @@ const ( bitArrayDataTerminator = 0x00 bitArrayDataDescTerminator = 0xff + timeTZMarker = bitArrayDescMarker + 1 + // IntMin is chosen such that the range of int tags does not overlap the // ascii character set that is frequently used in testing. IntMin = 0x80 // 128 @@ -100,12 +104,19 @@ const ( // without table descriptors. interleavedSentinel = 0xfe encodedNullDesc = 0xff + + // offsetSecsToMicros is a constant that allows conversion from seconds + // to microseconds for offsetSecs type calculations (e.g. for TimeTZ). + offsetSecsToMicros = 1000000 ) const ( // EncodedDurationMaxLen is the largest number of bytes used when encoding a // Duration. EncodedDurationMaxLen = 1 + 3*binary.MaxVarintLen64 // 3 varints are encoded. + // EncodedTimeTZMaxLen is the largest number of bytes used when encoding a + // TimeTZ. + EncodedTimeTZMaxLen = 1 + binary.MaxVarintLen64 + binary.MaxVarintLen32 ) // Direction for ordering results. @@ -881,7 +892,7 @@ func DecodeIfInterleavedSentinel(b []byte) ([]byte, bool) { // EncodeTimeAscending encodes a time value, appends it to the supplied buffer, // and returns the final buffer. The encoding is guaranteed to be ordered // Such that if t1.Before(t2) then after EncodeTime(b1, t1), and -// EncodeTime(b2, t1), Compare(b1, b2) < 0. The time zone offset not +// EncodeTime(b2, t2), Compare(b1, b2) < 0. The time zone offset not // included in the encoding. func EncodeTimeAscending(b []byte, t time.Time) []byte { return encodeTime(b, t.Unix(), int64(t.Nanosecond())) @@ -937,6 +948,77 @@ func decodeTime(b []byte) (r []byte, sec int64, nsec int64, err error) { return b, sec, nsec, nil } +// EncodeTimeTZAscending encodes a timetz.TimeTZ value and appends it to +// the supplied buffer and returns the final buffer. +// The encoding is guaranteed to be ordered such that if t1.Before(t2) +// then after encodeTimeTZ(b1, t1) and encodeTimeTZ(b2, t2), +// Compare(b1, b2) < 0. +// The time zone offset is included in the encoding. +func EncodeTimeTZAscending(b []byte, t timetz.TimeTZ) []byte { + // Do not use TimeOfDay's add function, as it loses 24:00:00 encoding. + return encodeTimeTZ(b, int64(t.TimeOfDay)+int64(t.OffsetSecs)*offsetSecsToMicros, t.OffsetSecs) +} + +// EncodeTimeTZDescending is the descending version of EncodeTimeTZAscending. +func EncodeTimeTZDescending(b []byte, t timetz.TimeTZ) []byte { + // Do not use TimeOfDay's add function, as it loses 24:00:00 encoding. + return encodeTimeTZ(b, ^(int64(t.TimeOfDay) + int64(t.OffsetSecs)*offsetSecsToMicros), ^t.OffsetSecs) +} + +func encodeTimeTZ(b []byte, unixMicros int64, offsetSecs int32) []byte { + b = append(b, timeTZMarker) + b = EncodeVarintAscending(b, unixMicros) + b = EncodeVarintAscending(b, int64(offsetSecs)) + return b +} + +// DecodeTimeTZAscending decodes a timetz.TimeTZ value which was encoded +// using encodeTimeTZ. The remainder of the input buffer and the decoded +// timetz.TimeTZ are returned. +func DecodeTimeTZAscending(b []byte) ([]byte, timetz.TimeTZ, error) { + b, unixMicros, offsetSecs, err := decodeTimeTZ(b) + if err != nil { + return nil, timetz.TimeTZ{}, err + } + // Do not use timeofday.FromInt, as it loses 24:00:00 encoding. + return b, timetz.TimeTZ{ + TimeOfDay: timeofday.TimeOfDay(unixMicros - int64(offsetSecs)*offsetSecsToMicros), + OffsetSecs: offsetSecs, + }, nil +} + +// DecodeTimeTZDescending is the descending version of DecodeTimeTZAscending. +func DecodeTimeTZDescending(b []byte) ([]byte, timetz.TimeTZ, error) { + b, unixMicros, offsetSecs, err := decodeTimeTZ(b) + if err != nil { + return nil, timetz.TimeTZ{}, err + } + // Do not use timeofday.FromInt, as it loses 24:00:00 encoding. + return b, timetz.TimeTZ{ + TimeOfDay: timeofday.TimeOfDay(^unixMicros - int64(^offsetSecs)*offsetSecsToMicros), + OffsetSecs: ^offsetSecs, + }, nil +} + +func decodeTimeTZ(b []byte) ([]byte, int64, int32, error) { + if PeekType(b) != TimeTZ { + return nil, 0, 0, errors.Errorf("did not find marker") + } + b = b[1:] + var err error + var unixMicros int64 + b, unixMicros, err = DecodeVarintAscending(b) + if err != nil { + return nil, 0, 0, err + } + var offsetSecs int64 + b, offsetSecs, err = DecodeVarintAscending(b) + if err != nil { + return nil, 0, 0, err + } + return b, unixMicros, int32(offsetSecs), nil +} + // EncodeDurationAscending encodes a duration.Duration value, appends it to the // supplied buffer, and returns the final buffer. The encoding is guaranteed to // be ordered such that if t1.Compare(t2) < 0 (or = 0 or > 0) then bytes.Compare @@ -1191,6 +1273,7 @@ const ( Tuple Type = 16 BitArray Type = 17 BitArrayDesc Type = 18 // BitArray encoded descendingly + TimeTZ Type = 19 ) // typMap maps an encoded type byte to a decoded Type. It's got 256 slots, one @@ -1233,6 +1316,8 @@ func slowPeekType(b []byte) Type { return BitArrayDesc case m == timeMarker: return Time + case m == timeTZMarker: + return TimeTZ case m == byte(Array): return Array case m == byte(True): @@ -1314,7 +1399,7 @@ func PeekLength(b []byte) (int, error) { return getJSONInvertedIndexKeyLength(b) case bytesDescMarker: return getBytesLength(b, descendingEscapes) - case timeMarker: + case timeMarker, timeTZMarker: return GetMultiVarintLen(b, 2) case durationBigNegMarker, durationMarker, durationBigPosMarker: return GetMultiVarintLen(b, 3) @@ -1520,6 +1605,17 @@ func prettyPrintFirstValue(dir Direction, b []byte) ([]byte, string, error) { return b, "", err } return b, t.UTC().Format(time.RFC3339Nano), nil + case TimeTZ: + var t timetz.TimeTZ + if dir == Descending { + b, t, err = DecodeTimeTZDescending(b) + } else { + b, t, err = DecodeTimeTZAscending(b) + } + if err != nil { + return b, "", err + } + return b, t.String(), nil case Duration: var d duration.Duration if dir == Descending { @@ -1810,6 +1906,20 @@ func EncodeUntaggedTimeValue(appendTo []byte, t time.Time) []byte { return EncodeNonsortingStdlibVarint(appendTo, int64(t.Nanosecond())) } +// EncodeTimeTZValue encodes a timetz.TimeTZ value with its value tag, appends it to +// the supplied buffer, and returns the final buffer. +func EncodeTimeTZValue(appendTo []byte, colID uint32, t timetz.TimeTZ) []byte { + appendTo = EncodeValueTag(appendTo, colID, TimeTZ) + return EncodeUntaggedTimeTZValue(appendTo, t) +} + +// EncodeUntaggedTimeTZValue encodes a time.Time value, appends it to the supplied buffer, +// and returns the final buffer. +func EncodeUntaggedTimeTZValue(appendTo []byte, t timetz.TimeTZ) []byte { + appendTo = EncodeNonsortingStdlibVarint(appendTo, int64(t.TimeOfDay)) + return EncodeNonsortingStdlibVarint(appendTo, int64(t.OffsetSecs)) +} + // EncodeDecimalValue encodes an apd.Decimal value with its value tag, appends // it to the supplied buffer, and returns the final buffer. func EncodeDecimalValue(appendTo []byte, colID uint32, d *apd.Decimal) []byte { @@ -2043,6 +2153,31 @@ func DecodeUntaggedTimeValue(b []byte) (remaining []byte, t time.Time, err error return b, timeutil.Unix(sec, nsec), nil } +// DecodeTimeTZValue decodes a value encoded by EncodeTimeTZValue. +func DecodeTimeTZValue(b []byte) (remaining []byte, t timetz.TimeTZ, err error) { + b, err = decodeValueTypeAssert(b, TimeTZ) + if err != nil { + return b, timetz.TimeTZ{}, err + } + return DecodeUntaggedTimeTZValue(b) +} + +// DecodeUntaggedTimeTZValue decodes a value encoded by EncodeUntaggedTimeTZValue. +func DecodeUntaggedTimeTZValue(b []byte) (remaining []byte, t timetz.TimeTZ, err error) { + var timeOfDayMicros int64 + b, _, timeOfDayMicros, err = DecodeNonsortingStdlibVarint(b) + if err != nil { + return b, timetz.TimeTZ{}, err + } + var offsetSecs int64 + b, _, offsetSecs, err = DecodeNonsortingStdlibVarint(b) + if err != nil { + return b, timetz.TimeTZ{}, err + } + // Do not use timeofday.FromInt as it truncates 24:00 into 00:00. + return b, timetz.MakeTimeTZ(timeofday.TimeOfDay(timeOfDayMicros), int32(offsetSecs)), nil +} + // DecodeDecimalValue decodes a value encoded by EncodeDecimalValue. func DecodeDecimalValue(b []byte) (remaining []byte, d apd.Decimal, err error) { b, err = decodeValueTypeAssert(b, Decimal) @@ -2248,7 +2383,7 @@ func PeekValueLengthWithOffsetsAndType(b []byte, dataOffset int, typ Type) (leng case Decimal: _, n, i, err := DecodeNonsortingStdlibUvarint(b) return dataOffset + n + int(i), err - case Time: + case Time, TimeTZ: n, err := getMultiNonsortingVarintLen(b, 2) return dataOffset + n, err case Duration: @@ -2339,6 +2474,13 @@ func PrettyPrintValueEncoded(b []byte) ([]byte, string, error) { return b, "", err } return b, t.UTC().Format(time.RFC3339Nano), nil + case TimeTZ: + var t timetz.TimeTZ + b, t, err = DecodeTimeTZValue(b) + if err != nil { + return b, "", err + } + return b, t.String(), nil case Duration: var d duration.Duration b, d, err = DecodeDurationValue(b) diff --git a/pkg/util/encoding/encoding_test.go b/pkg/util/encoding/encoding_test.go index d91e491be89a..404ce039e71a 100644 --- a/pkg/util/encoding/encoding_test.go +++ b/pkg/util/encoding/encoding_test.go @@ -24,9 +24,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/cockroachdb/cockroach/pkg/util/timeofday" + "github.com/cockroachdb/cockroach/pkg/util/timetz" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/pkg/errors" + "github.com/stretchr/testify/assert" ) func testBasicEncodeDecode32( @@ -1056,6 +1059,89 @@ func TestEncodeDecodeTime(t *testing.T) { } } } +func TestEncodeDecodeTimeTZ(t *testing.T) { + // Test cases are in ascending order for TimeTZ, which means: + // * UTC timestamp first preference + // * Negative Zone Offset second preference + // + // Tests these UTC times: + // 05:06:07 + // 10:11:12 + // 15:16:17 + // 20:21:22 + // Over time zones (if not overflowing): + // -12 + // -08 + // -04 + // 0 + // +04 + // +08 + // +12 + testCases := []string{ + "00:00:00+1559", // minimum + + "17:06:07+12", + "13:06:07+8", + "09:06:07+4", + "05:06:07+0", + "01:06:07-4", + + "22:11:12+12", + "18:11:12+8", + "14:11:12+4", + "10:41:12+0030", // special check of .5 hour offsets + "10:11:12+0", + "06:11:12-4", + "04:11:12-8", + + "23:16:17+8", + "19:16:17+4", + "15:16:17+0", + "11:16:17-4", + "09:16:17-8", + "05:16:17-12", + + "20:21:22+0", + "16:21:22-4", + "12:21:22-8", + "08:21:22-12", + + "24:00:00-1559", // maximum + } + + var lastEncoded []byte + for _, dir := range []Direction{Ascending, Descending} { + t.Run(fmt.Sprintf("dir:%d", dir), func(t *testing.T) { + for i := range testCases { + t.Run(fmt.Sprintf("tc:%d", i), func(t *testing.T) { + current, err := timetz.ParseTimeTZ(timeutil.Now(), testCases[i]) + assert.NoError(t, err) + + var b []byte + var decodedCurrent timetz.TimeTZ + if dir == Ascending { + b = EncodeTimeTZAscending(b, current) + _, decodedCurrent, err = DecodeTimeTZAscending(b) + } else { + b = EncodeTimeTZDescending(b, current) + _, decodedCurrent, err = DecodeTimeTZDescending(b) + } + assert.NoError(t, err) + assert.Equal(t, current, decodedCurrent) + testPeekLength(t, b) + if i > 0 { + if dir == Ascending { + assert.True(t, bytes.Compare(lastEncoded, b) < 0, "encodings %s, %s not increasing", testCases[i-1], testCases[i]) + } else { + assert.True(t, bytes.Compare(lastEncoded, b) > 0, "encodings %s, %s not decreasing", testCases[i-1], testCases[i]) + } + } + lastEncoded = b + }) + } + }) + } +} type testCaseDuration struct { value duration.Duration @@ -1154,6 +1240,8 @@ func TestPeekType(t *testing.T) { {EncodeBytesDescending(nil, []byte("")), BytesDesc}, {EncodeTimeAscending(nil, timeutil.Now()), Time}, {EncodeTimeDescending(nil, timeutil.Now()), Time}, + {EncodeTimeTZAscending(nil, timetz.Now()), TimeTZ}, + {EncodeTimeTZDescending(nil, timetz.Now()), TimeTZ}, {encodedDurationAscending, Duration}, {encodedDurationDescending, Duration}, {EncodeBitArrayAscending(nil, bitarray.BitArray{}), BitArray}, @@ -1194,6 +1282,13 @@ func (rd randData) time() time.Time { return timeutil.Unix(rd.Int63n(1000000), rd.Int63n(1000000)) } +func (rd randData) timetz() timetz.TimeTZ { + return timetz.MakeTimeTZ( + timeofday.FromInt(rd.Int63n(int64(timeofday.Max))), + rd.Int31n(timetz.MaxTimeTZOffsetSecs*2)-timetz.MaxTimeTZOffsetSecs, + ) +} + func (rd randData) bitArray() bitarray.BitArray { return bitarray.Rand(rd.Rand, uint(rd.Int31n(140))) } @@ -1676,6 +1771,25 @@ func TestValueEncodeDecodeTime(t *testing.T) { } } +func TestValueEncodeDecodeTimeTZ(t *testing.T) { + rng, seed := randutil.NewPseudoRand() + rd := randData{rng} + tests := make([]timetz.TimeTZ, 1000) + for i := range tests { + tests[i] = rd.timetz() + } + for _, test := range tests { + buf := EncodeTimeTZValue(nil, NoColumnID, test) + _, x, err := DecodeTimeTZValue(buf) + if err != nil { + t.Fatal(err) + } + if x != test { + t.Errorf("seed %d: expected %v got %v", seed, test, x) + } + } +} + func TestValueEncodeDecodeBitArray(t *testing.T) { rng, seed := randutil.NewPseudoRand() rd := randData{rng} @@ -1910,6 +2024,9 @@ func randValueEncode(rd randData, buf []byte, colID uint32, typ Type) ([]byte, i case Time: x := rd.time() return EncodeTimeValue(buf, colID, x), x, true + case TimeTZ: + x := rd.timetz() + return EncodeTimeTZValue(buf, colID, x), x, true case Duration: x := rd.duration() return EncodeDurationValue(buf, colID, x), x, true @@ -2066,6 +2183,8 @@ func TestValueEncodingRand(t *testing.T) { buf, decoded, err = DecodeBytesValue(buf) case Time: buf, decoded, err = DecodeTimeValue(buf) + case TimeTZ: + buf, decoded, err = DecodeTimeTZValue(buf) case Duration: buf, decoded, err = DecodeDurationValue(buf) case BitArray: @@ -2135,6 +2254,8 @@ func TestPrettyPrintValueEncoded(t *testing.T) { {EncodeDecimalValue(nil, NoColumnID, apd.New(628, -2)), "6.28"}, {EncodeTimeValue(nil, NoColumnID, time.Date(2016, 6, 29, 16, 2, 50, 5, time.UTC)), "2016-06-29T16:02:50.000000005Z"}, + {EncodeTimeTZValue(nil, NoColumnID, + timetz.MakeTimeTZ(timeofday.New(10, 11, 12, 0), 5*60*60+24)), "10:11:12-05:00:24"}, {EncodeDurationValue(nil, NoColumnID, duration.DecodeDuration(1, 2, 3)), "1 mon 2 days 00:00:00+3ns"}, {EncodeBytesValue(nil, NoColumnID, []byte{0x1, 0x2, 0xF, 0xFF}), "0x01020fff"}, @@ -2322,6 +2443,40 @@ func BenchmarkDecodeTimeValue(b *testing.B) { } } +func BenchmarkEncodeTimeTZValue(b *testing.B) { + rng, _ := randutil.NewPseudoRand() + rd := randData{rng} + + vals := make([]timetz.TimeTZ, 10000) + for i := range vals { + vals[i] = rd.timetz() + } + + buf := make([]byte, 0, 1000) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + _ = EncodeTimeTZValue(buf, NoColumnID, vals[i%len(vals)]) + } +} + +func BenchmarkDecodeTimeTZValue(b *testing.B) { + rng, _ := randutil.NewPseudoRand() + rd := randData{rng} + + vals := make([][]byte, 10000) + for i := range vals { + vals[i] = EncodeTimeTZValue(nil, uint32(rng.Intn(100)), rd.timetz()) + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + if _, _, err := DecodeTimeTZValue(vals[i%len(vals)]); err != nil { + b.Fatal(err) + } + } +} + func BenchmarkEncodeIPAddrValue(b *testing.B) { rng, _ := randutil.NewPseudoRand() rd := randData{rng} diff --git a/pkg/util/encoding/type_string.go b/pkg/util/encoding/type_string.go index 9787520c057e..d977c12a7a3a 100644 --- a/pkg/util/encoding/type_string.go +++ b/pkg/util/encoding/type_string.go @@ -27,11 +27,12 @@ func _() { _ = x[Tuple-16] _ = x[BitArray-17] _ = x[BitArrayDesc-18] + _ = x[TimeTZ-19] } -const _Type_name = "UnknownNullNotNullIntFloatDecimalBytesBytesDescTimeDurationTrueFalseUUIDArrayIPAddrJSONTupleBitArrayBitArrayDesc" +const _Type_name = "UnknownNullNotNullIntFloatDecimalBytesBytesDescTimeDurationTrueFalseUUIDArrayIPAddrJSONTupleBitArrayBitArrayDescTimeTZ" -var _Type_index = [...]uint8{0, 7, 11, 18, 21, 26, 33, 38, 47, 51, 59, 63, 68, 72, 77, 83, 87, 92, 100, 112} +var _Type_index = [...]uint8{0, 7, 11, 18, 21, 26, 33, 38, 47, 51, 59, 63, 68, 72, 77, 83, 87, 92, 100, 112, 118} func (i Type) String() string { if i < 0 || i >= Type(len(_Type_index)-1) { diff --git a/pkg/util/timetz/timetz.go b/pkg/util/timetz/timetz.go new file mode 100644 index 000000000000..ef3075134a0e --- /dev/null +++ b/pkg/util/timetz/timetz.go @@ -0,0 +1,159 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package timetz + +import ( + "fmt" + "regexp" + "time" + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/util/timeofday" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" +) + +var ( + // MaxTimeTZOffsetSecs is the maximum offset TimeTZ allows in seconds. + // NOTE: postgres documentation mentions 14:59, but up to 15:59 is accepted. + MaxTimeTZOffsetSecs = int32((15*time.Hour + 59*time.Minute) / time.Second) + // MinTimeTZOffsetSecs is the minimum offset TimeTZ allows in seconds. + // NOTE: postgres documentation mentions -14:59, but up to -15:59 is accepted. + MinTimeTZOffsetSecs = -1 * MaxTimeTZOffsetSecs + + // TimeTZMaxTimeRegex is a compiled regex for parsing the 24:00 timetz value. + timeTZMaxTimeRegex = regexp.MustCompile(`^24:`) +) + +// TimeTZ is an implementation of postgres' TimeTZ. +// Note that in this implementation, if time is equal in terms of UTC time +// the zone offset is further used to differentiate. +type TimeTZ struct { + // TimeOfDay is the time since midnight in a given zone + // dictated by OffsetSecs. + timeofday.TimeOfDay + // OffsetSecs is the offset of the zone, with the sign reversed. + // e.g. -0800 (PDT) would have OffsetSecs of +8*60*60. + // This is in line with the postgres implementation. + // This means timeofday.Secs() + OffsetSecs = UTC secs. + OffsetSecs int32 +} + +// MakeTimeTZ creates a TimeTZ from a TimeOfDay and offset. +func MakeTimeTZ(t timeofday.TimeOfDay, offsetSecs int32) TimeTZ { + return TimeTZ{TimeOfDay: t, OffsetSecs: offsetSecs} +} + +// MakeTimeTZFromLocation creates a TimeTZ from a TimeOfDay and time.Location. +func MakeTimeTZFromLocation(t timeofday.TimeOfDay, loc *time.Location) TimeTZ { + _, zoneOffsetSecs := timeutil.Unix(0, 0).In(loc).Zone() + return TimeTZ{TimeOfDay: t, OffsetSecs: -int32(zoneOffsetSecs)} +} + +// MakeTimeTZFromTime creates a TimeTZ from a time.Time. +// It will be trimmed to microsecond precision. +func MakeTimeTZFromTime(t time.Time) TimeTZ { + return MakeTimeTZFromLocation( + timeofday.New(t.Hour(), t.Minute(), t.Second(), t.Nanosecond()/1000), + t.Location(), + ) +} + +// Now returns the TimeTZ of the current location. +func Now() TimeTZ { + return MakeTimeTZFromTime(timeutil.Now()) +} + +// ParseTimeTZ parses and returns the TimeTZ represented by the +// provided string, or an error if parsing is unsuccessful. +func ParseTimeTZ(now time.Time, s string) (TimeTZ, error) { + // Special case as we have to use `ParseTimestamp` to get the date. + // We cannot use `ParseTime` as it does not have timezone awareness. + if s == "" { + return TimeTZ{}, pgerror.Newf( + pgcode.InvalidTextRepresentation, + "unable to parse %q as TimeTZ", + s, + ) + } + t, err := pgdate.ParseTimestamp(now, pgdate.ParseModeYMD, "1970-01-01 "+s) + if err != nil { + // Build our own error message to avoid exposing the dummy date. + return TimeTZ{}, pgerror.Wrapf( + err, + pgcode.InvalidTextRepresentation, + "unable to parse %q as TimeTZ", + s, + ) + } + retTime := timeofday.FromTime(t) + // Special case on 24:00 and 24:00:00 as the parser + // does not handle these correctly. + if timeTZMaxTimeRegex.MatchString(s) { + retTime = timeofday.Time2400 + } + + _, offsetSecsUnconverted := t.Zone() + offsetSecs := int32(-offsetSecsUnconverted) + if offsetSecs > MaxTimeTZOffsetSecs || offsetSecs < MinTimeTZOffsetSecs { + return TimeTZ{}, pgerror.Newf( + pgcode.NumericValueOutOfRange, + "time zone displacement out of range: %q", + s, + ) + } + return MakeTimeTZ(retTime, offsetSecs), nil +} + +// String implements the Stringer interface. +func (t *TimeTZ) String() string { + tTime := t.ToTime() + timeComponent := tTime.Format("15:04:05.999999") + // 24:00:00 gets returned as 00:00:00, which is incorrect. + if t.TimeOfDay == timeofday.Time2400 { + timeComponent = "24:00:00" + } + timeZoneComponent := tTime.Format("Z07:00:00") + // If it is UTC, .Format converts it to "Z". + // Fully expand this component. + if t.OffsetSecs == 0 { + timeZoneComponent = "+00:00:00" + } + // Go's time.Format functionality does not work for offsets which + // in the range -0s < offsetSecs < -60s, e.g. -22s offset prints as 00:00:-22. + // Manually correct for this. + if 0 < t.OffsetSecs && t.OffsetSecs < 60 { + timeZoneComponent = fmt.Sprintf("-00:00:%02d", t.OffsetSecs) + } + return timeComponent + timeZoneComponent +} + +// ToTime converts a DTimeTZ to a time.Time, corrected to the given location. +func (t *TimeTZ) ToTime() time.Time { + loc := timeutil.FixedOffsetTimeZoneToLocation(-int(t.OffsetSecs), "TimeTZ") + return t.TimeOfDay.ToTime().Add(time.Duration(t.OffsetSecs) * time.Second).In(loc) +} + +// Before returns whether the current is before the other TimeTZ. +func (t *TimeTZ) Before(other TimeTZ) bool { + return t.ToTime().Before(other.ToTime()) || (t.ToTime().Equal(other.ToTime()) && t.OffsetSecs < other.OffsetSecs) +} + +// After returns whether the TimeTZ is after the other TimeTZ. +func (t *TimeTZ) After(other TimeTZ) bool { + return t.ToTime().After(other.ToTime()) || (t.ToTime().Equal(other.ToTime()) && t.OffsetSecs > other.OffsetSecs) +} + +// Equal returns whether the TimeTZ is equal to the other TimeTZ. +func (t *TimeTZ) Equal(other TimeTZ) bool { + return t.TimeOfDay == other.TimeOfDay && t.OffsetSecs == other.OffsetSecs +} diff --git a/pkg/util/timetz/timetz_test.go b/pkg/util/timetz/timetz_test.go new file mode 100644 index 000000000000..d6cfd472c867 --- /dev/null +++ b/pkg/util/timetz/timetz_test.go @@ -0,0 +1,195 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package timetz + +import ( + "fmt" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/util/timeofday" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestParseTimeTZToStringRoundTrip(t *testing.T) { + testCases := []string{ + "24:00:00-1559", + "11:12:13+05:06:07", + "10:11:12+0", + "10:11:12.05+0", + } + for _, tc := range testCases { + t.Run(tc, func(t *testing.T) { + exampleTime, err := ParseTimeTZ(timeutil.Now(), tc) + assert.NoError(t, err) + + exampleTimeFromString, err := ParseTimeTZ(timeutil.Now(), exampleTime.String()) + assert.NoError(t, err) + + assert.True(t, exampleTime.Equal(exampleTimeFromString)) + }) + } +} + +func TestTimeTZString(t *testing.T) { + testCases := []struct { + input TimeTZ + expected string + }{ + {MakeTimeTZ(timeofday.New(0, 0, 0, 0), 0), "00:00:00+00:00:00"}, + {MakeTimeTZ(timeofday.New(10, 11, 12, 0), 0), "10:11:12+00:00:00"}, + {MakeTimeTZ(timeofday.New(10, 11, 12, 0), -30), "10:11:12+00:00:30"}, + {MakeTimeTZ(timeofday.New(10, 11, 12, 0), 30), "10:11:12-00:00:30"}, + {MakeTimeTZ(timeofday.New(10, 11, 12, 0), 120), "10:11:12-00:02:00"}, + {MakeTimeTZ(timeofday.New(10, 11, 12, 0), 3), "10:11:12-00:00:03"}, + {MakeTimeTZ(timeofday.Max, -10*60*60), "23:59:59.999999+10:00:00"}, + {MakeTimeTZ(timeofday.Time2400, 10*60*60), "24:00:00-10:00:00"}, + } + for i, tc := range testCases { + t.Run(fmt.Sprintf("%d:%s", i, tc.expected), func(t *testing.T) { + assert.Equal(t, tc.expected, tc.input.String()) + }) + } +} + +func TestTimeTZ(t *testing.T) { + maxTime, err := ParseTimeTZ(timeutil.Now(), "24:00:00-1559") + require.NoError(t, err) + minTime, err := ParseTimeTZ(timeutil.Now(), "00:00:00+1559") + require.NoError(t, err) + + // These are all the same UTC time equivalents. + utcTime, err := ParseTimeTZ(timeutil.Now(), "11:14:15+0") + require.NoError(t, err) + sydneyTime, err := ParseTimeTZ(timeutil.Now(), "21:14:15+10") + require.NoError(t, err) + + // No daylight savings in Hawaii! + hawaiiZone, err := time.LoadLocation("Pacific/Honolulu") + require.NoError(t, err) + hawaiiTime := MakeTimeTZFromLocation(timeofday.New(1, 14, 15, 0), hawaiiZone) + + weirdTimeZone := MakeTimeTZ(timeofday.New(10, 0, 0, 0), -((5 * 60 * 60) + 30*60 + 15)) + + testCases := []struct { + t TimeTZ + toTime time.Time + largerThan []TimeTZ + smallerThan []TimeTZ + equalTo []TimeTZ + }{ + { + t: weirdTimeZone, + toTime: time.Date(1970, 1, 1, 10, 0, 0, 0, timeutil.FixedOffsetTimeZoneToLocation((5*60*60)+(30*60)+15, "TimeTZ")), + largerThan: []TimeTZ{minTime}, + smallerThan: []TimeTZ{maxTime}, + equalTo: []TimeTZ{weirdTimeZone}, + }, + { + t: utcTime, + toTime: time.Date(1970, 1, 1, 11, 14, 15, 0, timeutil.FixedOffsetTimeZoneToLocation(0, "TimeTZ")), + largerThan: []TimeTZ{minTime, sydneyTime}, + smallerThan: []TimeTZ{maxTime, hawaiiTime}, + equalTo: []TimeTZ{utcTime}, + }, + { + t: sydneyTime, + toTime: time.Date(1970, 1, 1, 21, 14, 15, 0, timeutil.FixedOffsetTimeZoneToLocation(10*60*60, "TimeTZ")), + largerThan: []TimeTZ{minTime}, + smallerThan: []TimeTZ{maxTime, utcTime, hawaiiTime}, + equalTo: []TimeTZ{sydneyTime}, + }, + { + t: hawaiiTime, + toTime: time.Date(1970, 1, 1, 1, 14, 15, 0, timeutil.FixedOffsetTimeZoneToLocation(-10*60*60, "TimeTZ")), + largerThan: []TimeTZ{minTime, utcTime, sydneyTime}, + smallerThan: []TimeTZ{maxTime}, + equalTo: []TimeTZ{hawaiiTime}, + }, + { + t: minTime, + toTime: time.Date(1970, 1, 1, 0, 0, 0, 0, timeutil.FixedOffsetTimeZoneToLocation(15*60*60+59*60, "TimeTZ")), + largerThan: []TimeTZ{}, + smallerThan: []TimeTZ{maxTime, utcTime, sydneyTime, hawaiiTime}, + equalTo: []TimeTZ{minTime}, + }, + { + t: maxTime, + toTime: time.Date(1970, 1, 2, 0, 0, 0, 0, timeutil.FixedOffsetTimeZoneToLocation(-(15*60*60+59*60), "TimeTZ")), + largerThan: []TimeTZ{minTime, utcTime, sydneyTime, hawaiiTime}, + smallerThan: []TimeTZ{}, + equalTo: []TimeTZ{maxTime}, + }, + } + for i, tc := range testCases { + t.Run(fmt.Sprintf("#%d %s", i, tc.t.String()), func(t *testing.T) { + assert.Equal(t, tc.toTime, tc.t.ToTime()) + + for _, largerThan := range tc.largerThan { + assert.True(t, tc.t.After(largerThan), "%s > %s", tc.t.String(), largerThan) + } + + for _, smallerThan := range tc.smallerThan { + assert.True(t, tc.t.Before(smallerThan), "%s < %s", tc.t.String(), smallerThan) + } + + for _, equalTo := range tc.equalTo { + assert.True(t, tc.t.Equal(equalTo), "%s = %s", tc.t.String(), equalTo) + } + }) + } +} + +func TestParseTimeTZ(t *testing.T) { + testCases := []struct { + str string + expected TimeTZ + expectedError bool + }{ + {str: "01:02:03", expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), 0)}, + {str: "01:02:03.000123", expected: MakeTimeTZ(timeofday.New(1, 2, 3, 123), 0)}, + {str: "4:5:6", expected: MakeTimeTZ(timeofday.New(4, 5, 6, 0), 0)}, + {str: "24:00", expected: MakeTimeTZ(timeofday.Time2400, 0)}, + {str: "24:00:00", expected: MakeTimeTZ(timeofday.Time2400, 0)}, + {str: "24:00:00.000", expected: MakeTimeTZ(timeofday.Time2400, 0)}, + {str: "24:00:00.000000", expected: MakeTimeTZ(timeofday.Time2400, 0)}, + {str: "01:02:03+13", expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), -13*60*60)}, + {str: "01:02:03-13", expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), 13*60*60)}, + {str: "01:02:03+7", expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), -7*60*60)}, + {str: "01:02:03-0730", expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), 7*60*60+30*60)}, + {str: "24:00+3", expected: MakeTimeTZ(timeofday.Time2400, -3*60*60)}, + {str: "24:00:00+4", expected: MakeTimeTZ(timeofday.Time2400, -4*60*60)}, + {str: "24:00:00.000-5", expected: MakeTimeTZ(timeofday.Time2400, 5*60*60)}, + {str: "24:00:00.000000+6", expected: MakeTimeTZ(timeofday.Time2400, -6*60*60)}, + {str: "00:00-1559", expected: MakeTimeTZ(timeofday.New(0, 0, 0, 0), MaxTimeTZOffsetSecs)}, + {str: "00:00+1559", expected: MakeTimeTZ(timeofday.New(0, 0, 0, 0), MinTimeTZOffsetSecs)}, + + {str: "", expectedError: true}, + {str: "foo", expectedError: true}, + {str: "01", expectedError: true}, + {str: "01:00=wat", expectedError: true}, + {str: "00:00-1600", expectedError: true}, + {str: "00:00+1600", expectedError: true}, + } + for i, tc := range testCases { + t.Run(fmt.Sprintf("#%d: %s", i, tc.str), func(t *testing.T) { + actual, err := ParseTimeTZ(timeutil.Now(), tc.str) + if tc.expectedError { + assert.Error(t, err) + } else { + assert.NoError(t, err) + assert.Equal(t, tc.expected, actual) + } + }) + } +} diff --git a/pkg/util/timeutil/pgdate/pgdate_test.go b/pkg/util/timeutil/pgdate/pgdate_test.go index 113975fce391..4bb35be3dbd4 100644 --- a/pkg/util/timeutil/pgdate/pgdate_test.go +++ b/pkg/util/timeutil/pgdate/pgdate_test.go @@ -13,10 +13,9 @@ package pgdate import ( "fmt" "math" + "strings" "testing" "time" - - "github.com/cockroachdb/cockroach/pkg/testutils" ) func TestDateFromTime(t *testing.T) { @@ -72,9 +71,10 @@ func TestDateFromTime(t *testing.T) { } { t.Run(tc.s, func(t *testing.T) { d, err := ParseDate(time.Time{}, ParseModeYMD, tc.s) - if !testutils.IsError(err, tc.err) { - t.Fatalf("got %v, expected %v", err, tc.err) - } else if err != nil { + if tc.err != "" { + if err == nil || !strings.Contains(err.Error(), tc.err) { + t.Fatalf("got %v, expected %v", err, tc.err) + } return } pg := d.PGEpochDays()