Skip to content

Commit

Permalink
Merge #44812 #44819
Browse files Browse the repository at this point in the history
44812: tree: modify TimestampTZ operators to use ctx loc for operations r=solongordon a=otan

Resolve cockroachdb/django-cockroachdb#54.

This PR involves my audit of eval.go and builtins.go to make sure all
TimestampTZ operations are performed in the context timezone for
TimestampTZ. Results are described in the release note.

Important to note we can't parse everything as the current context
location, because it can be subject to change mid-session.

Release note (bug fix, sql change): Previously, some TimestampTZ
operations did not correctly take context timezone (set by `SET TIME
ZONE`) into account.

This caused a few bugs:
* it leads to bugs involving daylight saving in arithmetic,
e.g. with `America/Chicago`, evaluating
'2010-11-06 23:59:00-05'::timestamptz + '1 day'::interval would return
incorrect results as it assumed it was a fixed offset of `-5` instead.
* text conversion from timestamptz to string sometimes used the wrong
timezone offset if the location of the session does not match the
location when the timestamptz was parsed.
* to_json builtins with timestamptz does not take session timezone
into consideration.

These have all been fixed by the PR.

44819: sql: refactor system.namespace calls to use indexes where available r=solongordon a=otan

Resolves #44230.

Refactoring all system.namespace calls such that they always use indexes
when querying data if it previously used indexes before.

I left `crdb_internal.lookup_namespace_id` as functional as it was
before in that it does not lookup anything that isn't a public schema
or database. This is because the admin UI only looks up tables into the
public schema.

Release note: None



Co-authored-by: Oliver Tan <[email protected]>
  • Loading branch information
craig[bot] and otan committed Feb 10, 2020
3 parents 82a0043 + db4db2e + 9f884c4 commit 917005b
Show file tree
Hide file tree
Showing 19 changed files with 190 additions and 64 deletions.
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -676,7 +676,7 @@ func TestBackupRestoreSystemJobs(t *testing.T) {
fullDir := sanitizedFullDir + "moarSecretsHere"

backupDatabaseID := sqlutils.QueryDatabaseID(t, conn, "data")
backupTableID := sqlutils.QueryTableID(t, conn, "data", "bank")
backupTableID := sqlutils.QueryTableID(t, conn, "data", "public", "bank")

sqlDB.Exec(t, `CREATE DATABASE restoredb`)
restoreDatabaseID := sqlutils.QueryDatabaseID(t, conn, "restoredb")
Expand Down Expand Up @@ -775,7 +775,7 @@ func checkInProgressBackupRestore(

sqlDB.Exec(t, `CREATE DATABASE restoredb`)

backupTableID := sqlutils.QueryTableID(t, conn, "data", "bank")
backupTableID := sqlutils.QueryTableID(t, conn, "data", "public", "bank")

do := func(query string, check inProgressChecker) {
jobDone := make(chan error)
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -804,7 +804,7 @@ func fetchDescVersionModificationTime(
Key: tblKey,
EndKey: tblKey.PrefixEnd(),
}
dropColTblID := sqlutils.QueryTableID(t, db, `d`, tableName)
dropColTblID := sqlutils.QueryTableID(t, db, `d`, "public", tableName)
req := &roachpb.ExportRequest{
RequestHeader: header,
MVCCFilter: roachpb.MVCCFilter_All,
Expand Down
7 changes: 4 additions & 3 deletions pkg/ccl/changefeedccl/encoder.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"io/ioutil"
"net/url"
"path/filepath"
"time"

"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
Expand Down Expand Up @@ -147,7 +148,7 @@ func (e *jsonEncoder) encodeKeyRaw(row encodeRow) ([]interface{}, error) {
return nil, err
}
var err error
jsonEntries[i], err = tree.AsJSON(datum.Datum)
jsonEntries[i], err = tree.AsJSON(datum.Datum, time.UTC)
if err != nil {
return nil, err
}
Expand All @@ -172,7 +173,7 @@ func (e *jsonEncoder) EncodeValue(_ context.Context, row encodeRow) ([]byte, err
return nil, err
}
var err error
after[col.Name], err = tree.AsJSON(datum.Datum)
after[col.Name], err = tree.AsJSON(datum.Datum, time.UTC)
if err != nil {
return nil, err
}
Expand All @@ -190,7 +191,7 @@ func (e *jsonEncoder) EncodeValue(_ context.Context, row encodeRow) ([]byte, err
return nil, err
}
var err error
before[col.Name], err = tree.AsJSON(datum.Datum)
before[col.Name], err = tree.AsJSON(datum.Datum, time.UTC)
if err != nil {
return nil, err
}
Expand Down
18 changes: 14 additions & 4 deletions pkg/ccl/importccl/load.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,12 +54,22 @@ func getDescriptorFromDB(
// Due to the namespace migration, the row may not exist in system.namespace
// so a fallback to system.namespace_deprecated is required.
// TODO(sqlexec): In 20.2, this logic can be removed.
for _, tableName := range []string{"system.namespace", "system.namespace_deprecated"} {
if err := db.QueryRow(fmt.Sprintf(`SELECT
for _, t := range []struct {
tableName string
extraClause string
}{
{"system.namespace", `AND n."parentSchemaID" = 0`},
{"system.namespace_deprecated", ""},
} {
if err := db.QueryRow(
fmt.Sprintf(`SELECT
d.descriptor
FROM %s n INNER JOIN system.descriptor d ON n.id = d.id
WHERE n."parentID" = $1
AND n.name = $2`, tableName),
WHERE n."parentID" = $1 %s
AND n.name = $2`,
t.tableName,
t.extraClause,
),
keys.RootNamespaceID,
dbName,
).Scan(&dbDescBytes); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/partitionccl/zone_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {
partialZoneOverride.GC = &zonepb.GCPolicy{TTLSeconds: 42}

dbID := sqlutils.QueryDatabaseID(t, db, "d")
tableID := sqlutils.QueryTableID(t, db, "d", "t")
tableID := sqlutils.QueryTableID(t, db, "d", "public", "t")

defaultRow := sqlutils.ZoneRow{
ID: keys.RootNamespaceID,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/ambiguous_commit_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ func TestAmbiguousCommit(t *testing.T) {
t.Fatal(err)
}

tableID := sqlutils.QueryTableID(t, sqlDB, "test", "t")
tableID := sqlutils.QueryTableID(t, sqlDB, "test", "public", "t")
tableStartKey.Store(keys.MakeTablePrefix(tableID))

// Wait for new table to split & replication.
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/json_builtins
Original file line number Diff line number Diff line change
Expand Up @@ -187,7 +187,7 @@ SELECT to_json('2014-05-28 12:22:35.614298'::TIMESTAMP)
query T
SELECT to_json('2014-05-28 12:22:35.614298-04'::TIMESTAMPTZ)
----
"2014-05-28T12:22:35.614298-04:00"
"2014-05-28T16:22:35.614298Z"

query T
SELECT to_json('2014-05-28 12:22:35.614298-04'::TIMESTAMP)
Expand Down Expand Up @@ -287,7 +287,7 @@ SELECT to_jsonb('2014-05-28 12:22:35.614298'::TIMESTAMP)
query T
SELECT to_jsonb('2014-05-28 12:22:35.614298-04'::TIMESTAMPTZ)
----
"2014-05-28T12:22:35.614298-04:00"
"2014-05-28T16:22:35.614298Z"

query T
SELECT to_jsonb('2014-05-28 12:22:35.614298-04'::TIMESTAMP)
Expand Down Expand Up @@ -538,7 +538,7 @@ SELECT json_build_object('a', 2, 'b', 4)
query T
SELECT jsonb_build_object(true,'val',1, 0, 1.3, 2, date '2019-02-03' - date '2019-01-01', 4, '2001-01-01 11:00+3'::timestamptz, '11:00+3'::timetz)
----
{"1": 0, "1.3": 2, "2001-01-01 11:00:00+03:00": "11:00:00+03:00:00", "33": 4, "true": "val"}
{"1": 0, "1.3": 2, "2001-01-01 08:00:00+00:00": "11:00:00+03:00:00", "33": 4, "true": "val"}

query T
SELECT json_build_object('a',1,'b',1.2,'c',true,'d',null,'e','{"x": 3, "y": [1,2,3]}'::JSON)
Expand Down
74 changes: 74 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/timestamp
Original file line number Diff line number Diff line change
Expand Up @@ -327,3 +327,77 @@ SELECT a, b FROM regression_44774 ORDER BY a

statement ok
DROP TABLE regression_44774

# Test for timestamptz math with interval involving DST.
subtest regression-cockroachdb/django-cockroachdb_57

statement ok
SET TIME ZONE 'America/Chicago'

query T
WITH a(a) AS ( VALUES
('2010-11-06 23:59:00'::timestamptz + '24 hours'::interval), -- no offset specified
('2010-11-06 23:59:00'::timestamptz + '1 day'::interval),
('2010-11-06 23:59:00'::timestamptz + '1 month'::interval),
('2010-11-07 23:59:00'::timestamptz - '24 hours'::interval),
('2010-11-07 23:59:00'::timestamptz - '1 day'::interval),
('2010-11-07 23:59:00'::timestamptz - '1 month'::interval),
('2010-11-06 23:59:00-05'::timestamptz + '24 hours'::interval), -- offset at time zone
('2010-11-06 23:59:00-05'::timestamptz + '1 day'::interval),
('2010-11-06 23:59:00-05'::timestamptz + '1 month'::interval),
('2010-11-07 23:59:00-06'::timestamptz - '24 hours'::interval),
('2010-11-07 23:59:00-06'::timestamptz - '1 day'::interval),
('2010-11-07 23:59:00-06'::timestamptz - '1 month'::interval),
('2010-11-06 23:59:00-04'::timestamptz + '24 hours'::interval), -- different offset
('2010-11-06 23:59:00-04'::timestamptz + '1 day'::interval),
('2010-11-06 23:59:00-04'::timestamptz + '1 month'::interval),
('2010-11-07 23:59:00-04'::timestamptz - '24 hours'::interval),
('2010-11-07 23:59:00-04'::timestamptz - '1 day'::interval),
('2010-11-07 23:59:00-04'::timestamptz - '1 month'::interval)
) select * from a;
----
2010-11-07 22:59:00 -0600 CST
2010-11-07 23:59:00 -0600 CST
2010-12-06 23:59:00 -0600 CST
2010-11-07 00:59:00 -0500 CDT
2010-11-06 23:59:00 -0500 CDT
2010-10-07 23:59:00 -0500 CDT
2010-11-07 22:59:00 -0600 CST
2010-11-07 23:59:00 -0600 CST
2010-12-06 23:59:00 -0600 CST
2010-11-07 00:59:00 -0500 CDT
2010-11-06 23:59:00 -0500 CDT
2010-10-07 23:59:00 -0500 CDT
2010-11-07 21:59:00 -0600 CST
2010-11-07 22:59:00 -0600 CST
2010-12-06 22:59:00 -0600 CST
2010-11-06 22:59:00 -0500 CDT
2010-11-06 21:59:00 -0500 CDT
2010-10-07 21:59:00 -0500 CDT

statement ok
CREATE TABLE example (a timestamptz)

statement ok
INSERT INTO example VALUES
('2010-11-06 23:59:00'),
('2010-11-07 23:59:00')

query TTTTTTTTT
SELECT
a + '24 hours'::interval, a + '1 day'::interval, a + '1 month'::interval,
a - '24 hours'::interval, a - '1 day'::interval, a - '1 month'::interval,
a - '2010-11-06 23:59:00'::timestamptz,
a - '2010-11-07 23:59:00'::timestamptz,
a::string
FROM example
ORDER BY a
----
2010-11-07 22:59:00 -0600 CST 2010-11-07 23:59:00 -0600 CST 2010-12-06 23:59:00 -0600 CST 2010-11-05 23:59:00 -0500 CDT 2010-11-05 23:59:00 -0500 CDT 2010-10-06 23:59:00 -0500 CDT 00:00:00 -25:00:00 2010-11-06 23:59:00-05:00
2010-11-08 23:59:00 -0600 CST 2010-11-08 23:59:00 -0600 CST 2010-12-07 23:59:00 -0600 CST 2010-11-07 00:59:00 -0500 CDT 2010-11-06 23:59:00 -0500 CDT 2010-10-07 23:59:00 -0500 CDT 25:00:00 00:00:00 2010-11-07 23:59:00-06:00

statement ok
DROP TABLE example

statement ok
SET TIME ZONE 0
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/timetz
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ SELECT '11:00'::time::timetz
query T
SELECT '2001-01-01 11:00+04:00'::timestamptz::timetz
----
0000-01-01 11:00:00 +0400 +0400
0000-01-01 02:00:00 -0500 -0500

statement ok
SET TIME ZONE UTC
Expand Down Expand Up @@ -307,7 +307,7 @@ SELECT '09:00:00.456-07' COLLATE de::TIMETZ;
query T
SELECT '2017-01-01 12:00:00-07':::TIMESTAMPTZ::TIMETZ;
----
0000-01-01 12:00:00 -0700 -0700
0000-01-01 19:00:00 +0000 UTC

query T
SELECT '12:00:00-07':::TIME::TIMETZ;
Expand Down
15 changes: 12 additions & 3 deletions pkg/sql/privileged_accessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,14 +22,23 @@ import (
)

// LookupNamespaceID implements tree.PrivilegedAccessor.
// TODO(sqlexec): make this work for any arbitrary schema.
// This currently only works for public schemas and databases.
func (p *planner) LookupNamespaceID(
ctx context.Context, parentID int64, name string,
) (tree.DInt, bool, error) {
var r tree.Datums
for _, tableName := range []string{"system.namespace", "system.namespace_deprecated"} {
for _, t := range []struct {
tableName string
extraClause string
}{
{"system.namespace", `AND "parentSchemaID" IN (0, 29)`},
{"system.namespace_deprecated", ""},
} {
query := fmt.Sprintf(
`SELECT id FROM %s WHERE "parentID" = $1 AND name = $2`,
tableName,
`SELECT id FROM %s WHERE "parentID" = $1 AND name = $2 %s`,
t.tableName,
t.extraClause,
)
var err error
r, err = p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryRowEx(
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/sem/builtins/aggregate_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"context"
"fmt"
"math"
"time"
"unsafe"

"github.com/cockroachdb/apd"
Expand Down Expand Up @@ -2182,13 +2183,15 @@ func (a *intXorAggregate) Size() int64 {
}

type jsonAggregate struct {
loc *time.Location
builder *json.ArrayBuilderWithCounter
acc mon.BoundAccount
sawNonNull bool
}

func newJSONAggregate(_ []*types.T, evalCtx *tree.EvalContext, _ tree.Datums) tree.AggregateFunc {
return &jsonAggregate{
loc: evalCtx.GetLocation(),
builder: json.NewArrayBuilderWithCounter(),
acc: evalCtx.Mon.MakeBoundAccount(),
sawNonNull: false,
Expand All @@ -2197,7 +2200,7 @@ func newJSONAggregate(_ []*types.T, evalCtx *tree.EvalContext, _ tree.Datums) tr

// Add accumulates the transformed json into the JSON array.
func (a *jsonAggregate) Add(ctx context.Context, datum tree.Datum, _ ...tree.Datum) error {
j, err := tree.AsJSON(datum)
j, err := tree.AsJSON(datum, a.loc)
if err != nil {
return err
}
Expand Down
Loading

0 comments on commit 917005b

Please sign in to comment.