Skip to content

Commit

Permalink
Merge #34202
Browse files Browse the repository at this point in the history
34202: sql: remove nanoseconds from INTERVAL r=mjibson a=mjibson

Nanoseconds were not representable over pgwire binary mode and were being
truncated. We previously encountered this problem with timestamps (#6597)
and removed nanoseconds from timestamps at that time. We should have
done the same for intervals, since they have the same kind of problem,
but did not.

It is no longer possible to create intervals with nanosecond
precision. Parsing from string or converting from float or decimal will
round to the nearest microsecond. Similarly any arithmetic operation
(add, sub, mul, div) on intervals will also round to nearest micro. We
round instead of truncate because that's what Postgres does.

Existing on-disk intervals that contain nanoseconds will retain their
underlying value when doing encode/decode operations (so that indexes can
be correctly maintained. However there is no longer any way to retrieve
the nanosecond part. Converting to string, float, or decimal will first
round and then convert.

The reasoning for this restriction on existing on-disk nanoseconds is
related to the original bug, where we were truncating nanos to micros over
binary pgwire. The problem there was that depending on how you queried
the data (text or binary mode), you would get a different result, and
one of them was wrong. Similarly, it would be wrong to have the results
of an interval -> string conversion return a different result than just
querying the interval.

It is unfortunate that upgrading from 2.1 -> 2.2 will completely remove
the ability for users to continue accessing their nanoseconds. Due to
that, we must describe in the major release notes this change. Users who
require nanoseconds to be present will have to modify their application
to use a different data type before upgrading. Further, applications
that do comparisons on intervals may have some edge cases errors due to
rounding and seeming equality. That is, some intervals with nanos will
be rounded up to the next microsecond, possibly changing the results of
an existing query. Also, it is not possible to compare equality to any
existing interval with on-disk nanos. We believe the number of users
affected by this will be very small, and that it is still a necessary
change because of the unavoidable pgwire binary mode bug above, which
may already have been unknowningly affecting them.

Other implementations were worked on, like one where the user could
specify the desired precision of each operation (similary to how
timestamps work). This ended up being very tedious since there
are many operations and they all required the same microsecond
precision. Timestamps are different since there are some operations
that actually do need nanosecond precision, but intervals have no such
need. Thus, it was better to remove the precision argument and hard
code rounding. Another attempt was made to replace Nanos with Micros,
with an additional nanos field to hold on-disk nanoseconds. This had
difficult problems since all of our encoding infra uses nanoseconds
on disk. Converting the Micros field to nanos increased the possibilty
of overflow due multiplying by 1000. Handling the possibility of this
overflow in all possibly locations would require many large and risky
changes.

The implementation changes here are a bit odd and surprising at
first. This change leaves the duration.Nanos field, but (excepting the
Decode func) automatically rounds Nanos to nearist micro. This does leave
open the possible misuse of the Nanos field, since durations are created
directly instead of via a constructor. However, I think this problem is
less of a risk as the other attempts listed above.

See #6604 and #8864 for previous PRs and discussion about this problem
when we fixed it for timestamps.

Fixes #32143

Release note (sql change): INTERVAL values are now stored with microsecond
precision instead of nanoseconds. Existing intervals with nanoseconds
are no longer able to return their nanosecond part. An existing table t
with nanoseconds in intervals of column s can round them to the nearest
microsecond with `UPDATE t SET s = s + '0s'`. Note that this could
potentially cause uniqueness problems if the interval is a primary key.

Co-authored-by: Matt Jibson <matt.jibson@gmail.com>
  • Loading branch information
craig[bot] and maddyblue committed Jan 31, 2019
2 parents 407932a + c535354 commit 72102ea
Show file tree
Hide file tree
Showing 27 changed files with 439 additions and 316 deletions.
6 changes: 1 addition & 5 deletions pkg/cli/dump_test.go
Expand Up @@ -238,11 +238,7 @@ func TestDumpRandom(t *testing.T) {
d := timeutil.Unix(0, rnd.Int63()).Round(time.Hour * 24)
m := timeutil.Unix(0, rnd.Int63()).Round(time.Microsecond)
sign := 1 - rnd.Int63n(2)*2
dur := duration.Duration{
Months: sign * rnd.Int63n(1000),
Days: sign * rnd.Int63n(1000),
Nanos: sign * rnd.Int63(),
}
dur := duration.MakeDuration(sign*rnd.Int63(), sign*rnd.Int63n(1000), sign*rnd.Int63n(1000))
n := dur.String()
o := rnd.Intn(2) == 1
e := apd.New(rnd.Int63(), rnd.Int31n(20)-10).String()
Expand Down
4 changes: 2 additions & 2 deletions pkg/keys/printer_test.go
Expand Up @@ -32,7 +32,7 @@ import (

func TestPrettyPrint(t *testing.T) {
tm, _ := time.Parse(time.RFC3339Nano, "2016-03-30T13:40:35.053725008Z")
duration := duration.Duration{Months: 1, Days: 1, Nanos: 1 * time.Second.Nanoseconds()}
duration := duration.MakeDuration(1*time.Second.Nanoseconds(), 1, 1)
durationAsc, _ := encoding.EncodeDurationAscending(nil, duration)
durationDesc, _ := encoding.EncodeDurationDescending(nil, duration)
bitArray := bitarray.MakeBitArrayFromInt64(8, 58, 7)
Expand Down Expand Up @@ -166,7 +166,7 @@ func TestPrettyPrint(t *testing.T) {
"/Table/42/1 mon 1 day 00:00:01"},
{makeKey(MakeTablePrefix(42),
roachpb.RKey(durationDesc)),
"/Table/42/-2 mons -2 days +743:59:58.999999999"},
"/Table/42/-2 mons -2 days +743:59:58.999999+999ns"},

// sequence
{MakeSequenceKey(55), `/Table/55/1/0/0`},
Expand Down
2 changes: 1 addition & 1 deletion pkg/roachpb/data.go
Expand Up @@ -721,7 +721,7 @@ func (v Value) PrettyPrint() string {
case ValueType_DURATION:
var d duration.Duration
d, err = v.GetDuration()
buf.WriteString(d.String())
buf.WriteString(d.StringNanos())
default:
err = errors.Errorf("unknown tag: %s", t)
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/roachpb/data_test.go
Expand Up @@ -1400,7 +1400,7 @@ func TestValuePrettyPrint(t *testing.T) {
_ = decimalValue.SetDecimal(apd.New(628, -2))

var durationValue Value
_ = durationValue.SetDuration(duration.Duration{Months: 1, Days: 2, Nanos: 3})
_ = durationValue.SetDuration(duration.DecodeDuration(1, 2, 3))

var tupleValue Value
tupleBytes := encoding.EncodeBytesValue(encoding.EncodeIntValue(nil, 1, 8), 2, []byte("foo"))
Expand Down Expand Up @@ -1431,7 +1431,7 @@ func TestValuePrettyPrint(t *testing.T) {
{floatValue, "/FLOAT/6.28"},
{timeValue, "/TIME/2016-06-29T16:02:50.000000005Z"},
{decimalValue, "/DECIMAL/6.28"},
{durationValue, "/DURATION/1 mon 2 days 00:00:00.000000003"},
{durationValue, "/DURATION/1 mon 2 days 00:00:00+3ns"},
{MakeValueFromBytes([]byte{0x1, 0x2, 0xF, 0xFF}), "/BYTES/0x01020fff"},
{MakeValueFromString("foo"), "/BYTES/foo"},
{tupleValue, "/TUPLE/1:1:Int/8/2:3:Bytes/foo"},
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/server_systemlog_gc_test.go
Expand Up @@ -228,7 +228,7 @@ func TestLogGCTrigger(t *testing.T) {
maxTS,
)

_, err = db.Exec(fmt.Sprintf("SET CLUSTER SETTING server.%s.ttl='1ns'", tc.table))
_, err = db.Exec(fmt.Sprintf("SET CLUSTER SETTING server.%s.ttl='1us'", tc.table))
a.NoError(err)

<-gcDone
Expand Down
8 changes: 3 additions & 5 deletions pkg/sql/exec_util.go
Expand Up @@ -586,7 +586,7 @@ func golangFillQueryArguments(args ...interface{}) tree.Datums {
case time.Time:
d = tree.MakeDTimestamp(t, time.Microsecond)
case time.Duration:
d = &tree.DInterval{Duration: duration.Duration{Nanos: t.Nanoseconds()}}
d = &tree.DInterval{Duration: duration.MakeDuration(t.Nanoseconds(), 0, 0)}
case bitarray.BitArray:
d = &tree.DBitArray{BitArray: t}
case *apd.Decimal:
Expand Down Expand Up @@ -1424,9 +1424,7 @@ func generateSessionTraceVTable(spans []tracing.RecordedSpan) ([]traceRow, error
opMap[spanIdx] = tree.NewDString(lrr.span.Operation)
if lrr.span.Duration != 0 {
durMap[spanIdx] = &tree.DInterval{
Duration: duration.Duration{
Nanos: lrr.span.Duration.Nanoseconds(),
},
Duration: duration.MakeDuration(lrr.span.Duration.Nanoseconds(), 0, 0),
}
}
}
Expand Down Expand Up @@ -1480,7 +1478,7 @@ func generateSessionTraceVTable(spans []tracing.RecordedSpan) ([]traceRow, error

ts := res[i][traceTimestampCol].(*tree.DTimestampTZ)
res[i][traceAgeCol] = &tree.DInterval{
Duration: duration.Duration{Nanos: ts.Sub(minTimestamp).Nanoseconds()},
Duration: duration.MakeDuration(ts.Sub(minTimestamp).Nanoseconds(), 0, 0),
}
}

Expand Down
6 changes: 1 addition & 5 deletions pkg/sql/pgwire/pgwirebase/encoding.go
Expand Up @@ -510,11 +510,7 @@ func DecodeOidDatum(
days := int32(binary.BigEndian.Uint32(b[8:]))
months := int32(binary.BigEndian.Uint32(b[12:]))

duration := duration.Duration{
Nanos: nanos,
Days: int64(days),
Months: int64(months),
}
duration := duration.MakeDuration(nanos, int64(days), int64(months))
return &tree.DInterval{Duration: duration}, nil
case oid.T_uuid:
u, err := tree.ParseDUuidFromBytes(b)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/types.go
Expand Up @@ -390,7 +390,7 @@ func (b *writeBuffer) writeBinaryDatum(

case *tree.DInterval:
b.putInt32(16)
b.putInt64(v.Nanos / int64(time.Microsecond/time.Nanosecond))
b.putInt64(v.Nanos() / int64(time.Microsecond/time.Nanosecond))
b.putInt32(int32(v.Days))
b.putInt32(int32(v.Months))

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/schema_changer_test.go
Expand Up @@ -78,7 +78,7 @@ func TestSchemaChangeLease(t *testing.T) {
if err != nil {
t.Fatal(err)
}
leaseDuration := time.Duration(leaseInterval.Duration.Nanos)
leaseDuration := time.Duration(leaseInterval.Duration.Nanos())
sqlRun.Exec(t, `
CREATE DATABASE t;
CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR);
Expand Down
5 changes: 1 addition & 4 deletions pkg/sql/sem/builtins/aggregate_builtins_test.go
Expand Up @@ -234,10 +234,7 @@ func makeIntervalTestDatum(count int) []tree.Datum {

vals := make([]tree.Datum, count)
for i := range vals {
vals[i] = &tree.DInterval{Duration: duration.Duration{Months: rng.Int63n(1000),
Days: rng.Int63n(1000),
Nanos: rng.Int63n(1000000),
}}
vals[i] = &tree.DInterval{Duration: duration.MakeDuration(rng.Int63n(1000000), rng.Int63n(1000), rng.Int63n(1000))}
}
return vals
}
Expand Down
16 changes: 7 additions & 9 deletions pkg/sql/sem/builtins/builtins.go
Expand Up @@ -1843,20 +1843,20 @@ may increase either contention or retry errors, or both.`,
timeSpan := strings.ToLower(string(tree.MustBeDString(args[0])))
switch timeSpan {
case "hour", "hours":
return tree.NewDInt(tree.DInt(fromInterval.Nanos / int64(time.Hour))), nil
return tree.NewDInt(tree.DInt(fromInterval.Nanos() / int64(time.Hour))), nil

case "minute", "minutes":
return tree.NewDInt(tree.DInt(fromInterval.Nanos / int64(time.Minute))), nil
return tree.NewDInt(tree.DInt(fromInterval.Nanos() / int64(time.Minute))), nil

case "second", "seconds":
return tree.NewDInt(tree.DInt(fromInterval.Nanos / int64(time.Second))), nil
return tree.NewDInt(tree.DInt(fromInterval.Nanos() / int64(time.Second))), nil

case "millisecond", "milliseconds":
// This a PG extension not supported in MySQL.
return tree.NewDInt(tree.DInt(fromInterval.Nanos / int64(time.Millisecond))), nil
return tree.NewDInt(tree.DInt(fromInterval.Nanos() / int64(time.Millisecond))), nil

case "microsecond", "microseconds":
return tree.NewDInt(tree.DInt(fromInterval.Nanos / int64(time.Microsecond))), nil
return tree.NewDInt(tree.DInt(fromInterval.Nanos() / int64(time.Microsecond))), nil

default:
return nil, pgerror.NewErrorf(
Expand Down Expand Up @@ -1946,7 +1946,7 @@ may increase either contention or retry errors, or both.`,
if err != nil {
return nil, err
}
return &tree.DInterval{Duration: duration.Duration{Nanos: int64(*time) * 1000}}, nil
return &tree.DInterval{Duration: duration.MakeDuration(int64(*time)*1000, 0, 0)}, nil
},
Info: "Truncates `input` to precision `element`. Sets all fields that are less\n" +
"significant than `element` to zero.\n\n" +
Expand Down Expand Up @@ -2870,9 +2870,7 @@ may increase either contention or retry errors, or both.`,
return nil, err
}
minDuration := args[0].(*tree.DInterval).Duration
elapsed := duration.Duration{
Nanos: int64(ctx.StmtTimestamp.Sub(ctx.TxnTimestamp)),
}
elapsed := duration.MakeDuration(int64(ctx.StmtTimestamp.Sub(ctx.TxnTimestamp)), 0, 0)
if elapsed.Compare(minDuration) < 0 {
return nil, ctx.Txn.GenerateForcedRetryableError(
ctx.Ctx(), "forced by crdb_internal.force_retry()")
Expand Down
44 changes: 18 additions & 26 deletions pkg/sql/sem/tree/datum.go
Expand Up @@ -2173,7 +2173,7 @@ const (
// ParseDInterval parses and returns the *DInterval Datum value represented by the provided
// string, or an error if parsing is unsuccessful.
func ParseDInterval(s string) (*DInterval, error) {
return parseDInterval(s, Second)
return ParseDIntervalWithField(s, Second)
}

// truncateDInterval truncates the input DInterval downward to the nearest
Expand All @@ -2183,16 +2183,16 @@ func truncateDInterval(d *DInterval, field DurationField) {
case Year:
d.Duration.Months = d.Duration.Months - d.Duration.Months%12
d.Duration.Days = 0
d.Duration.Nanos = 0
d.Duration.SetNanos(0)
case Month:
d.Duration.Days = 0
d.Duration.Nanos = 0
d.Duration.SetNanos(0)
case Day:
d.Duration.Nanos = 0
d.Duration.SetNanos(0)
case Hour:
d.Duration.Nanos = d.Duration.Nanos - d.Duration.Nanos%time.Hour.Nanoseconds()
d.Duration.SetNanos(d.Duration.Nanos() - d.Duration.Nanos()%time.Hour.Nanoseconds())
case Minute:
d.Duration.Nanos = d.Duration.Nanos - d.Duration.Nanos%time.Minute.Nanoseconds()
d.Duration.SetNanos(d.Duration.Nanos() - d.Duration.Nanos()%time.Minute.Nanoseconds())
case Second:
// Postgres doesn't truncate to whole seconds.
}
Expand Down Expand Up @@ -2244,13 +2244,13 @@ func parseDInterval(s string, field DurationField) (*DInterval, error) {
case Day:
ret.Days = int64(f)
case Hour:
ret.Nanos = time.Hour.Nanoseconds() * int64(f)
ret.SetNanos(time.Hour.Nanoseconds() * int64(f))
case Minute:
ret.Nanos = time.Minute.Nanoseconds() * int64(f)
ret.SetNanos(time.Minute.Nanoseconds() * int64(f))
case Second:
ret.Nanos = int64(float64(time.Second.Nanoseconds()) * f)
ret.SetNanos(int64(float64(time.Second.Nanoseconds()) * f))
case Millisecond:
ret.Nanos = int64(float64(time.Millisecond.Nanoseconds()) * f)
ret.SetNanos(int64(float64(time.Millisecond.Nanoseconds()) * f))
default:
panic(fmt.Sprintf("unhandled DurationField constant %d", field))
}
Expand Down Expand Up @@ -2304,26 +2304,18 @@ func (d *DInterval) Next(_ *EvalContext) (Datum, bool) {

// IsMax implements the Datum interface.
func (d *DInterval) IsMax(_ *EvalContext) bool {
return d.Months == math.MaxInt64 && d.Days == math.MaxInt64 && d.Nanos == math.MaxInt64
return d.Duration == dMaxInterval.Duration
}

// IsMin implements the Datum interface.
func (d *DInterval) IsMin(_ *EvalContext) bool {
return d.Months == math.MinInt64 && d.Days == math.MinInt64 && d.Nanos == math.MinInt64
}

var dMaxInterval = &DInterval{
duration.Duration{
Months: math.MaxInt64,
Days: math.MaxInt64,
Nanos: math.MaxInt64,
}}
var dMinInterval = &DInterval{
duration.Duration{
Months: math.MinInt64,
Days: math.MinInt64,
Nanos: math.MinInt64,
}}
return d.Duration == dMinInterval.Duration
}

var (
dMaxInterval = &DInterval{duration.MakeDuration(math.MaxInt64, math.MaxInt64, math.MaxInt64)}
dMinInterval = &DInterval{duration.MakeDuration(math.MinInt64, math.MinInt64, math.MinInt64)}
)

// Max implements the Datum interface.
func (d *DInterval) Max(_ *EvalContext) (Datum, bool) {
Expand Down
16 changes: 8 additions & 8 deletions pkg/sql/sem/tree/datum_test.go
Expand Up @@ -110,20 +110,20 @@ func TestDatumOrdering(t *testing.T) {

// Intervals
{`'1 day':::interval`, noPrev, noNext,
`'-768614336404564650 years -8 mons -9223372036854775808 days -2562047:47:16.854775808'`,
`'768614336404564650 years 7 mons 9223372036854775807 days 2562047:47:16.854775807'`},
`'-768614336404564650 years -8 mons -9223372036854775808 days -2562047:47:16.854775'`,
`'768614336404564650 years 7 mons 9223372036854775807 days 2562047:47:16.854775'`},
// Max interval: we use Postgres syntax, because Go doesn't accept
// months/days and ISO8601 doesn't accept nanoseconds.
{`'9223372036854775807 months 9223372036854775807 days ` +
`2562047 hours 47 minutes 16 seconds 854775807 nanoseconds':::interval`,
`2562047 hours 47 minutes 16 seconds 854775 us':::interval`,
noPrev, valIsMax,
`'-768614336404564650 years -8 mons -9223372036854775808 days -2562047:47:16.854775808'`,
`'768614336404564650 years 7 mons 9223372036854775807 days 2562047:47:16.854775807'`},
`'-768614336404564650 years -8 mons -9223372036854775808 days -2562047:47:16.854775'`,
`'768614336404564650 years 7 mons 9223372036854775807 days 2562047:47:16.854775'`},
{`'-9223372036854775808 months -9223372036854775808 days ` +
`-2562047 h -47 m -16 s -854775808 ns':::interval`,
`-2562047 h -47 m -16 s -854775 us':::interval`,
valIsMin, noNext,
`'-768614336404564650 years -8 mons -9223372036854775808 days -2562047:47:16.854775808'`,
`'768614336404564650 years 7 mons 9223372036854775807 days 2562047:47:16.854775807'`},
`'-768614336404564650 years -8 mons -9223372036854775808 days -2562047:47:16.854775'`,
`'768614336404564650 years 7 mons 9223372036854775807 days 2562047:47:16.854775'`},

// UUIDs
{`'ffffffff-ffff-ffff-ffff-ffffffffffff'::uuid`, `'ffffffff-ffff-ffff-ffff-fffffffffffe'`, valIsMax,
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/sem/tree/eval.go
Expand Up @@ -138,7 +138,7 @@ var UnaryOps = map[UnaryOperator]unaryOpOverload{
ReturnType: types.Interval,
Fn: func(_ *EvalContext, d Datum) (Datum, error) {
i := d.(*DInterval).Duration
i.Nanos = -i.Nanos
i.SetNanos(-i.Nanos())
i.Days = -i.Days
i.Months = -i.Months
return &DInterval{Duration: i}, nil
Expand Down Expand Up @@ -766,7 +766,7 @@ var BinOps = map[BinaryOperator]binOpOverload{
ReturnType: types.Interval,
Fn: func(_ *EvalContext, left Datum, right Datum) (Datum, error) {
nanos := left.(*DTimestamp).Sub(right.(*DTimestamp).Time).Nanoseconds()
return &DInterval{Duration: duration.Duration{Nanos: nanos}}, nil
return &DInterval{Duration: duration.MakeDuration(nanos, 0, 0)}, nil
},
},
&BinOp{
Expand All @@ -775,7 +775,7 @@ var BinOps = map[BinaryOperator]binOpOverload{
ReturnType: types.Interval,
Fn: func(_ *EvalContext, left Datum, right Datum) (Datum, error) {
nanos := left.(*DTimestampTZ).Sub(right.(*DTimestampTZ).Time).Nanoseconds()
return &DInterval{Duration: duration.Duration{Nanos: nanos}}, nil
return &DInterval{Duration: duration.MakeDuration(nanos, 0, 0)}, nil
},
},
&BinOp{
Expand All @@ -786,7 +786,7 @@ var BinOps = map[BinaryOperator]binOpOverload{
// These two quantities aren't directly comparable. Convert the
// TimestampTZ to a timestamp first.
nanos := left.(*DTimestamp).Sub(right.(*DTimestampTZ).stripTimeZone(ctx).Time).Nanoseconds()
return &DInterval{Duration: duration.Duration{Nanos: nanos}}, nil
return &DInterval{Duration: duration.MakeDuration(nanos, 0, 0)}, nil
},
},
&BinOp{
Expand All @@ -797,7 +797,7 @@ var BinOps = map[BinaryOperator]binOpOverload{
// These two quantities aren't directly comparable. Convert the
// TimestampTZ to a timestamp first.
nanos := left.(*DTimestampTZ).stripTimeZone(ctx).Sub(right.(*DTimestamp).Time).Nanoseconds()
return &DInterval{Duration: duration.Duration{Nanos: nanos}}, nil
return &DInterval{Duration: duration.MakeDuration(nanos, 0, 0)}, nil
},
},
&BinOp{
Expand Down Expand Up @@ -3254,7 +3254,7 @@ func PerformCast(ctx *EvalContext, d Datum, t coltypes.CastTargetType) (Datum, e
case *DFloat:
return &DInterval{Duration: duration.FromFloat64(float64(*v))}, nil
case *DTime:
return &DInterval{Duration: duration.Duration{Nanos: int64(*v) * 1000}}, nil
return &DInterval{Duration: duration.MakeDuration(int64(*v)*1000, 0, 0)}, nil
case *DDecimal:
d := ctx.getTmpDec()
dnanos := v.Decimal
Expand Down

0 comments on commit 72102ea

Please sign in to comment.