diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 748628bec229..958f78d5112e 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -14,6 +14,7 @@ import ( cryptorand "crypto/rand" "fmt" "net/url" + "sort" "strconv" "github.com/cockroachdb/cockroach/pkg/base" @@ -146,20 +147,176 @@ func (e *encryptedDataKeyMap) rangeOverMap(fn func(masterKeyID hashedMasterKeyID } } +type sortedIndexIDs []descpb.IndexID + +func (s sortedIndexIDs) Less(i, j int) bool { + return s[i] < s[j] +} + +func (s sortedIndexIDs) Swap(i, j int) { + s[i], s[j] = s[j], s[i] +} + +func (s sortedIndexIDs) Len() int { + return len(s) +} + +// getLogicallyMergedTableSpans returns all the non-drop index spans of the +// provided table but after merging them so as to minimize the number of spans +// generated. The following rules are used to logically merge the sorted set of +// non-drop index spans: +// - Contiguous index spans are merged. +// - Two non-contiguous index spans are merged if a scan request for the index +// IDs between them does not return any results. +// +// Egs: {/Table/51/1 - /Table/51/2}, {/Table/51/3 - /Table/51/4} => {/Table/51/1 - /Table/51/4} +// provided the dropped index represented by the span +// {/Table/51/2 - /Table/51/3} has been gc'ed. +func getLogicallyMergedTableSpans( + table catalog.TableDescriptor, + added map[tableAndIndex]bool, + codec keys.SQLCodec, + endTime hlc.Timestamp, + checkForKVInBounds func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error), +) ([]roachpb.Span, error) { + var nonDropIndexIDs []descpb.IndexID + if err := table.ForeachNonDropIndex(func(idxDesc *descpb.IndexDescriptor) error { + key := tableAndIndex{tableID: table.GetID(), indexID: idxDesc.ID} + if added[key] { + return nil + } + added[key] = true + nonDropIndexIDs = append(nonDropIndexIDs, idxDesc.ID) + return nil + }); err != nil { + return nil, err + } + + if len(nonDropIndexIDs) == 0 { + return nil, nil + } + + // There is no merging possible with only a single index, short circuit. + if len(nonDropIndexIDs) == 1 { + return []roachpb.Span{table.IndexSpan(codec, nonDropIndexIDs[0])}, nil + } + + sort.Sort(sortedIndexIDs(nonDropIndexIDs)) + + var mergedIndexSpans []roachpb.Span + + // mergedSpan starts off as the first span in the set of spans being + // considered for a logical merge. + // The logical span merge algorithm walks over the table's non drop indexes + // using an lhsSpan and rhsSpan (always offset by 1). It checks all index IDs + // between lhsSpan and rhsSpan to look for dropped but non-gced KVs. The + // existence of such a KV indicates that the rhsSpan cannot be included in the + // current set of spans being logically merged, and so we update the + // mergedSpan to encompass the lhsSpan as that is the furthest we can go. + // After recording the new "merged" span, we update mergedSpan to be the + // rhsSpan, and start processing the next logically mergeable span set. + mergedSpan := table.IndexSpan(codec, nonDropIndexIDs[0]) + for curIndex := 0; curIndex < len(nonDropIndexIDs)-1; curIndex++ { + lhsIndexID := nonDropIndexIDs[curIndex] + rhsIndexID := nonDropIndexIDs[curIndex+1] + + lhsSpan := table.IndexSpan(codec, lhsIndexID) + rhsSpan := table.IndexSpan(codec, rhsIndexID) + + lhsIndex, err := table.FindIndexByID(lhsIndexID) + if err != nil { + return nil, err + } + rhsIndex, err := table.FindIndexByID(rhsIndexID) + if err != nil { + return nil, err + } + + // If either the lhs or rhs is an interleaved index, we do not attempt to + // perform a logical merge of the spans because the index span for + // interleaved contains the tableID/indexID of the furthest ancestor in + // the interleaved chain. + if lhsIndex.IsInterleaved() || rhsIndex.IsInterleaved() { + mergedIndexSpans = append(mergedIndexSpans, mergedSpan) + mergedSpan = rhsSpan + } else { + var foundDroppedKV bool + // Iterate over all index IDs between the two candidates (lhs and rhs) + // which may be logically merged. These index IDs represent dropped + // indexes between the two non-drop index spans. + for i := lhsIndexID + 1; i < rhsIndexID; i++ { + // If we find an index which has been dropped but not gc'ed, we cannot + // merge the lhs and rhs spans. + foundDroppedKV, err = checkForKVInBounds(lhsSpan.EndKey, rhsSpan.Key, endTime) + if err != nil { + return nil, err + } + if foundDroppedKV { + mergedSpan.EndKey = lhsSpan.EndKey + mergedIndexSpans = append(mergedIndexSpans, mergedSpan) + mergedSpan = rhsSpan + break + } + } + } + + // The loop will terminate after this iteration and so we must update the + // current mergedSpan to encompass the last element in the nonDropIndexIDs + // slice as well. + if curIndex == len(nonDropIndexIDs)-2 { + mergedSpan.EndKey = rhsSpan.EndKey + mergedIndexSpans = append(mergedIndexSpans, mergedSpan) + } + } + + return mergedIndexSpans, nil +} + // spansForAllTableIndexes returns non-overlapping spans for every index and // table passed in. They would normally overlap if any of them are interleaved. +// The outputted spans are merged as described by the method +// getLogicallyMergedTableSpans, so as to optimize the size/number of the spans +// we BACKUP and lay protected ts records for. func spansForAllTableIndexes( - codec keys.SQLCodec, tables []catalog.TableDescriptor, revs []BackupManifest_DescriptorRevision, -) []roachpb.Span { + ctx context.Context, + execCfg *sql.ExecutorConfig, + endTime hlc.Timestamp, + tables []catalog.TableDescriptor, + revs []BackupManifest_DescriptorRevision, +) ([]roachpb.Span, error) { added := make(map[tableAndIndex]bool, len(tables)) sstIntervalTree := interval.NewTree(interval.ExclusiveOverlapper) + var mergedIndexSpans []roachpb.Span + var err error + + // checkForKVInBounds issues a scan request between start and end at endTime, + // and returns true if a non-nil result is returned. + checkForKVInBounds := func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error) { + var foundKV bool + err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.SetFixedTimestamp(ctx, endTime) + res, err := txn.Scan(ctx, start, end, 1 /* maxRows */) + if err != nil { + return err + } + foundKV = len(res) != 0 + return nil + }) + return foundKV, err + } + for _, table := range tables { - for _, index := range table.AllNonDropIndexes() { - if err := sstIntervalTree.Insert(intervalSpan(table.IndexSpan(codec, index.ID)), false); err != nil { + mergedIndexSpans, err = getLogicallyMergedTableSpans(table, added, execCfg.Codec, endTime, + checkForKVInBounds) + if err != nil { + return nil, err + } + + for _, indexSpan := range mergedIndexSpans { + if err := sstIntervalTree.Insert(intervalSpan(indexSpan), false); err != nil { panic(errors.NewAssertionErrorWithWrappedErrf(err, "IndexSpan")) } - added[tableAndIndex{tableID: table.GetID(), indexID: index.ID}] = true } } // If there are desc revisions, ensure that we also add any index spans @@ -175,13 +332,16 @@ func spansForAllTableIndexes( rawTbl := descpb.TableFromDescriptor(rev.Desc, hlc.Timestamp{}) if rawTbl != nil && rawTbl.State != descpb.DescriptorState_DROP { tbl := tabledesc.NewImmutable(*rawTbl) - for _, idx := range tbl.AllNonDropIndexes() { - key := tableAndIndex{tableID: tbl.ID, indexID: idx.ID} - if !added[key] { - if err := sstIntervalTree.Insert(intervalSpan(tbl.IndexSpan(codec, idx.ID)), false); err != nil { - panic(errors.NewAssertionErrorWithWrappedErrf(err, "IndexSpan")) - } - added[key] = true + revSpans, err := getLogicallyMergedTableSpans(tbl, added, execCfg.Codec, rev.Time, + checkForKVInBounds) + if err != nil { + return nil, err + } + + mergedIndexSpans = append(mergedIndexSpans, revSpans...) + for _, indexSpan := range mergedIndexSpans { + if err := sstIntervalTree.Insert(intervalSpan(indexSpan), false); err != nil { + panic(errors.NewAssertionErrorWithWrappedErrf(err, "IndexSpan")) } } } @@ -195,7 +355,19 @@ func spansForAllTableIndexes( }) return false }) - return spans + + // Attempt to merge any contiguous spans generated from the tables and revs. + mergedSpans, distinct := roachpb.MergeSpans(spans) + if !distinct { + return nil, errors.NewAssertionErrorWithWrappedErrf(errors.New("expected all resolved spans for the BACKUP to be distinct"), "IndexSpan") + } + + knobs := execCfg.BackupRestoreTestingKnobs + if knobs != nil && knobs.CaptureResolvedTableDescSpans != nil { + knobs.CaptureResolvedTableDescSpans(mergedSpans) + } + + return mergedSpans, nil } func getLocalityAndBaseURI(uri, appendPath string) (string, string, error) { @@ -769,11 +941,14 @@ func backupPlanHook( tenantRows = append(tenantRows, ds) } else { - spans = append(spans, spansForAllTableIndexes(p.ExecCfg().Codec, tables, revs)...) + tableSpans, err := spansForAllTableIndexes(ctx, p.ExecCfg(), endTime, tables, revs) + if err != nil { + return err + } + spans = append(spans, tableSpans...) // Include all tenants. // TODO(tbg): make conditional on cluster setting. - var err error tenantRows, err = p.ExecCfg().InternalExecutor.Query( ctx, "backup-lookup-tenant", p.ExtendedEvalContext().Txn, `SELECT id, active, info FROM system.tenants`, diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 5ffc09decf33..b0e1f3e5f035 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -52,6 +52,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/storage/cloud" @@ -63,6 +64,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -5676,7 +5678,283 @@ func TestProtectedTimestampsDuringBackup(t *testing.T) { }) require.NoError(t, g.Wait()) } +} + +// TestSpanSelectionDuringBackup tests the method spansForAllTableIndexes which +// is used to resolve the spans which will be backed up, and spans for which +// protected ts records will be created. +func TestProtectedTimestampSpanSelectionDuringBackup(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + skip.UnderStressRace(t, + "not worth starting/stopping the server for each subtest as they all rely on the shared"+ + " variable `actualResolvedSpan`") + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + dir, dirCleanupFn := testutils.TempDir(t) + defer dirCleanupFn() + params := base.TestClusterArgs{} + params.ServerArgs.ExternalIODir = dir + var actualResolvedSpans []string + params.ServerArgs.Knobs.BackupRestore = &sql.BackupRestoreTestingKnobs{ + CaptureResolvedTableDescSpans: func(mergedSpans []roachpb.Span) { + for _, span := range mergedSpans { + actualResolvedSpans = append(actualResolvedSpans, span.String()) + } + }, + } + tc := testcluster.StartTestCluster(t, 3, params) + defer tc.Stopper().Stop(ctx) + + tc.WaitForNodeLiveness(t) + require.NoError(t, tc.WaitForFullReplication()) + + conn := tc.ServerConn(0) + runner := sqlutils.MakeSQLRunner(conn) + baseBackupURI := "nodelocal://0/foo/" + + t.Run("contiguous-span-merge", func(t *testing.T) { + runner.Exec(t, "CREATE DATABASE test; USE test;") + runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, "+ + "INDEX baz(name), INDEX bar (v))") + + runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) + require.Equal(t, []string{"/Table/53/{1-4}"}, actualResolvedSpans) + runner.Exec(t, "DROP DATABASE test;") + actualResolvedSpans = nil + }) + + t.Run("drop-index-span-merge", func(t *testing.T) { + runner.Exec(t, "CREATE DATABASE test; USE test;") + runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, "+ + "INDEX baz(name), INDEX bar (v))") + runner.Exec(t, "INSERT INTO foo VALUES (1, NULL, 'testuser')") + runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=60") + runner.Exec(t, "DROP INDEX foo@baz") + + runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) + require.Equal(t, []string{"/Table/55/{1-2}", "/Table/55/{3-4}"}, actualResolvedSpans) + runner.Exec(t, "DROP DATABASE test;") + actualResolvedSpans = nil + }) + t.Run("drop-index-gced-span-merge", func(t *testing.T) { + runner.Exec(t, "CREATE DATABASE test; USE test;") + runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, "+ + "INDEX baz(name), INDEX bar (v))") + runner.Exec(t, "INSERT INTO foo VALUES (1, NULL, 'testuser')") + runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=1") + runner.Exec(t, "DROP INDEX foo@baz") + time.Sleep(time.Second * 2) + + runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) + require.Equal(t, []string{"/Table/57/{1-4}"}, actualResolvedSpans) + runner.Exec(t, "DROP DATABASE test;") + actualResolvedSpans = nil + }) + + t.Run("interleaved-spans", func(t *testing.T) { + runner.Exec(t, "CREATE DATABASE test; USE test;") + runner.Exec(t, "CREATE TABLE grandparent (a INT PRIMARY KEY, v BYTES, INDEX gpindex (v))") + runner.Exec(t, "CREATE TABLE parent (a INT, b INT, v BYTES, "+ + "PRIMARY KEY(a, b)) INTERLEAVE IN PARENT grandparent(a)") + runner.Exec(t, "CREATE TABLE child (a INT, b INT, c INT, v BYTES, "+ + "PRIMARY KEY(a, b, c), INDEX childindex(c)) INTERLEAVE IN PARENT parent(a, b)") + + runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) + // /Table/59/{1-2} encompasses the pk of grandparent, and the interleaved + // tables parent and child. + // /Table/59/2 - /Table/59/3 is for the gpindex + // /Table/61/{2-3} is for the childindex + require.Equal(t, []string{"/Table/59/{1-3}", "/Table/61/{2-3}"}, actualResolvedSpans) + runner.Exec(t, "DROP DATABASE test;") + actualResolvedSpans = nil + }) + + t.Run("revs-span-merge", func(t *testing.T) { + runner.Exec(t, "CREATE DATABASE test; USE test;") + runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, "+ + "INDEX baz(name), INDEX bar (v))") + runner.Exec(t, "INSERT INTO foo VALUES (1, NULL, 'testuser')") + runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=60") + runner.Exec(t, "DROP INDEX foo@baz") + + runner.Exec(t, `BACKUP DATABASE test TO 'nodelocal://0/fooz' WITH revision_history`) + + // The BACKUP with revision history will pickup the dropped index baz as + // well because it existed in a non-drop state at some point in the interval + // covered by this BACKUP. + require.Equal(t, []string{"/Table/63/{1-4}"}, actualResolvedSpans) + actualResolvedSpans = nil + runner.Exec(t, "DROP TABLE foo") + + runner.Exec(t, "CREATE TABLE foo2 (k INT PRIMARY KEY, v BYTES, name STRING, "+ + "INDEX baz(name), INDEX bar (v))") + runner.Exec(t, "INSERT INTO foo2 VALUES (1, NULL, 'testuser')") + runner.Exec(t, "ALTER TABLE foo2 CONFIGURE ZONE USING gc.ttlseconds=60") + runner.Exec(t, "DROP INDEX foo2@baz") + + runner.Exec(t, `BACKUP DATABASE test TO 'nodelocal://0/fooz' WITH revision_history`) + // We expect to see only the non-drop indexes of table foo in this + // incremental backup with revision history. We also expect to see both drop + // and non-drop indexes of table foo2 as all the indexes were live at some + // point in the interval covered by this BACKUP. + require.Equal(t, []string{"/Table/63/{1-2}", "/Table/63/{3-4}", "/Table/64/{1-4}"}, actualResolvedSpans) + runner.Exec(t, "DROP DATABASE test;") + actualResolvedSpans = nil + }) + + t.Run("last-index-dropped", func(t *testing.T) { + runner.Exec(t, "CREATE DATABASE test; USE test;") + runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, INDEX baz(name))") + runner.Exec(t, "CREATE TABLE foo2 (k INT PRIMARY KEY, v BYTES, name STRING, INDEX baz(name))") + runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=60") + runner.Exec(t, "DROP INDEX foo@baz") + + runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) + require.Equal(t, []string{"/Table/66/{1-2}", "/Table/67/{1-3}"}, actualResolvedSpans) + runner.Exec(t, "DROP DATABASE test;") + actualResolvedSpans = nil + }) + + t.Run("last-index-gced", func(t *testing.T) { + runner.Exec(t, "CREATE DATABASE test; USE test;") + runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, INDEX baz(name))") + runner.Exec(t, "INSERT INTO foo VALUES (1, NULL, 'test')") + runner.Exec(t, "CREATE TABLE foo2 (k INT PRIMARY KEY, v BYTES, name STRING, INDEX baz(name))") + runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=1") + runner.Exec(t, "DROP INDEX foo@baz") + time.Sleep(time.Second * 2) + runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=60") + + runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) + require.Equal(t, []string{"/Table/69/{1-2}", "/Table/70/{1-3}"}, actualResolvedSpans) + runner.Exec(t, "DROP DATABASE test;") + actualResolvedSpans = nil + }) +} + +func getMockIndexDesc(indexID descpb.IndexID) descpb.IndexDescriptor { + mockIndexDescriptor := descpb.IndexDescriptor{ID: indexID} + return mockIndexDescriptor +} + +func getMockTableDesc( + tableID descpb.ID, pkIndex descpb.IndexDescriptor, indexes []descpb.IndexDescriptor, +) tabledesc.Immutable { + mockTableDescriptor := descpb.TableDescriptor{ + ID: tableID, + PrimaryIndex: pkIndex, + Indexes: indexes, + } + mockImmutableTableDesc := tabledesc.Immutable{TableDescriptor: mockTableDescriptor} + return mockImmutableTableDesc +} + +// Unit tests for the getLogicallyMergedTableSpans() method. +func TestLogicallyMergedTableSpans(t *testing.T) { + defer leaktest.AfterTest(t)() + codec := keys.TODOSQLCodec + unusedMap := make(map[tableAndIndex]bool) + testCases := []struct { + name string + checkForKVInBoundsOverride func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error) + tableID descpb.ID + pkIndex descpb.IndexDescriptor + indexes []descpb.IndexDescriptor + expectedSpans []string + }{ + { + name: "contiguous-spans", + checkForKVInBoundsOverride: func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error) { + return false, nil + }, + tableID: 55, + pkIndex: getMockIndexDesc(1), + indexes: []descpb.IndexDescriptor{getMockIndexDesc(1), getMockIndexDesc(2)}, + expectedSpans: []string{"/Table/55/{1-3}"}, + }, + { + name: "dropped-span-between-two-spans", + checkForKVInBoundsOverride: func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error) { + if start.String() == "/Table/56/2" && end.String() == "/Table/56/3" { + return true, nil + } + return false, nil + }, + tableID: 56, + pkIndex: getMockIndexDesc(1), + indexes: []descpb.IndexDescriptor{getMockIndexDesc(1), getMockIndexDesc(3)}, + expectedSpans: []string{"/Table/56/{1-2}", "/Table/56/{3-4}"}, + }, + { + name: "gced-span-between-two-spans", + checkForKVInBoundsOverride: func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error) { + return false, nil + }, + tableID: 57, + pkIndex: getMockIndexDesc(1), + indexes: []descpb.IndexDescriptor{getMockIndexDesc(1), getMockIndexDesc(3)}, + expectedSpans: []string{"/Table/57/{1-4}"}, + }, + { + name: "alternate-spans-dropped", + checkForKVInBoundsOverride: func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error) { + if (start.String() == "/Table/58/2" && end.String() == "/Table/58/3") || + (start.String() == "/Table/58/4" && end.String() == "/Table/58/5") { + return true, nil + } + return false, nil + }, + tableID: 58, + pkIndex: getMockIndexDesc(1), + indexes: []descpb.IndexDescriptor{getMockIndexDesc(1), getMockIndexDesc(3), + getMockIndexDesc(5)}, + expectedSpans: []string{"/Table/58/{1-2}", "/Table/58/{3-4}", "/Table/58/{5-6}"}, + }, + { + name: "alternate-spans-gced", + checkForKVInBoundsOverride: func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error) { + return false, nil + }, + tableID: 59, + pkIndex: getMockIndexDesc(1), + indexes: []descpb.IndexDescriptor{getMockIndexDesc(1), getMockIndexDesc(3), + getMockIndexDesc(5)}, + expectedSpans: []string{"/Table/59/{1-6}"}, + }, + { + name: "one-drop-one-gc", + checkForKVInBoundsOverride: func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error) { + if start.String() == "/Table/60/2" && end.String() == "/Table/60/3" { + return true, nil + } + return false, nil + }, + tableID: 60, + pkIndex: getMockIndexDesc(1), + indexes: []descpb.IndexDescriptor{getMockIndexDesc(1), getMockIndexDesc(3), + getMockIndexDesc(5)}, + expectedSpans: []string{"/Table/60/{1-2}", "/Table/60/{3-6}"}, + }, + } + + for _, test := range testCases { + t.Run(test.name, func(t *testing.T) { + tableDesc := getMockTableDesc(test.tableID, test.pkIndex, test.indexes) + spans, err := getLogicallyMergedTableSpans(&tableDesc, unusedMap, codec, + hlc.Timestamp{}, test.checkForKVInBoundsOverride) + var mergedSpans []string + for _, span := range spans { + mergedSpans = append(mergedSpans, span.String()) + } + require.NoError(t, err) + require.Equal(t, test.expectedSpans, mergedSpans) + }) + } } func getFirstStoreReplica( diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 2b96b74fe57b..075d155972d2 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -830,6 +830,59 @@ func getTempSystemDBID(details jobspb.RestoreDetails) descpb.ID { return descpb.ID(tempSystemDBID) } +// spansForAllRestoreTableIndexes returns non-overlapping spans for every index +// and table passed in. They would normally overlap if any of them are +// interleaved. +func spansForAllRestoreTableIndexes( + codec keys.SQLCodec, tables []catalog.TableDescriptor, revs []BackupManifest_DescriptorRevision, +) []roachpb.Span { + + added := make(map[tableAndIndex]bool, len(tables)) + sstIntervalTree := interval.NewTree(interval.ExclusiveOverlapper) + for _, table := range tables { + for _, index := range table.AllNonDropIndexes() { + if err := sstIntervalTree.Insert(intervalSpan(table.IndexSpan(codec, index.ID)), false); err != nil { + panic(errors.NewAssertionErrorWithWrappedErrf(err, "IndexSpan")) + } + added[tableAndIndex{tableID: table.GetID(), indexID: index.ID}] = true + } + } + // If there are desc revisions, ensure that we also add any index spans + // in them that we didn't already get above e.g. indexes or tables that are + // not in latest because they were dropped during the time window in question. + for _, rev := range revs { + // If the table was dropped during the last interval, it will have + // at least 2 revisions, and the first one should have the table in a PUBLIC + // state. We want (and do) ignore tables that have been dropped for the + // entire interval. DROPPED tables should never later become PUBLIC. + // TODO(pbardea): Consider and test the interaction between revision_history + // backups and OFFLINE tables. + rawTbl := descpb.TableFromDescriptor(rev.Desc, hlc.Timestamp{}) + if rawTbl != nil && rawTbl.State != descpb.DescriptorState_DROP { + tbl := tabledesc.NewImmutable(*rawTbl) + for _, idx := range tbl.AllNonDropIndexes() { + key := tableAndIndex{tableID: tbl.ID, indexID: idx.ID} + if !added[key] { + if err := sstIntervalTree.Insert(intervalSpan(tbl.IndexSpan(codec, idx.ID)), false); err != nil { + panic(errors.NewAssertionErrorWithWrappedErrf(err, "IndexSpan")) + } + added[key] = true + } + } + } + } + + var spans []roachpb.Span + _ = sstIntervalTree.Do(func(r interval.Interface) bool { + spans = append(spans, roachpb.Span{ + Key: roachpb.Key(r.Range().Start), + EndKey: roachpb.Key(r.Range().End), + }) + return false + }) + return spans +} + // createImportingDescriptors create the tables that we will restore into. It also // fetches the information from the old tables that we need for the restore. func createImportingDescriptors( @@ -878,7 +931,7 @@ func createImportingDescriptors( // We get the spans of the restoring tables _as they appear in the backup_, // that is, in the 'old' keyspace, before we reassign the table IDs. - spans = spansForAllTableIndexes(p.ExecCfg().Codec, tables, nil) + spans = spansForAllRestoreTableIndexes(p.ExecCfg().Codec, tables, nil) log.Eventf(ctx, "starting restore for %d tables", len(mutableTables)) diff --git a/pkg/ccl/importccl/import_stmt_test.go b/pkg/ccl/importccl/import_stmt_test.go index adedd9dfb8d5..b0a2f8da5199 100644 --- a/pkg/ccl/importccl/import_stmt_test.go +++ b/pkg/ccl/importccl/import_stmt_test.go @@ -5334,6 +5334,91 @@ func TestImportPgDumpGeo(t *testing.T) { }) } +func TestImportPgDumpDropTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + baseDir := filepath.Join("testdata") + args := base.TestServerArgs{ExternalIODir: baseDir} + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ServerArgs: args}) + defer tc.Stopper().Stop(ctx) + conn := tc.Conns[0] + sqlDB := sqlutils.MakeSQLRunner(conn) + + var data string + srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + if r.Method == "GET" { + _, _ = w.Write([]byte(data)) + } + })) + defer srv.Close() + + // If the target table for a DROP exists, we throw an error. + t.Run("table exists", func(t *testing.T) { + + // Set up table `t` exists for testing. + sqlDB.Exec(t, `DROP TABLE IF EXISTS t; CREATE TABLE t (a INT);`) + + // Import PGDump data which includes DROP TABLE. + data = `DROP TABLE t; CREATE TABLE t (a INT); INSERT INTO t VALUES (4);` + sqlDB.ExpectErr(t, `drop table "t" and then retry the import`, `IMPORT PGDUMP ($1)`, srv.URL) + + // Also expect error on existing table with IF EXISTS. + data = `DROP TABLE IF EXISTS t; CREATE TABLE t (a INT); INSERT INTO t VALUES (4);` + sqlDB.ExpectErr(t, `drop table "t" and then retry the import`, `IMPORT PGDUMP ($1)`, srv.URL) + + // Cleanup. + sqlDB.Exec(t, `DROP TABLE t`) + }) + + // If the target table for a DROP does not exist, we ignore the statement. + t.Run("table does not exist", func(t *testing.T) { + + // Set up table `t` does not exist for testing. + sqlDB.Exec(t, `DROP TABLE IF EXISTS t;`) + + // No error should be thrown with DROP statement. + data = `DROP TABLE t; CREATE TABLE t (a INT); INSERT INTO t VALUES (4);` + expected := [][]string{{"4"}} + + sqlDB.Exec(t, `IMPORT PGDUMP ($1)`, srv.URL) + sqlDB.CheckQueryResults(t, `SELECT * FROM t`, expected) + + // Drop the table `t` that pgdump imported. + // Now table `t` does not exist for the IF EXISTS example. + sqlDB.Exec(t, `DROP TABLE t;`) + + // Also expect no errors and successful import with IF EXISTS. + data = `DROP TABLE IF EXISTS t; CREATE TABLE t (a INT); INSERT INTO t VALUES (4);` + sqlDB.Exec(t, `IMPORT PGDUMP ($1)`, srv.URL) + sqlDB.CheckQueryResults(t, `SELECT * FROM t`, expected) + + // Cleanup. + sqlDB.Exec(t, `DROP TABLE t`) + }) + + t.Run("multiple tables and drops", func(t *testing.T) { + // Set up. + sqlDB.Exec(t, `DROP TABLE IF EXISTS t, u;`) + + // Import table `t` successfully. + data = `DROP TABLE t; CREATE TABLE t (a INT)` + sqlDB.Exec(t, `IMPORT PGDUMP ($1)`, srv.URL) + + // Table `u` does not exist, so create it successfully. + // Table `t` exists, so an error is thrown for table `t`. + data = `DROP TABLE u; + CREATE TABLE u (a INT); + INSERT INTO u VALUES (55); + DROP TABLE t;` + sqlDB.ExpectErr(t, `drop table "t" and then retry the import`, `IMPORT PGDUMP ($1)`, srv.URL) + + // Since the PGDump failed on error, table `u` should not exist. + sqlDB.ExpectErr(t, `does not exist`, `SELECT * FROM u`) + }) +} + func TestImportCockroachDump(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/ccl/importccl/read_import_pgdump.go b/pkg/ccl/importccl/read_import_pgdump.go index 84c698311b88..bf9dbcb60088 100644 --- a/pkg/ccl/importccl/read_import_pgdump.go +++ b/pkg/ccl/importccl/read_import_pgdump.go @@ -15,7 +15,9 @@ import ( "regexp" "strings" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" @@ -309,7 +311,7 @@ func readPostgresCreateTable( if err != nil { return nil, errors.Wrap(err, "postgres parse error") } - if err := readPostgresStmt(ctx, evalCtx, match, fks, createTbl, createSeq, tableFKs, stmt); err != nil { + if err := readPostgresStmt(ctx, evalCtx, match, fks, createTbl, createSeq, tableFKs, stmt, p, parentID); err != nil { return nil, err } } @@ -324,6 +326,8 @@ func readPostgresStmt( createSeq map[string]*tree.CreateSequence, tableFKs map[string][]*tree.ForeignKeyConstraintTableDef, stmt interface{}, + p sql.JobExecContext, + parentID descpb.ID, ) error { switch stmt := stmt.(type) { case *tree.CreateTable: @@ -490,7 +494,7 @@ func readPostgresStmt( for _, fnStmt := range fnStmts { switch ast := fnStmt.AST.(type) { case *tree.AlterTable: - if err := readPostgresStmt(ctx, evalCtx, match, fks, createTbl, createSeq, tableFKs, ast); err != nil { + if err := readPostgresStmt(ctx, evalCtx, match, fks, createTbl, createSeq, tableFKs, ast, p, parentID); err != nil { return err } default: @@ -505,6 +509,32 @@ func readPostgresStmt( default: return errors.Errorf("unsupported %T SELECT: %s", sel, sel) } + case *tree.DropTable: + names := stmt.Names + + // If we find a table with the same name in the target DB we are importing + // into and same public schema, then we throw an error telling the user to + // drop the conflicting existing table to proceed. + // Otherwise, we silently ignore the drop statement and continue with the import. + for _, name := range names { + tableName := name.ToUnresolvedObjectName().String() + if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + err := backupccl.CheckObjectExists( + ctx, + txn, + p.ExecCfg().Codec, + parentID, + keys.PublicSchemaID, + tableName, + ) + if err != nil { + return errors.Wrapf(err, `drop table "%s" and then retry the import`, tableName) + } + return nil + }); err != nil { + return err + } + } case *tree.BeginTransaction, *tree.CommitTransaction: // ignore txns. case *tree.SetVar, *tree.Insert, *tree.CopyFrom, copyData, *tree.Delete: @@ -870,7 +900,7 @@ func (m *pgDumpReader) readFile( } case *tree.SetVar, *tree.BeginTransaction, *tree.CommitTransaction, *tree.Analyze: // ignored. - case *tree.CreateTable, *tree.AlterTable, *tree.CreateIndex, *tree.CreateSequence: + case *tree.CreateTable, *tree.AlterTable, *tree.CreateIndex, *tree.CreateSequence, *tree.DropTable: // handled during schema extraction. case *tree.Delete: switch stmt := i.Table.(type) { diff --git a/pkg/geo/geos/BUILD.bazel b/pkg/geo/geos/BUILD.bazel index f7218e79d1c6..f48207307ace 100644 --- a/pkg/geo/geos/BUILD.bazel +++ b/pkg/geo/geos/BUILD.bazel @@ -57,6 +57,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/geo/geos", visibility = ["//visibility:public"], deps = [ + "//pkg/docs", "//pkg/geo/geopb", "//vendor/github.com/cockroachdb/errors", ], diff --git a/pkg/geo/geos/geos.go b/pkg/geo/geos/geos.go index 779addf7acc5..da3381af526d 100644 --- a/pkg/geo/geos/geos.go +++ b/pkg/geo/geos/geos.go @@ -21,6 +21,7 @@ import ( "sync" "unsafe" + "github.com/cockroachdb/cockroach/pkg/docs" "github.com/cockroachdb/cockroach/pkg/geo/geopb" "github.com/cockroachdb/errors" ) @@ -190,9 +191,24 @@ func initGEOS(dirs []string) (*C.CR_GEOS, string, error) { ) } if err != nil { - return nil, "", errors.Wrap(err, "geos: error during GEOS init") + return nil, "", wrapGEOSInitError(errors.Wrap(err, "geos: error during GEOS init")) } - return nil, "", errors.Newf("geos: no locations to init GEOS") + return nil, "", wrapGEOSInitError(errors.Newf("geos: no locations to init GEOS")) +} + +func wrapGEOSInitError(err error) error { + page := "linux" + switch runtime.GOOS { + case "darwin": + page = "mac" + case "windows": + page = "windows" + } + return errors.WithHintf( + err, + "Ensure you have the spatial libraries installed as per the instructions in %s", + docs.URL("install-cockroachdb-"+page), + ) } // goToCSlice returns a CR_GEOS_Slice from a given Go byte slice. diff --git a/pkg/geo/geos/geos_test.go b/pkg/geo/geos/geos_test.go index 71694230699a..a3c500943eaa 100644 --- a/pkg/geo/geos/geos_test.go +++ b/pkg/geo/geos/geos_test.go @@ -11,6 +11,7 @@ package geos import ( + "strings" "testing" "github.com/cockroachdb/errors" @@ -21,11 +22,13 @@ func TestInitGEOS(t *testing.T) { t.Run("test no initGEOS paths", func(t *testing.T) { _, _, err := initGEOS([]string{}) require.Error(t, err) + require.Regexp(t, "Ensure you have the spatial libraries installed as per the instructions in .*install-cockroachdb-", strings.Join(errors.GetAllHints(err), "\n")) }) t.Run("test invalid initGEOS paths", func(t *testing.T) { _, _, err := initGEOS([]string{"/invalid/path"}) require.Error(t, err) + require.Regexp(t, "Ensure you have the spatial libraries installed as per the instructions in .*install-cockroachdb-", strings.Join(errors.GetAllHints(err), "\n")) }) t.Run("test valid initGEOS paths", func(t *testing.T) { diff --git a/pkg/sql/conn_executor_prepare.go b/pkg/sql/conn_executor_prepare.go index 5402cefe3371..75a6f93aef9f 100644 --- a/pkg/sql/conn_executor_prepare.go +++ b/pkg/sql/conn_executor_prepare.go @@ -339,8 +339,6 @@ func (ex *connExecutor) execBind( "expected %d arguments, got %d", numQArgs, len(bindCmd.Args))) } - ptCtx := tree.NewParseTimeContext(ex.state.sqlTimestamp.In(ex.sessionData.GetLocation())) - for i, arg := range bindCmd.Args { k := tree.PlaceholderIdx(i) t := ps.InferredTypes[i] @@ -348,7 +346,20 @@ func (ex *connExecutor) execBind( // nil indicates a NULL argument value. qargs[k] = tree.DNull } else { - d, err := pgwirebase.DecodeOidDatum(ctx, ptCtx, t, qArgFormatCodes[i], arg, &ex.planner) + typ, ok := types.OidToType[t] + if !ok { + var err error + typ, err = ex.planner.ResolveTypeByOID(ctx, t) + if err != nil { + return nil, err + } + } + d, err := pgwirebase.DecodeDatum( + ex.planner.EvalContext(), + typ, + qArgFormatCodes[i], + arg, + ) if err != nil { return retErr(pgerror.Wrapf(err, pgcode.ProtocolViolation, "error in argument for %s", k)) diff --git a/pkg/sql/copy.go b/pkg/sql/copy.go index 83040459d740..3994bcf6f264 100644 --- a/pkg/sql/copy.go +++ b/pkg/sql/copy.go @@ -371,13 +371,11 @@ func (c *copyMachine) readBinaryTuple(ctx context.Context) error { if len(data) != int(byteCount) { return errors.Newf("partial copy data row") } - d, err := pgwirebase.DecodeOidDatum( - ctx, + d, err := pgwirebase.DecodeDatum( c.parsingEvalCtx, - c.resultColumns[i].Typ.Oid(), + c.resultColumns[i].Typ, pgwirebase.FormatBinary, data, - &c.p, ) if err != nil { return pgerror.Wrapf(err, pgcode.BadCopyFileFormat, diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 9a4cac578f75..977ac667a904 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -882,6 +882,11 @@ type BackupRestoreTestingKnobs struct { // AllowImplicitAccess allows implicit access to data sources for non-admin // users. This enables using nodelocal for testing BACKUP/RESTORE permissions. AllowImplicitAccess bool + + // CaptureResolvedTableDescSpans allows for intercepting the spans which are + // resolved during backup planning, and will eventually be backed up during + // execution. + CaptureResolvedTableDescSpans func([]roachpb.Span) } var _ base.ModuleTestingKnobs = &BackupRestoreTestingKnobs{} diff --git a/pkg/sql/logictest/testdata/logic_test/pgoidtype b/pkg/sql/logictest/testdata/logic_test/pgoidtype index 1b92e621be74..c018e041417b 100644 --- a/pkg/sql/logictest/testdata/logic_test/pgoidtype +++ b/pkg/sql/logictest/testdata/logic_test/pgoidtype @@ -386,3 +386,9 @@ SELECT '12345'::regprocedure::string ---- 12345 12345 12345 12345 12345 + +query T +PREPARE regression_56193 AS SELECT $1::regclass; +EXECUTE regression_56193('regression_53686"'::regclass) +---- +"regression_53686""" diff --git a/pkg/sql/pgwire/encoding_test.go b/pkg/sql/pgwire/encoding_test.go index f5b32277fe5d..0602c62ae8e6 100644 --- a/pkg/sql/pgwire/encoding_test.go +++ b/pkg/sql/pgwire/encoding_test.go @@ -182,7 +182,12 @@ func TestEncodings(t *testing.T) { pgwirebase.FormatText: tc.TextAsBinary, pgwirebase.FormatBinary: tc.Binary, } { - d, err := pgwirebase.DecodeOidDatum(context.Background(), nil, tc.Oid, code, value, nil) + d, err := pgwirebase.DecodeDatum( + &evalCtx, + types.OidToType[tc.Oid], + code, + value, + ) if err != nil { t.Fatal(err) } @@ -240,7 +245,12 @@ func TestExoticNumericEncodings(t *testing.T) { evalCtx := tree.MakeTestingEvalContext(nil) for i, c := range testCases { t.Run(fmt.Sprintf("%d_%s", i, c.Value), func(t *testing.T) { - d, err := pgwirebase.DecodeOidDatum(context.Background(), nil, oid.T_numeric, pgwirebase.FormatBinary, c.Encoding, nil) + d, err := pgwirebase.DecodeDatum( + &evalCtx, + types.Decimal, + pgwirebase.FormatBinary, + c.Encoding, + ) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/pgwire/pgwirebase/encoding.go b/pkg/sql/pgwire/pgwirebase/encoding.go index 1198acd47973..4a8cbfe7c178 100644 --- a/pkg/sql/pgwire/pgwirebase/encoding.go +++ b/pkg/sql/pgwire/pgwirebase/encoding.go @@ -13,7 +13,6 @@ package pgwirebase import ( "bufio" "bytes" - "context" "encoding/binary" "io" "math" @@ -297,20 +296,23 @@ func validateArrayDimensions(nDimensions int, nElements int) error { return nil } -// DecodeOidDatum decodes bytes with specified Oid and format code into -// a datum. If the ParseTimeContext is nil, reasonable defaults -// will be applied. If res is nil, then user defined types are not attempted +// DecodeDatum decodes bytes with specified type and format code into +// a datum. If res is nil, then user defined types are not attempted // to be resolved. -func DecodeOidDatum( - ctx context.Context, - pCtx tree.ParseTimeContext, - id oid.Oid, - code FormatCode, - b []byte, - res tree.TypeReferenceResolver, +func DecodeDatum( + evalCtx *tree.EvalContext, t *types.T, code FormatCode, b []byte, ) (tree.Datum, error) { + id := t.Oid() switch code { case FormatText: + switch t.Family() { + case types.EnumFamily: + if err := validateStringBytes(b); err != nil { + return nil, err + } + return tree.MakeDEnumFromLogicalRepresentation(t, string(b)) + } + switch id { case oid.T_bool: t, err := strconv.ParseBool(string(b)) @@ -330,12 +332,18 @@ func DecodeOidDatum( return nil, err } return tree.NewDInt(tree.DInt(i)), nil - case oid.T_oid: - u, err := strconv.ParseUint(string(b), 10, 32) - if err != nil { - return nil, err - } - return tree.NewDOid(tree.DInt(u)), nil + case oid.T_oid, + oid.T_regoper, + oid.T_regproc, + oid.T_regrole, + oid.T_regclass, + oid.T_regtype, + oid.T_regconfig, + oid.T_regoperator, + oid.T_regnamespace, + oid.T_regprocedure, + oid.T_regdictionary: + return tree.ParseDOid(evalCtx, string(b), t) case oid.T_float4, oid.T_float8: f, err := strconv.ParseFloat(string(b), 64) if err != nil { @@ -373,19 +381,19 @@ func DecodeOidDatum( } return tree.NewDBytes(tree.DBytes(res)), nil case oid.T_timestamp: - d, _, err := tree.ParseDTimestamp(pCtx, string(b), time.Microsecond) + d, _, err := tree.ParseDTimestamp(evalCtx, string(b), time.Microsecond) if err != nil { return nil, pgerror.Newf(pgcode.Syntax, "could not parse string %q as timestamp", b) } return d, nil case oid.T_timestamptz: - d, _, err := tree.ParseDTimestampTZ(pCtx, string(b), time.Microsecond) + d, _, err := tree.ParseDTimestampTZ(evalCtx, string(b), time.Microsecond) if err != nil { return nil, pgerror.Newf(pgcode.Syntax, "could not parse string %q as timestamptz", b) } return d, nil case oid.T_date: - d, _, err := tree.ParseDDate(pCtx, string(b)) + d, _, err := tree.ParseDDate(evalCtx, string(b)) if err != nil { return nil, pgerror.Newf(pgcode.Syntax, "could not parse string %q as date", b) } @@ -397,7 +405,7 @@ func DecodeOidDatum( } return d, nil case oid.T_timetz: - d, _, err := tree.ParseDTimeTZ(pCtx, string(b), time.Microsecond) + d, _, err := tree.ParseDTimeTZ(evalCtx, string(b), time.Microsecond) if err != nil { return nil, pgerror.Newf(pgcode.Syntax, "could not parse string %q as timetz", b) } @@ -757,8 +765,7 @@ func DecodeOidDatum( return &tree.DBitArray{BitArray: ba}, err default: if _, ok := types.ArrayOids[id]; ok { - innerOid := types.OidToType[id].ArrayContents().Oid() - return decodeBinaryArray(ctx, pCtx, innerOid, b, code, res) + return decodeBinaryArray(evalCtx, types.OidToType[id].ArrayContents(), b, code) } } default: @@ -787,27 +794,6 @@ func DecodeOidDatum( return tree.NewDName(string(b)), nil } - // Finally, try to resolve the type's oid as a user defined type if a resolver - // was provided. - if res != nil { - typ, err := res.ResolveTypeByOID(ctx, id) - if err != nil { - return nil, err - } - switch typ.Family() { - case types.EnumFamily: - if code != FormatText { - return nil, pgerror.Newf(pgcode.Syntax, "expected FormatText for ENUM value encoding") - } - if err := validateStringBytes(b); err != nil { - return nil, err - } - return tree.MakeDEnumFromLogicalRepresentation(typ, string(b)) - default: - return nil, errors.AssertionFailedf("unsupported user defined type family %s", typ.Family().String()) - } - } - // Fallthrough case. return nil, errors.AssertionFailedf( "unsupported OID %v with format code %s", errors.Safe(id), errors.Safe(code)) @@ -909,12 +895,7 @@ func pgBinaryToIPAddr(b []byte) (ipaddr.IPAddr, error) { } func decodeBinaryArray( - ctx context.Context, - pCtx tree.ParseTimeContext, - elemOid oid.Oid, - b []byte, - code FormatCode, - res tree.TypeReferenceResolver, + evalCtx *tree.EvalContext, t *types.T, b []byte, code FormatCode, ) (tree.Datum, error) { var hdr struct { Ndims int32 @@ -934,10 +915,10 @@ func decodeBinaryArray( if err := binary.Read(r, binary.BigEndian, &hdr); err != nil { return nil, err } - if elemOid != oid.Oid(hdr.ElemOid) { + if t.Oid() != oid.Oid(hdr.ElemOid) { return nil, pgerror.Newf(pgcode.DatatypeMismatch, "wrong element type") } - arr := tree.NewDArray(types.OidToType[elemOid]) + arr := tree.NewDArray(types.OidToType[t.Oid()]) if hdr.Ndims == 0 { return arr, nil } @@ -959,7 +940,7 @@ func decodeBinaryArray( continue } buf := r.Next(int(vlen)) - elem, err := DecodeOidDatum(ctx, pCtx, elemOid, code, buf, res) + elem, err := DecodeDatum(evalCtx, t, code, buf) if err != nil { return nil, err } diff --git a/pkg/sql/pgwire/pgwirebase/fuzz.go b/pkg/sql/pgwire/pgwirebase/fuzz.go index d0002ab584c9..fa9f3a8e07f6 100644 --- a/pkg/sql/pgwire/pgwirebase/fuzz.go +++ b/pkg/sql/pgwire/pgwirebase/fuzz.go @@ -15,34 +15,35 @@ package pgwirebase import ( "context" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/lib/pq/oid" ) var ( - timeCtx = tree.NewParseTimeContext(timeutil.Now()) - // Compile a slice of all oids. - oids = func() []oid.Oid { - var ret []oid.Oid - for oid := range types.OidToType { - ret = append(ret, oid) + // Compile a slice of all typs. + typs = func() []*types.T { + var ret []*types.T + for _, typ := range types.OidToType { + ret = append(ret, typ) } return ret }() ) -func FuzzDecodeOidDatum(data []byte) int { +func FuzzDecodeDatum(data []byte) int { if len(data) < 2 { return 0 } - id := oids[int(data[1])%len(oids)] + typ := typs[int(data[1])%len(typs)] code := FormatCode(data[0]) % (FormatBinary + 1) b := data[2:] - _, err := DecodeOidDatum(context.Background(), timeCtx, id, code, b, nil) + evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + defer evalCtx.Stop(context.Background()) + + _, err := DecodeDatum(evalCtx, typ, code, b) if err != nil { return 0 } diff --git a/pkg/sql/pgwire/types_test.go b/pkg/sql/pgwire/types_test.go index ca245678ca7f..8c1904021c11 100644 --- a/pkg/sql/pgwire/types_test.go +++ b/pkg/sql/pgwire/types_test.go @@ -30,7 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" - "github.com/lib/pq/oid" ) // The assertions in this test should also be caught by the integration tests on @@ -136,12 +135,12 @@ func TestIntArrayRoundTrip(t *testing.T) { b := buf.wrapped.Bytes() - got, err := pgwirebase.DecodeOidDatum(context.Background(), nil, oid.T__int8, pgwirebase.FormatText, b[4:], nil) + evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + defer evalCtx.Stop(context.Background()) + got, err := pgwirebase.DecodeDatum(evalCtx, types.IntArray, pgwirebase.FormatText, b[4:]) if err != nil { t.Fatal(err) } - evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) - defer evalCtx.Stop(context.Background()) if got.Compare(evalCtx, d) != 0 { t.Fatalf("expected %s, got %s", d, got) } @@ -217,15 +216,15 @@ func TestByteArrayRoundTrip(t *testing.T) { b := buf.wrapped.Bytes() t.Logf("encoded: %v (%q)", b, b) - got, err := pgwirebase.DecodeOidDatum(context.Background(), nil, oid.T_bytea, pgwirebase.FormatText, b[4:], nil) + evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + defer evalCtx.Stop(context.Background()) + got, err := pgwirebase.DecodeDatum(evalCtx, types.Bytes, pgwirebase.FormatText, b[4:]) if err != nil { t.Fatal(err) } if _, ok := got.(*tree.DBytes); !ok { t.Fatalf("parse does not return DBytes, got %T", got) } - evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) - defer evalCtx.Stop(context.Background()) if got.Compare(evalCtx, d) != 0 { t.Fatalf("expected %s, got %s", d, got) } @@ -487,11 +486,11 @@ func BenchmarkDecodeBinaryDecimal(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - b.StartTimer() - got, err := pgwirebase.DecodeOidDatum(context.Background(), nil, oid.T_numeric, pgwirebase.FormatBinary, bytes, nil) - b.StopTimer() evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) defer evalCtx.Stop(context.Background()) + b.StartTimer() + got, err := pgwirebase.DecodeDatum(evalCtx, types.Decimal, pgwirebase.FormatBinary, bytes) + b.StopTimer() if err != nil { b.Fatal(err) } else if got.Compare(evalCtx, expected) != 0 { diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index ac2c8f6f2612..dac73db47599 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -162,6 +162,7 @@ go_test( "compare_test.go", "constant_eval_test.go", "constant_test.go", + "datum_integration_test.go", "datum_invariants_test.go", "datum_test.go", "eval_internal_test.go", diff --git a/pkg/sql/sem/tree/casts.go b/pkg/sql/sem/tree/casts.go index 16a614861af7..734ca052f3bd 100644 --- a/pkg/sql/sem/tree/casts.go +++ b/pkg/sql/sem/tree/casts.go @@ -1082,230 +1082,10 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { return oid, nil } case *DString: - s := string(*v) - - // If it is an integer in string form, convert it as an int. - if val, err := ParseDInt(strings.TrimSpace(s)); err == nil { - tmpOid := NewDOid(*val) - oid, err := queryOid(ctx, t, tmpOid) - if err != nil { - oid = tmpOid - oid.semanticType = t - } - return oid, nil - } - - switch t.Oid() { - case oid.T_regproc, oid.T_regprocedure: - // Trim procedure type parameters, e.g. `max(int)` becomes `max`. - // Postgres only does this when the cast is ::regprocedure, but we're - // going to always do it. - // We additionally do not yet implement disambiguation based on type - // parameters: we return the match iff there is exactly one. - s = pgSignatureRegexp.ReplaceAllString(s, "$1") - - substrs, err := splitIdentifierList(s) - if err != nil { - return nil, err - } - if len(substrs) > 3 { - // A fully qualified function name in pg's dialect can contain - // at most 3 parts: db.schema.funname. - // For example mydb.pg_catalog.max(). - // Anything longer is always invalid. - return nil, pgerror.Newf(pgcode.Syntax, - "invalid function name: %s", s) - } - name := UnresolvedName{NumParts: len(substrs)} - for i := 0; i < len(substrs); i++ { - name.Parts[i] = substrs[len(substrs)-1-i] - } - funcDef, err := name.ResolveFunction(ctx.SessionData.SearchPath) - if err != nil { - return nil, err - } - return queryOid(ctx, t, NewDString(funcDef.Name)) - case oid.T_regtype: - parsedTyp, err := ctx.Planner.ParseType(s) - if err == nil { - return &DOid{ - semanticType: t, - DInt: DInt(parsedTyp.Oid()), - name: parsedTyp.SQLStandardName(), - }, nil - } - - // Fall back to searching pg_type, since we don't provide syntax for - // every postgres type that we understand OIDs for. - // Note this section does *not* work if there is a schema in front of the - // type, e.g. "pg_catalog"."int4" (if int4 was not defined). - - // Trim whitespace and unwrap outer quotes if necessary. - // This is required to mimic postgres. - s = strings.TrimSpace(s) - if len(s) > 1 && s[0] == '"' && s[len(s)-1] == '"' { - s = s[1 : len(s)-1] - } - // Trim type modifiers, e.g. `numeric(10,3)` becomes `numeric`. - s = pgSignatureRegexp.ReplaceAllString(s, "$1") - - dOid, missingTypeErr := queryOid(ctx, t, NewDString(s)) - if missingTypeErr == nil { - return dOid, missingTypeErr - } - // Fall back to some special cases that we support for compatibility - // only. Client use syntax like 'sometype'::regtype to produce the oid - // for a type that they want to search a catalog table for. Since we - // don't support that type, we return an artificial OID that will never - // match anything. - switch s { - // We don't support triggers, but some tools search for them - // specifically. - case "trigger": - default: - return nil, missingTypeErr - } - return &DOid{ - semanticType: t, - // Types we don't support get OID -1, so they won't match anything - // in catalogs. - DInt: -1, - name: s, - }, nil - - case oid.T_regclass: - tn, err := castStringToRegClassTableName(s) - if err != nil { - return nil, err - } - id, err := ctx.Planner.ResolveTableName(ctx.Ctx(), &tn) - if err != nil { - return nil, err - } - return &DOid{ - semanticType: t, - DInt: DInt(id), - name: tn.ObjectName.String(), - }, nil - default: - return queryOid(ctx, t, NewDString(s)) - } + return ParseDOid(ctx, string(*v), t) } } return nil, pgerror.Newf( pgcode.CannotCoerce, "invalid cast: %s -> %s", d.ResolvedType(), t) } - -// castStringToRegClassTableName normalizes a TableName from a string. -func castStringToRegClassTableName(s string) (TableName, error) { - components, err := splitIdentifierList(s) - if err != nil { - return TableName{}, err - } - - if len(components) > 3 { - return TableName{}, pgerror.Newf( - pgcode.InvalidName, - "too many components: %s", - s, - ) - } - var retComponents [3]string - for i := 0; i < len(components); i++ { - retComponents[len(components)-1-i] = components[i] - } - u, err := NewUnresolvedObjectName( - len(components), - retComponents, - 0, - ) - if err != nil { - return TableName{}, err - } - return u.ToTableName(), nil -} - -// splitIdentifierList splits identifiers to individual components, lower -// casing non-quoted identifiers and escaping quoted identifiers as appropriate. -// It is based on PostgreSQL's SplitIdentifier. -func splitIdentifierList(in string) ([]string, error) { - var pos int - var ret []string - const separator = '.' - - for pos < len(in) { - if isWhitespace(in[pos]) { - pos++ - continue - } - if in[pos] == '"' { - var b strings.Builder - // Attempt to find the ending quote. If the quote is double "", - // fold it into a " character for the str (e.g. "a""" means a"). - for { - pos++ - endIdx := strings.IndexByte(in[pos:], '"') - if endIdx == -1 { - return nil, pgerror.Newf( - pgcode.InvalidName, - `invalid name: unclosed ": %s`, - in, - ) - } - b.WriteString(in[pos : pos+endIdx]) - pos += endIdx + 1 - // If we reached the end, or the following character is not ", - // we can break and assume this is one identifier. - // There are checks below to ensure EOF or whitespace comes - // afterward. - if pos == len(in) || in[pos] != '"' { - break - } - b.WriteByte('"') - } - ret = append(ret, b.String()) - } else { - var b strings.Builder - for pos < len(in) && in[pos] != separator && !isWhitespace(in[pos]) { - b.WriteByte(in[pos]) - pos++ - } - // Anything with no quotations should be lowered. - ret = append(ret, strings.ToLower(b.String())) - } - - // Further ignore all white space. - for pos < len(in) && isWhitespace(in[pos]) { - pos++ - } - - // At this stage, we expect separator or end of string. - if pos == len(in) { - break - } - - if in[pos] != separator { - return nil, pgerror.Newf( - pgcode.InvalidName, - "invalid name: expected separator %c: %s", - separator, - in, - ) - } - - pos++ - } - - return ret, nil -} - -// isWhitespace returns true if the given character is a space. -// This must match parser.SkipWhitespace above. -func isWhitespace(ch byte) bool { - switch ch { - case ' ', '\t', '\r', '\f', '\n': - return true - } - return false -} diff --git a/pkg/sql/sem/tree/casts_test.go b/pkg/sql/sem/tree/casts_test.go index 9b152a5fa907..f427d112b0b7 100644 --- a/pkg/sql/sem/tree/casts_test.go +++ b/pkg/sql/sem/tree/casts_test.go @@ -208,77 +208,3 @@ func TestTupleCastVolatility(t *testing.T) { } } } - -func TestCastStringToRegClassTableName(t *testing.T) { - defer leaktest.AfterTest(t)() - - testCases := []struct { - in string - expected TableName - }{ - {"a", MakeUnqualifiedTableName("a")}, - {`a"`, MakeUnqualifiedTableName(`a"`)}, - {`"a""".bB."cD" `, MakeTableNameWithSchema(`a"`, "bb", "cD")}, - } - - for _, tc := range testCases { - t.Run(tc.in, func(t *testing.T) { - out, err := castStringToRegClassTableName(tc.in) - require.NoError(t, err) - require.Equal(t, tc.expected, out) - }) - } - - errorTestCases := []struct { - in string - expectedError string - }{ - {"a.b.c.d", "too many components: a.b.c.d"}, - {"", `invalid table name: `}, - } - - for _, tc := range errorTestCases { - t.Run(tc.in, func(t *testing.T) { - _, err := castStringToRegClassTableName(tc.in) - require.EqualError(t, err, tc.expectedError) - }) - } - -} - -func TestSplitIdentifierList(t *testing.T) { - defer leaktest.AfterTest(t)() - - testCases := []struct { - in string - expected []string - }{ - {`abc`, []string{"abc"}}, - {`abc.dEf `, []string{"abc", "def"}}, - {` "aBc" . d ."HeLLo"""`, []string{"aBc", "d", `HeLLo"`}}, - } - - for _, tc := range testCases { - t.Run(tc.in, func(t *testing.T) { - out, err := splitIdentifierList(tc.in) - require.NoError(t, err) - require.Equal(t, tc.expected, out) - }) - } - - errorTestCases := []struct { - in string - expectedError string - }{ - {`"unclosed`, `invalid name: unclosed ": "unclosed`}, - {`"unclosed""`, `invalid name: unclosed ": "unclosed""`}, - {`hello !`, `invalid name: expected separator .: hello !`}, - } - - for _, tc := range errorTestCases { - t.Run(tc.in, func(t *testing.T) { - _, err := splitIdentifierList(tc.in) - require.EqualError(t, err, tc.expectedError) - }) - } -} diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index 732452b4c737..ef2e7aa882bd 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -4214,6 +4214,229 @@ func NewDOid(d DInt) *DOid { return &oid } +// ParseDOid parses and returns an Oid family datum. +func ParseDOid(ctx *EvalContext, s string, t *types.T) (*DOid, error) { + // If it is an integer in string form, convert it as an int. + if val, err := ParseDInt(strings.TrimSpace(s)); err == nil { + tmpOid := NewDOid(*val) + oid, err := queryOid(ctx, t, tmpOid) + if err != nil { + oid = tmpOid + oid.semanticType = t + } + return oid, nil + } + + switch t.Oid() { + case oid.T_regproc, oid.T_regprocedure: + // Trim procedure type parameters, e.g. `max(int)` becomes `max`. + // Postgres only does this when the cast is ::regprocedure, but we're + // going to always do it. + // We additionally do not yet implement disambiguation based on type + // parameters: we return the match iff there is exactly one. + s = pgSignatureRegexp.ReplaceAllString(s, "$1") + + substrs, err := splitIdentifierList(s) + if err != nil { + return nil, err + } + if len(substrs) > 3 { + // A fully qualified function name in pg's dialect can contain + // at most 3 parts: db.schema.funname. + // For example mydb.pg_catalog.max(). + // Anything longer is always invalid. + return nil, pgerror.Newf(pgcode.Syntax, + "invalid function name: %s", s) + } + name := UnresolvedName{NumParts: len(substrs)} + for i := 0; i < len(substrs); i++ { + name.Parts[i] = substrs[len(substrs)-1-i] + } + funcDef, err := name.ResolveFunction(ctx.SessionData.SearchPath) + if err != nil { + return nil, err + } + return queryOid(ctx, t, NewDString(funcDef.Name)) + case oid.T_regtype: + parsedTyp, err := ctx.Planner.ParseType(s) + if err == nil { + return &DOid{ + semanticType: t, + DInt: DInt(parsedTyp.Oid()), + name: parsedTyp.SQLStandardName(), + }, nil + } + + // Fall back to searching pg_type, since we don't provide syntax for + // every postgres type that we understand OIDs for. + // Note this section does *not* work if there is a schema in front of the + // type, e.g. "pg_catalog"."int4" (if int4 was not defined). + + // Trim whitespace and unwrap outer quotes if necessary. + // This is required to mimic postgres. + s = strings.TrimSpace(s) + if len(s) > 1 && s[0] == '"' && s[len(s)-1] == '"' { + s = s[1 : len(s)-1] + } + // Trim type modifiers, e.g. `numeric(10,3)` becomes `numeric`. + s = pgSignatureRegexp.ReplaceAllString(s, "$1") + + dOid, missingTypeErr := queryOid(ctx, t, NewDString(s)) + if missingTypeErr == nil { + return dOid, missingTypeErr + } + // Fall back to some special cases that we support for compatibility + // only. Client use syntax like 'sometype'::regtype to produce the oid + // for a type that they want to search a catalog table for. Since we + // don't support that type, we return an artificial OID that will never + // match anything. + switch s { + // We don't support triggers, but some tools search for them + // specifically. + case "trigger": + default: + return nil, missingTypeErr + } + return &DOid{ + semanticType: t, + // Types we don't support get OID -1, so they won't match anything + // in catalogs. + DInt: -1, + name: s, + }, nil + + case oid.T_regclass: + tn, err := castStringToRegClassTableName(s) + if err != nil { + return nil, err + } + id, err := ctx.Planner.ResolveTableName(ctx.Ctx(), &tn) + if err != nil { + return nil, err + } + return &DOid{ + semanticType: t, + DInt: DInt(id), + name: tn.ObjectName.String(), + }, nil + default: + return queryOid(ctx, t, NewDString(s)) + } +} + +// castStringToRegClassTableName normalizes a TableName from a string. +func castStringToRegClassTableName(s string) (TableName, error) { + components, err := splitIdentifierList(s) + if err != nil { + return TableName{}, err + } + + if len(components) > 3 { + return TableName{}, pgerror.Newf( + pgcode.InvalidName, + "too many components: %s", + s, + ) + } + var retComponents [3]string + for i := 0; i < len(components); i++ { + retComponents[len(components)-1-i] = components[i] + } + u, err := NewUnresolvedObjectName( + len(components), + retComponents, + 0, + ) + if err != nil { + return TableName{}, err + } + return u.ToTableName(), nil +} + +// splitIdentifierList splits identifiers to individual components, lower +// casing non-quoted identifiers and escaping quoted identifiers as appropriate. +// It is based on PostgreSQL's SplitIdentifier. +func splitIdentifierList(in string) ([]string, error) { + var pos int + var ret []string + const separator = '.' + + for pos < len(in) { + if isWhitespace(in[pos]) { + pos++ + continue + } + if in[pos] == '"' { + var b strings.Builder + // Attempt to find the ending quote. If the quote is double "", + // fold it into a " character for the str (e.g. "a""" means a"). + for { + pos++ + endIdx := strings.IndexByte(in[pos:], '"') + if endIdx == -1 { + return nil, pgerror.Newf( + pgcode.InvalidName, + `invalid name: unclosed ": %s`, + in, + ) + } + b.WriteString(in[pos : pos+endIdx]) + pos += endIdx + 1 + // If we reached the end, or the following character is not ", + // we can break and assume this is one identifier. + // There are checks below to ensure EOF or whitespace comes + // afterward. + if pos == len(in) || in[pos] != '"' { + break + } + b.WriteByte('"') + } + ret = append(ret, b.String()) + } else { + var b strings.Builder + for pos < len(in) && in[pos] != separator && !isWhitespace(in[pos]) { + b.WriteByte(in[pos]) + pos++ + } + // Anything with no quotations should be lowered. + ret = append(ret, strings.ToLower(b.String())) + } + + // Further ignore all white space. + for pos < len(in) && isWhitespace(in[pos]) { + pos++ + } + + // At this stage, we expect separator or end of string. + if pos == len(in) { + break + } + + if in[pos] != separator { + return nil, pgerror.Newf( + pgcode.InvalidName, + "invalid name: expected separator %c: %s", + separator, + in, + ) + } + + pos++ + } + + return ret, nil +} + +// isWhitespace returns true if the given character is a space. +// This must match parser.SkipWhitespace above. +func isWhitespace(ch byte) bool { + switch ch { + case ' ', '\t', '\r', '\f', '\n': + return true + } + return false +} + // AsDOid attempts to retrieve a DOid from an Expr, returning a DOid and // a flag signifying whether the assertion was successful. The function should // be used instead of direct type assertions wherever a *DOid wrapped by a diff --git a/pkg/sql/sem/tree/datum_integration_test.go b/pkg/sql/sem/tree/datum_integration_test.go new file mode 100644 index 000000000000..a4d90c0ccdc6 --- /dev/null +++ b/pkg/sql/sem/tree/datum_integration_test.go @@ -0,0 +1,1155 @@ +// Copyright 2016 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 tree_test + +import ( + "context" + "fmt" + "math" + "strings" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/timeofday" + "github.com/cockroachdb/cockroach/pkg/util/timetz" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func prepareExpr(t *testing.T, datumExpr string) tree.Datum { + expr, err := parser.ParseExpr(datumExpr) + if err != nil { + t.Fatalf("%s: %v", datumExpr, err) + } + // Type checking ensures constant folding is performed and type + // annotations have come into effect. + ctx := context.Background() + sema := tree.MakeSemaContext() + typedExpr, err := tree.TypeCheck(ctx, expr, &sema, types.Any) + if err != nil { + t.Fatalf("%s: %v", datumExpr, err) + } + // Normalization ensures that casts are processed. + evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + defer evalCtx.Stop(context.Background()) + typedExpr, err = evalCtx.NormalizeExpr(typedExpr) + if err != nil { + t.Fatalf("%s: %v", datumExpr, err) + } + d, err := typedExpr.Eval(evalCtx) + if err != nil { + t.Fatalf("%s: %v", datumExpr, err) + } + return d +} + +func TestDatumOrdering(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const valIsMin = `min` + const valIsMax = `max` + const noPrev = `` + const noNext = `` + const noMin = `` + const noMax = `` + + testData := []struct { + datumExpr string + prev string + next string + min string + max string + }{ + // Integers + {`1`, `0`, `2`, `-9223372036854775808`, `9223372036854775807`}, + {`-9223372036854775808`, valIsMin, `-9223372036854775807`, `-9223372036854775808`, `9223372036854775807`}, + {`9223372036854775807`, `9223372036854775806`, valIsMax, `-9223372036854775808`, `9223372036854775807`}, + + // Boolean + {`true`, `false`, valIsMax, `false`, `true`}, + {`false`, valIsMin, `true`, `false`, `true`}, + + // Floats + {`3.14:::float`, `3.1399999999999997`, `3.1400000000000006`, `NaN`, `+Inf`}, + {`9.223372036854776e+18:::float`, `9.223372036854775e+18`, `9.223372036854778e+18`, `NaN`, `+Inf`}, + {`'NaN':::float`, valIsMin, `-Inf`, `NaN`, `+Inf`}, + {`-1.7976931348623157e+308:::float`, `-Inf`, `-1.7976931348623155e+308`, `NaN`, `+Inf`}, + {`1.7976931348623157e+308:::float`, `1.7976931348623155e+308`, `+Inf`, `NaN`, `+Inf`}, + + // Decimal + {`1.0:::decimal`, noPrev, noNext, `NaN`, `Infinity`}, + + // Strings and byte arrays + {`'':::string`, valIsMin, `e'\x00'`, `''`, noMax}, + {`e'\x00'`, noPrev, `e'\x00\x00'`, `''`, noMax}, + {`'abc':::string`, noPrev, `e'abc\x00'`, `''`, noMax}, + {`'':::bytes`, valIsMin, `'\x00'`, `'\x'`, noMax}, + {`'abc':::bytes`, noPrev, `'\x61626300'`, `'\x'`, noMax}, + + // Dates + {`'2006-01-02':::date`, `'2006-01-01'`, `'2006-01-03'`, `'-infinity'`, `'infinity'`}, + {`'0001-01-01':::date`, `'0001-12-31 BC'`, `'0001-01-02'`, `'-infinity'`, `'infinity'`}, + {`'4000-01-01 BC':::date`, `'4001-12-31 BC'`, `'4000-01-02 BC'`, `'-infinity'`, `'infinity'`}, + {`'2006-01-02 03:04:05.123123':::timestamp`, + `'2006-01-02 03:04:05.123122'`, `'2006-01-02 03:04:05.123124'`, `'-4713-11-24 00:00:00'`, `'294276-12-31 23:59:59.999999'`}, + + // Geospatial types + {`'BOX(1 2,3 4)'::box2d`, noPrev, noNext, noMin, noMax}, + {`'POINT(1.0 1.0)'::geometry`, noPrev, noNext, noMin, noMax}, + {`'POINT(1.0 1.0)'::geography`, noPrev, noNext, noMin, noMax}, + + // Times + {`'00:00:00':::time`, valIsMin, `'00:00:00.000001'`, + `'00:00:00'`, `'24:00:00'`}, + {`'12:00:00':::time`, `'11:59:59.999999'`, `'12:00:00.000001'`, + `'00:00:00'`, `'24:00:00'`}, + {`'24:00:00':::time`, `'23:59:59.999999'`, valIsMax, `'00:00:00'`, `'24:00:00'`}, + + // Intervals + {`'1 day':::interval`, noPrev, noNext, + `'-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 854775 us':::interval`, + noPrev, valIsMax, + `'-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 -854775 us':::interval`, + valIsMin, noNext, + `'-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, + `'00000000-0000-0000-0000-000000000000'`, `'ffffffff-ffff-ffff-ffff-ffffffffffff'`}, + {`'00000000-0000-0000-0000-000000000000'::uuid`, valIsMin, `'00000000-0000-0000-0000-000000000001'`, + `'00000000-0000-0000-0000-000000000000'`, `'ffffffff-ffff-ffff-ffff-ffffffffffff'`}, + {`'ffffffff-ffff-ffff-0000-000000000000'::uuid`, `'ffffffff-ffff-fffe-ffff-ffffffffffff'`, + `'ffffffff-ffff-ffff-0000-000000000001'`, `'00000000-0000-0000-0000-000000000000'`, + `'ffffffff-ffff-ffff-ffff-ffffffffffff'`}, + {`'00000000-0000-0000-ffff-ffffffffffff'::uuid`, `'00000000-0000-0000-ffff-fffffffffffe'`, + `'00000000-0000-0001-0000-000000000000'`, `'00000000-0000-0000-0000-000000000000'`, + `'ffffffff-ffff-ffff-ffff-ffffffffffff'`}, + + // INETs + {`'0.0.0.0'::inet`, `'255.255.255.255/31'`, `'0.0.0.1'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'0.0.0.0/0'::inet`, noPrev, `'0.0.0.1/0'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'192.168.255.255'::inet`, `'192.168.255.254'`, `'192.169.0.0'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'127.0.0.1'::inet`, `'127.0.0.0'`, `'127.0.0.2'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'192.168.0.1/20'::inet`, `'192.168.0.0/20'`, `'192.168.0.2/20'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'192.168.0.0/20'::inet`, `'192.167.255.255/20'`, `'192.168.0.1/20'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'::ffff:1.2.3.4'::inet`, `'::ffff:1.2.3.3'`, `'::ffff:1.2.3.5'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'::0'::inet`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/127'`, `'::1'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'::0/0'::inet`, `'255.255.255.255'`, `'::1/0'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'255.255.255.255/32'::inet`, `'255.255.255.254'`, `'::/0'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'255.255.255.255/16'::inet`, `'255.255.255.254/16'`, `'0.0.0.0/17'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + {`'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/128'::inet`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'`, noNext, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, + + // NULL + {`NULL`, valIsMin, valIsMax, `NULL`, `NULL`}, + + // Tuples + {`row()`, valIsMin, valIsMax, `()`, `()`}, + + {`(NULL,)`, valIsMin, valIsMax, `(NULL,)`, `(NULL,)`}, + + {`(true,)`, `(false,)`, valIsMax, `(false,)`, `(true,)`}, + {`(false,)`, valIsMin, `(true,)`, `(false,)`, `(true,)`}, + + {`(true, false, false)`, `(false, true, true)`, `(true, false, true)`, + `(false, false, false)`, `(true, true, true)`}, + {`(false, true, true)`, `(false, true, false)`, `(true, NULL, NULL)`, + `(false, false, false)`, `(true, true, true)`}, + + {`(0, 0)`, `(0, -1)`, `(0, 1)`, + `(-9223372036854775808, -9223372036854775808)`, + `(9223372036854775807, 9223372036854775807)`}, + + {`(0, 9223372036854775807)`, + `(0, 9223372036854775806)`, `(1, NULL)`, + `(-9223372036854775808, -9223372036854775808)`, + `(9223372036854775807, 9223372036854775807)`}, + {`(9223372036854775807, 9223372036854775807)`, + `(9223372036854775807, 9223372036854775806)`, valIsMax, + `(-9223372036854775808, -9223372036854775808)`, + `(9223372036854775807, 9223372036854775807)`}, + + {`(0, 0:::decimal)`, noPrev, noNext, + `(-9223372036854775808, NaN)`, + `(9223372036854775807, Infinity)`}, + {`(0:::decimal, 0)`, `(0, -1)`, `(0, 1)`, + `(NaN, -9223372036854775808)`, + `(Infinity, 9223372036854775807)`}, + + {`(10, '')`, noPrev, `(10, e'\x00')`, + `(-9223372036854775808, '')`, noMax}, + {`(-9223372036854775808, '')`, valIsMin, `(-9223372036854775808, e'\x00')`, + `(-9223372036854775808, '')`, noMax}, + {`(-9223372036854775808, 'abc')`, noPrev, `(-9223372036854775808, e'abc\x00')`, + `(-9223372036854775808, '')`, noMax}, + + {`(10, NULL)`, `(9, NULL)`, `(11, NULL)`, + `(-9223372036854775808, NULL)`, `(9223372036854775807, NULL)`}, + {`(NULL, 10)`, `(NULL, 9)`, `(NULL, 11)`, + `(NULL, -9223372036854775808)`, `(NULL, 9223372036854775807)`}, + + {`(true, NULL, false)`, `(false, NULL, true)`, `(true, NULL, true)`, + `(false, NULL, false)`, `(true, NULL, true)`}, + {`(false, NULL, true)`, `(false, NULL, false)`, `(true, NULL, NULL)`, + `(false, NULL, false)`, `(true, NULL, true)`}, + + {`((true,), (false,))`, `((false,), (true,))`, `((true,), (true,))`, + `((false,), (false,))`, `((true,), (true,))`}, + {`((false,), (true,))`, `((false,), (false,))`, `((true,), NULL)`, + `((false,), (false,))`, `((true,), (true,))`}, + + // Arrays + + {`'{}'::INT[]`, valIsMin, `ARRAY[NULL]`, `ARRAY[]`, noMax}, + + {`array[NULL]`, noPrev, `ARRAY[NULL,NULL]`, `ARRAY[]`, noMax}, + {`array[true]`, noPrev, `ARRAY[true,NULL]`, `ARRAY[]`, noMax}, + + // Mixed tuple/array datums. + {`(ARRAY[true], (true,))`, `(ARRAY[true], (false,))`, `(ARRAY[true,NULL], NULL)`, + `(ARRAY[], (false,))`, noMax}, + {`((false,), ARRAY[true])`, noPrev, `((false,), ARRAY[true,NULL])`, + `((false,), ARRAY[])`, noMax}, + } + ctx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + for _, td := range testData { + d := prepareExpr(t, td.datumExpr) + + prevVal, hasPrev := d.Prev(ctx) + nextVal, hasNext := d.Next(ctx) + if td.prev == noPrev { + if hasPrev { + if !d.IsMin(ctx) { + t.Errorf("%s: value should not have a prev, yet hasPrev true and IsMin() false (expected (!hasPrev || IsMin()))", td.datumExpr) + } + } + } else { + if !hasPrev && td.prev != valIsMin { + t.Errorf("%s: hasPrev: got false, expected true", td.datumExpr) + continue + } + isMin := d.IsMin(ctx) + if isMin != (td.prev == valIsMin) { + t.Errorf("%s: IsMin() %v, expected %v", td.datumExpr, isMin, (td.prev == valIsMin)) + continue + } + if !isMin { + dPrev := prevVal.String() + if dPrev != td.prev { + t.Errorf("%s: Prev(): got %s, expected %s", td.datumExpr, dPrev, td.prev) + } + } + } + if td.next == noNext { + if hasNext { + if !d.IsMax(ctx) { + t.Errorf("%s: value should not have a next, yet hasNext true and IsMax() false (expected (!hasNext || IsMax()))", td.datumExpr) + } + } + } else { + if !hasNext && td.next != valIsMax { + t.Errorf("%s: HasNext(): got false, expected true", td.datumExpr) + continue + } + isMax := d.IsMax(ctx) + if isMax != (td.next == valIsMax) { + t.Errorf("%s: IsMax() %v, expected %v", td.datumExpr, isMax, (td.next == valIsMax)) + continue + } + if !isMax { + dNext := nextVal.String() + if dNext != td.next { + t.Errorf("%s: Next(): got %s, expected %s", td.datumExpr, dNext, td.next) + } + } + } + + minVal, hasMin := d.Min(ctx) + maxVal, hasMax := d.Max(ctx) + + if td.min == noMin { + if hasMin { + t.Errorf("%s: hasMin true, expected false", td.datumExpr) + } + } else { + dMin := minVal.String() + if dMin != td.min { + t.Errorf("%s: min(): got %s, expected %s", td.datumExpr, dMin, td.min) + } + } + if td.max == noMax { + if hasMax { + t.Errorf("%s: hasMax true, expected false", td.datumExpr) + } + } else { + dMax := maxVal.String() + if dMax != td.max { + t.Errorf("%s: max(): got %s, expected %s", td.datumExpr, dMax, td.max) + } + } + } +} + +func TestDFloatCompare(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + values := []tree.Datum{tree.DNull} + for _, x := range []float64{math.NaN(), math.Inf(-1), -1, 0, 1, math.Inf(1)} { + values = append(values, tree.NewDFloat(tree.DFloat(x))) + } + for i, x := range values { + for j, y := range values { + expected := 0 + if i < j { + expected = -1 + } else if i > j { + expected = 1 + } + evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + defer evalCtx.Stop(context.Background()) + got := x.Compare(evalCtx, y) + if got != expected { + t.Errorf("comparing DFloats %s and %s: expected %d, got %d", x, y, expected, got) + } + } + } +} + +// TestParseDIntervalWithTypeMetadata tests that the additional features available +// to tree.ParseDIntervalWithTypeMetadata beyond those in tree.ParseDInterval behave as expected. +func TestParseDIntervalWithTypeMetadata(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var ( + second = types.IntervalTypeMetadata{ + DurationField: types.IntervalDurationField{ + DurationType: types.IntervalDurationType_SECOND, + }, + } + minute = types.IntervalTypeMetadata{ + DurationField: types.IntervalDurationField{ + DurationType: types.IntervalDurationType_MINUTE, + }, + } + hour = types.IntervalTypeMetadata{ + DurationField: types.IntervalDurationField{ + DurationType: types.IntervalDurationType_HOUR, + }, + } + day = types.IntervalTypeMetadata{ + DurationField: types.IntervalDurationField{ + DurationType: types.IntervalDurationType_DAY, + }, + } + month = types.IntervalTypeMetadata{ + DurationField: types.IntervalDurationField{ + DurationType: types.IntervalDurationType_MONTH, + }, + } + year = types.IntervalTypeMetadata{ + DurationField: types.IntervalDurationField{ + DurationType: types.IntervalDurationType_YEAR, + }, + } + ) + + testData := []struct { + str string + dtype types.IntervalTypeMetadata + expected string + }{ + // Test cases for raw numbers with fields + {"5", second, "5s"}, + {"5.8", second, "5.8s"}, + {"5", minute, "5m"}, + {"5.8", minute, "5m"}, + {"5", hour, "5h"}, + {"5.8", hour, "5h"}, + {"5", day, "5 day"}, + {"5.8", day, "5 day"}, + {"5", month, "5 month"}, + {"5.8", month, "5 month"}, + {"5", year, "5 year"}, + {"5.8", year, "5 year"}, + // Test cases for truncation based on fields + {"1-2 3 4:56:07", second, "1-2 3 4:56:07"}, + {"1-2 3 4:56:07", minute, "1-2 3 4:56:00"}, + {"1-2 3 4:56:07", hour, "1-2 3 4:00:00"}, + {"1-2 3 4:56:07", day, "1-2 3 0:"}, + {"1-2 3 4:56:07", month, "1-2 0 0:"}, + {"1-2 3 4:56:07", year, "1 year"}, + } + for _, td := range testData { + actual, err := tree.ParseDIntervalWithTypeMetadata(td.str, td.dtype) + if err != nil { + t.Errorf("unexpected error while parsing INTERVAL %s %#v: %s", td.str, td.dtype, err) + continue + } + expected, err := tree.ParseDInterval(td.expected) + if err != nil { + t.Errorf("unexpected error while parsing expected value INTERVAL %s: %s", td.expected, err) + continue + } + evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + defer evalCtx.Stop(context.Background()) + if expected.Compare(evalCtx, actual) != 0 { + t.Errorf("INTERVAL %s %#v: got %s, expected %s", td.str, td.dtype, actual, expected) + } + } +} + +func TestParseDDate(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := testParseTimeContext( + time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.FixedZone("foo", -18000)), + ) + + testData := []struct { + str string + expected string + expectedDepOnCtx bool + }{ + {"now", "2001-02-03", true}, + {"today", "2001-02-03", true}, + {"tomorrow", "2001-02-04", true}, + {"yesterday", "2001-02-02", true}, + {"2017-03-03 01:00:00.00000", "2017-03-03", false}, + {"2017-03-03 01:00:00.00000-05", "2017-03-03", false}, + {"2017-03-03 01:00:00.00000+05", "2017-03-03", false}, + {"2017-03-03 -01:00:00", "2017-03-03", false}, + {"2017-03-03 -01:00:00 America/New_York", "2017-03-03", false}, + {"2017-03-03 -1:0:0", "2017-03-03", false}, + {"2017-03-03 -01:00", "2017-03-03", false}, + {"2017-03-03 -01", "2017-03-03", false}, + {"2017-03-03 -010000", "2017-03-03", false}, + {"2017-03-03 -0100", "2017-03-03", false}, + {"2017-03-03 -1", "2017-03-03", false}, + {"2017-03-03", "2017-03-03", false}, + {"2017-3-3 -01:00:00", "2017-03-03", false}, + {"2017-3-3 -1:0:0", "2017-03-03", false}, + {"2017-3-3 -01:00", "2017-03-03", false}, + {"2017-3-3 -01", "2017-03-03", false}, + {"2017-3-3 -010000", "2017-03-03", false}, + {"2017-3-3 -0100", "2017-03-03", false}, + {"2017-3-3 -1", "2017-03-03", false}, + {"2017-3-3", "2017-03-03", false}, + } + for _, td := range testData { + actual, depOnCtx, err := tree.ParseDDate(ctx, td.str) + if err != nil { + t.Errorf("unexpected error while parsing DATE %s: %s", td.str, err) + continue + } + expected, _, err := tree.ParseDDate(nil, td.expected) + if err != nil { + t.Errorf("unexpected error while parsing expected value DATE %s: %s", td.expected, err) + continue + } + evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + defer evalCtx.Stop(context.Background()) + if expected.Compare(evalCtx, actual) != 0 { + t.Errorf("DATE %s: got %s, expected %s", td.str, actual, expected) + } + if td.expectedDepOnCtx != depOnCtx { + t.Errorf("DATE %s: expected depOnCtx=%v", td.str, td.expectedDepOnCtx) + } + } +} + +func TestParseDBool(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + testData := []struct { + str string + expected *tree.DBool + err bool + }{ + {str: "t", expected: tree.DBoolTrue}, + {str: "tr", expected: tree.DBoolTrue}, + {str: "tru", expected: tree.DBoolTrue}, + {str: "true", expected: tree.DBoolTrue}, + {str: "tr", expected: tree.DBoolTrue}, + {str: "TRUE", expected: tree.DBoolTrue}, + {str: "tRUe", expected: tree.DBoolTrue}, + {str: " tRUe ", expected: tree.DBoolTrue}, + {str: " tR ", expected: tree.DBoolTrue}, + {str: "on", expected: tree.DBoolTrue}, + {str: "On", expected: tree.DBoolTrue}, + {str: "oN", expected: tree.DBoolTrue}, + {str: "ON", expected: tree.DBoolTrue}, + {str: "1", expected: tree.DBoolTrue}, + {str: "yes", expected: tree.DBoolTrue}, + {str: "ye", expected: tree.DBoolTrue}, + {str: "y", expected: tree.DBoolTrue}, + + {str: "false", expected: tree.DBoolFalse}, + {str: "FALSE", expected: tree.DBoolFalse}, + {str: "fALse", expected: tree.DBoolFalse}, + {str: "f", expected: tree.DBoolFalse}, + {str: "off", expected: tree.DBoolFalse}, + {str: "Off", expected: tree.DBoolFalse}, + {str: "oFF", expected: tree.DBoolFalse}, + {str: "OFF", expected: tree.DBoolFalse}, + {str: "0", expected: tree.DBoolFalse}, + + {str: "foo", err: true}, + {str: "tr ue", err: true}, + {str: "o", err: true}, + {str: "", err: true}, + {str: " ", err: true}, + {str: " ", err: true}, + } + + for _, td := range testData { + t.Run(td.str, func(t *testing.T) { + result, err := tree.ParseDBool(td.str) + if td.err { + if err == nil { + t.Fatalf("expected parsing %v to error, got %v", td.str, result) + } + return + } + if err != nil { + t.Fatalf("expected parsing %v to be %s, got error: %s", td.str, td.expected, err) + } + if *td.expected != *result { + t.Fatalf("expected parsing %v to be %s, got %s", td.str, td.expected, result) + } + }) + } +} + +func TestParseDTime(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := testParseTimeContext( + time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.FixedZone("foo", -18000)), + ) + // Since ParseDTime shares most of the underlying parsing logic to + // ParseDTimestamp, we only test a subset of the timestamp test cases. + testData := []struct { + str string + precision time.Duration + expected timeofday.TimeOfDay + expectedDepOnCtx bool + }{ + {"now", time.Microsecond, timeofday.New(4, 5, 6, 1), true}, + {" 04:05:06 ", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, + {"04:05:06", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, + {"04:05:06.000001", time.Microsecond, timeofday.New(4, 5, 6, 1), false}, + {"04:05:06.000001+00", time.Microsecond, timeofday.New(4, 5, 6, 1), false}, + {"04:05:06.000001-05", time.Microsecond, timeofday.New(4, 5, 6, 1), false}, + {"04:05:06.000001+05", time.Microsecond, timeofday.New(4, 5, 6, 1), false}, + {"04:05:06.000001", time.Second, timeofday.New(4, 5, 6, 0), false}, + {"04:05:06-07", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, + {"0000-01-01 04:05:06", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, + {"2001-01-01 04:05:06", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, + {"4:5:6", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, + {"24:00:00", time.Microsecond, timeofday.Time2400, false}, + {"24:00:00.000", time.Microsecond, timeofday.Time2400, false}, + {"24:00:00.000000", time.Microsecond, timeofday.Time2400, false}, + {"0000-01-01T24:00:00", time.Microsecond, timeofday.Time2400, false}, + {"0000-01-01T24:00:00.0", time.Microsecond, timeofday.Time2400, false}, + {"0000-01-01 24:00:00", time.Microsecond, timeofday.Time2400, false}, + {"0000-01-01 24:00:00.0", time.Microsecond, timeofday.Time2400, false}, + {" 24:00:00.0", time.Microsecond, timeofday.Time2400, false}, + {" 24:00:00.0 ", time.Microsecond, timeofday.Time2400, false}, + } + for _, td := range testData { + actual, depOnCtx, err := tree.ParseDTime(ctx, td.str, td.precision) + if err != nil { + t.Errorf("unexpected error while parsing TIME %s: %s", td.str, err) + continue + } + if *actual != tree.DTime(td.expected) { + t.Errorf("TIME %s: got %s, expected %s", td.str, actual, td.expected) + } + if td.expectedDepOnCtx != depOnCtx { + t.Errorf("TIME %s: expected depOnCtx=%v", td.str, td.expectedDepOnCtx) + } + } +} + +func TestParseDTimeError(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + testData := []string{ + "", + "foo", + "01", + "today", + "yesterday", + + // TODO(radu): these exceptions seem dubious. They work in postgres. + "24:00:00.000000+00", + "24:00:00.000000-05", + "24:00:00.000000+05", + } + for _, s := range testData { + actual, _, _ := tree.ParseDTime(nil, s, time.Microsecond) + if actual != nil { + t.Errorf("TIME %s: got %s, expected error", s, actual) + } + } +} + +func TestParseDTimeTZ(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := testParseTimeContext( + time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.FixedZone("foo", 18000)), + ) + + mk := func(hour, min, sec, micro int, offset int32) timetz.TimeTZ { + return timetz.MakeTimeTZ(timeofday.New(hour, min, sec, micro), offset) + } + + testData := []struct { + str string + precision time.Duration + expected timetz.TimeTZ + expectedDepOnCtx bool + }{ + {" 04:05:06 ", time.Microsecond, mk(4, 5, 6, 0, -18000), true}, + {"04:05:06", time.Microsecond, mk(4, 5, 6, 0, -18000), true}, + {"04:05:06.000001", time.Microsecond, mk(4, 5, 6, 1, -18000), true}, + {"04:05:06.000001", time.Second, mk(4, 5, 6, 0, -18000), true}, + {"04:05:06.000001+00", time.Microsecond, mk(4, 5, 6, 1, 0), false}, + {"04:05:06.000001-04", time.Microsecond, mk(4, 5, 6, 1, 4*3600), false}, + {"04:05:06.000001+04", time.Microsecond, mk(4, 5, 6, 1, -4*3600), false}, + {"04:05:06-07", time.Microsecond, mk(4, 5, 6, 0, 7*3600), false}, + {"0000-01-01 04:05:06", time.Microsecond, mk(4, 5, 6, 0, -18000), true}, + {"2001-01-01 04:05:06", time.Microsecond, mk(4, 5, 6, 0, -18000), true}, + {"4:5:6", time.Microsecond, mk(4, 5, 6, 0, -18000), true}, + {"24:00:00", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, + {"24:00:00.000", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, + {"24:00:00.000000", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, + {"24:00:00.000000+00", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, 0), false}, + {"24:00:00.000000-04", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, 4*3600), false}, + {"24:00:00.000000+04", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -4*3600), false}, + {"0000-01-01T24:00:00", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, + {"0000-01-01T24:00:00.0", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, + {"0000-01-01 24:00:00", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, + {"0000-01-01 24:00:00.0", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, + {" 24:00:00.0", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, + {" 24:00:00.0 ", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, + } + for _, td := range testData { + actual, depOnCtx, err := tree.ParseDTimeTZ(ctx, td.str, td.precision) + if err != nil { + t.Errorf("unexpected error while parsing TIME %s: %s", td.str, err) + continue + } + exp := tree.DTimeTZ{TimeTZ: td.expected} + if *actual != exp { + t.Errorf("TIMETZ %s: got %s, expected %s", td.str, actual, &exp) + } + if td.expectedDepOnCtx != depOnCtx { + t.Errorf("TIME %s: expected depOnCtx=%v", td.str, td.expectedDepOnCtx) + } + } +} + +func TestParseDTimeTZError(t *testing.T) { + defer leaktest.AfterTest(t)() + testData := []string{ + "", + "foo", + "01", + "today", + "yesterday", + + // TODO(radu): this should work. + "now", + } + for _, s := range testData { + actual, _, _ := tree.ParseDTimeTZ(nil, s, time.Microsecond) + if actual != nil { + t.Errorf("TIMETZ %s: got %s, expected error", s, actual) + } + } +} + +func TestParseDTimestamp(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := testParseTimeContext( + time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.FixedZone("foo", -18000)), + ) + + testData := []struct { + str string + expected time.Time + expectedDepOnCtx bool + }{ + {"now", time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.UTC), true}, + {"today", time.Date(2001, time.February, 3, 0, 0, 0, 0, time.UTC), true}, + {"tomorrow", time.Date(2001, time.February, 4, 0, 0, 0, 0, time.UTC), true}, + {"yesterday", time.Date(2001, time.February, 2, 0, 0, 0, 0, time.UTC), true}, + {"2001-02-03", time.Date(2001, time.February, 3, 0, 0, 0, 0, time.UTC), false}, + {"2001-02-03 04:05:06", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.UTC), false}, + {"2001-02-03 04:05:06.000001", time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.UTC), false}, + {"2001-02-03 04:05:06.00001", time.Date(2001, time.February, 3, 4, 5, 6, 10000, time.UTC), false}, + {"2001-02-03 04:05:06.0001", time.Date(2001, time.February, 3, 4, 5, 6, 100000, time.UTC), false}, + {"2001-02-03 04:05:06.001", time.Date(2001, time.February, 3, 4, 5, 6, 1000000, time.UTC), false}, + {"2001-02-03 04:05:06.01", time.Date(2001, time.February, 3, 4, 5, 6, 10000000, time.UTC), false}, + {"2001-02-03 04:05:06.1", time.Date(2001, time.February, 3, 4, 5, 6, 100000000, time.UTC), false}, + {"2001-02-03 04:05:06.12", time.Date(2001, time.February, 3, 4, 5, 6, 120000000, time.UTC), false}, + {"2001-02-03 04:05:06.123", time.Date(2001, time.February, 3, 4, 5, 6, 123000000, time.UTC), false}, + {"2001-02-03 04:05:06.1234", time.Date(2001, time.February, 3, 4, 5, 6, 123400000, time.UTC), false}, + {"2001-02-03 04:05:06.12345", time.Date(2001, time.February, 3, 4, 5, 6, 123450000, time.UTC), false}, + {"2001-02-03 04:05:06.123456", time.Date(2001, time.February, 3, 4, 5, 6, 123456000, time.UTC), false}, + {"2001-02-03 04:05:06.123-07", time.Date(2001, time.February, 3, 4, 5, 6, 123000000, time.UTC), false}, + {"2001-02-03 04:05:06-07", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.UTC), false}, + {"2001-02-03 04:05:06-07:42", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.UTC), false}, + {"2001-02-03 04:05:06-07:30:09", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.UTC), false}, + {"2001-02-03 04:05:06+07", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.UTC), false}, + {"2001-02-03 04:0:06", time.Date(2001, time.February, 3, 4, 0, 6, 0, time.UTC), false}, + {"2001-02-03 0:0:06", time.Date(2001, time.February, 3, 0, 0, 6, 0, time.UTC), false}, + {"2001-02-03 4:05:0", time.Date(2001, time.February, 3, 4, 5, 0, 0, time.UTC), false}, + {"2001-02-03 4:05:0-07:0:00", time.Date(2001, time.February, 3, 4, 5, 0, 0, time.UTC), false}, + {"2001-02-03 4:0:6 +3:0:0", time.Date(2001, time.February, 3, 4, 0, 6, 0, time.UTC), false}, + } + for _, td := range testData { + actual, depOnCtx, err := tree.ParseDTimestamp(ctx, td.str, time.Nanosecond) + if err != nil { + t.Errorf("unexpected error while parsing TIMESTAMP %s: %s", td.str, err) + continue + } + if !actual.Time.Equal(td.expected) { + t.Errorf("TIMESTAMP %s: got %s, expected %s", td.str, actual, td.expected) + } + if td.expectedDepOnCtx != depOnCtx { + t.Errorf("TIMESTAMP %s: expected depOnCtx=%v", td.str, td.expectedDepOnCtx) + } + } +} + +func TestParseDTimestampTZ(t *testing.T) { + defer leaktest.AfterTest(t)() + + local := time.FixedZone("foo", -18000) + ctx := testParseTimeContext(time.Date(2001, time.February, 3, 4, 5, 6, 1000, local)) + + testData := []struct { + str string + expected time.Time + expectedDepOnCtx bool + }{ + {"now", time.Date(2001, time.February, 3, 4, 5, 6, 1000, local), true}, + {"today", time.Date(2001, time.February, 3, 0, 0, 0, 0, local), true}, + {"tomorrow", time.Date(2001, time.February, 4, 0, 0, 0, 0, local), true}, + {"yesterday", time.Date(2001, time.February, 2, 0, 0, 0, 0, local), true}, + {"2001-02-03", time.Date(2001, time.February, 3, 0, 0, 0, 0, local), true}, + {"2001-02-03 04:05:06", time.Date(2001, time.February, 3, 4, 5, 6, 0, local), true}, + {"2001-02-03 04:05:06.000001", time.Date(2001, time.February, 3, 4, 5, 6, 1000, local), true}, + {"2001-02-03 04:05:06.00001", time.Date(2001, time.February, 3, 4, 5, 6, 10000, local), true}, + {"2001-02-03 04:05:06.0001", time.Date(2001, time.February, 3, 4, 5, 6, 100000, local), true}, + {"2001-02-03 04:05:06.001", time.Date(2001, time.February, 3, 4, 5, 6, 1000000, local), true}, + {"2001-02-03 04:05:06.01", time.Date(2001, time.February, 3, 4, 5, 6, 10000000, local), true}, + {"2001-02-03 04:05:06.1", time.Date(2001, time.February, 3, 4, 5, 6, 100000000, local), true}, + {"2001-02-03 04:05:06.12", time.Date(2001, time.February, 3, 4, 5, 6, 120000000, local), true}, + {"2001-02-03 04:05:06.123", time.Date(2001, time.February, 3, 4, 5, 6, 123000000, local), true}, + {"2001-02-03 04:05:06.1234", time.Date(2001, time.February, 3, 4, 5, 6, 123400000, local), true}, + {"2001-02-03 04:05:06.12345", time.Date(2001, time.February, 3, 4, 5, 6, 123450000, local), true}, + {"2001-02-03 04:05:06.123456", time.Date(2001, time.February, 3, 4, 5, 6, 123456000, local), true}, + {"2001-02-03 04:05:06.123-07", time.Date(2001, time.February, 3, 4, 5, 6, 123000000, time.FixedZone("", -7*3600)), false}, + {"2001-02-03 04:05:06-07", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.FixedZone("", -7*3600)), false}, + {"2001-02-03 04:05:06-07:42", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.FixedZone("", -7*3600-42*60)), false}, + {"2001-02-03 04:05:06-07:30:09", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.FixedZone("", -7*3600-30*60-9)), false}, + {"2001-02-03 04:05:06+07", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.FixedZone("", 7*3600)), false}, + {"2001-02-03 04:0:06", time.Date(2001, time.February, 3, 4, 0, 6, 0, local), true}, + {"2001-02-03 0:0:06", time.Date(2001, time.February, 3, 0, 0, 6, 0, local), true}, + {"2001-02-03 4:05:0", time.Date(2001, time.February, 3, 4, 5, 0, 0, local), true}, + {"2001-02-03 4:05:0-07:0:00", time.Date(2001, time.February, 3, 4, 5, 0, 0, time.FixedZone("", -7*3600)), false}, + {"2001-02-03 4:0:6 +3:0:0", time.Date(2001, time.February, 3, 4, 0, 6, 0, time.FixedZone("", 3*3600)), false}, + } + for _, td := range testData { + actual, depOnCtx, err := tree.ParseDTimestampTZ(ctx, td.str, time.Nanosecond) + if err != nil { + t.Errorf("unexpected error while parsing TIMESTAMP %s: %s", td.str, err) + continue + } + if !actual.Time.Equal(td.expected) { + t.Errorf("TIMESTAMPTZ %s: got %s, expected %s", td.str, actual, td.expected) + } + if td.expectedDepOnCtx != depOnCtx { + t.Errorf("TIMESTAMPTZ %s: expected depOnCtx=%v", td.str, td.expectedDepOnCtx) + } + } +} + +func TestMakeDJSON(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + j1, err := tree.MakeDJSON(1) + if err != nil { + t.Fatal(err) + } + j2, err := tree.MakeDJSON(2) + if err != nil { + t.Fatal(err) + } + if j1.Compare(tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()), j2) != -1 { + t.Fatal("expected JSON 1 < 2") + } +} + +func TestDTimeTZ(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := &tree.EvalContext{ + SessionData: &sessiondata.SessionData{ + Location: time.UTC, + }, + } + + maxTime, depOnCtx, err := tree.ParseDTimeTZ(ctx, "24:00:00-1559", time.Microsecond) + require.NoError(t, err) + require.False(t, depOnCtx) + minTime, depOnCtx, err := tree.ParseDTimeTZ(ctx, "00:00:00+1559", time.Microsecond) + require.NoError(t, err) + require.False(t, depOnCtx) + + // These are all the same UTC time equivalents. + utcTime, depOnCtx, err := tree.ParseDTimeTZ(ctx, "11:14:15+0", time.Microsecond) + require.NoError(t, err) + require.False(t, depOnCtx) + sydneyTime, depOnCtx, err := tree.ParseDTimeTZ(ctx, "21:14:15+10", time.Microsecond) + require.NoError(t, err) + require.False(t, depOnCtx) + + // No daylight savings in Hawaii! + hawaiiZone, err := timeutil.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(ctx) + 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(ctx, largerThan), "%s > %s", tc.t.String(), largerThan.String()) + } + + var smallerThan []tree.Datum + next, ok := tc.t.Next(ctx) + 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(ctx, smallerThan), "%s < %s", tc.t.String(), smallerThan.String()) + } + + for _, equalTo := range tc.equalTo { + assert.Equal(t, 0, tc.t.Compare(ctx, equalTo), "%s = %s", tc.t.String(), equalTo.String()) + } + + assert.Equal(t, tc.isMax, tc.t.IsMax(ctx)) + assert.Equal(t, tc.isMin, tc.t.IsMin(ctx)) + }) + } +} + +func TestIsDistinctFrom(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + testData := []struct { + a string // comma separated list of strings, `NULL` is converted to a NULL + b string // same as a + expected bool + }{ + {"a", "a", false}, + {"a", "b", true}, + {"b", "b", false}, + {"a,a", "a,a", false}, + {"a,a", "a,b", true}, + {"a,a", "b,a", true}, + {"a,a,a", "a,a,a", false}, + {"a,a,a", "a,a,b", true}, + {"a,a,a", "a,b,a", true}, + {"a,a,a", "a,b,b", true}, + {"a,a,a", "b,a,a", true}, + {"a,a,a", "b,a,b", true}, + {"a,a,a", "b,b,a", true}, + {"a,a,a", "b,b,b", true}, + {"NULL", "NULL", false}, + {"a", "NULL", true}, + {"a,a", "a,NULL", true}, + {"a,a", "NULL,a", true}, + {"a,a", "NULL,NULL", true}, + {"a,NULL", "a,a", true}, + {"a,NULL", "a,NULL", false}, + {"a,NULL", "NULL,a", true}, + {"a,NULL", "NULL,NULL", true}, + {"NULL,a", "a,a", true}, + {"NULL,a", "a,NULL", true}, + {"NULL,a", "NULL,a", false}, + {"NULL,a", "NULL,NULL", true}, + {"NULL,NULL", "a,a", true}, + {"NULL,NULL", "a,NULL", true}, + {"NULL,NULL", "NULL,a", true}, + {"NULL,NULL", "NULL,NULL", false}, + {"a,a,a", "a,a,NULL", true}, + {"a,a,a", "a,NULL,a", true}, + {"a,a,a", "a,NULL,NULL", true}, + {"a,a,a", "NULL,a,a", true}, + {"a,a,a", "NULL,a,NULL", true}, + {"a,a,a", "NULL,NULL,a", true}, + {"a,a,a", "NULL,NULL,NULL", true}, + {"a,NULL,a", "a,a,a", true}, + {"a,NULL,a", "a,a,NULL", true}, + {"a,NULL,a", "a,NULL,a", false}, + {"a,NULL,a", "a,NULL,NULL", true}, + {"a,NULL,a", "NULL,a,a", true}, + {"a,NULL,a", "NULL,a,NULL", true}, + {"a,NULL,a", "NULL,NULL,a", true}, + {"a,NULL,a", "NULL,NULL,NULL", true}, + {"NULL,a,NULL", "a,a,a", true}, + {"NULL,a,NULL", "a,a,NULL", true}, + {"NULL,a,NULL", "a,NULL,a", true}, + {"NULL,a,NULL", "a,NULL,NULL", true}, + {"NULL,a,NULL", "NULL,a,a", true}, + {"NULL,a,NULL", "NULL,a,NULL", false}, + {"NULL,a,NULL", "NULL,NULL,a", true}, + {"NULL,a,NULL", "NULL,NULL,NULL", true}, + {"NULL,NULL,NULL", "a,a,a", true}, + {"NULL,NULL,NULL", "a,a,NULL", true}, + {"NULL,NULL,NULL", "a,NULL,a", true}, + {"NULL,NULL,NULL", "a,NULL,NULL", true}, + {"NULL,NULL,NULL", "NULL,a,a", true}, + {"NULL,NULL,NULL", "NULL,a,NULL", true}, + {"NULL,NULL,NULL", "NULL,NULL,a", true}, + {"NULL,NULL,NULL", "NULL,NULL,NULL", false}, + } + convert := func(s string) tree.Datums { + splits := strings.Split(s, ",") + result := make(tree.Datums, len(splits)) + for i, value := range splits { + if value == "NULL" { + result[i] = tree.DNull + continue + } + result[i] = tree.NewDString(value) + } + return result + } + for _, td := range testData { + t.Run(fmt.Sprintf("%s to %s", td.a, td.b), func(t *testing.T) { + datumsA := convert(td.a) + datumsB := convert(td.b) + if e, a := td.expected, datumsA.IsDistinctFrom(&tree.EvalContext{}, datumsB); e != a { + if e { + t.Errorf("expected %s to be distinct from %s, but got %t", datumsA, datumsB, e) + } else { + t.Errorf("expected %s to not be distinct from %s, but got %t", datumsA, datumsB, e) + } + } + }) + } +} + +func TestAllTypesAsJSON(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + for _, typ := range types.Scalar { + d := tree.SampleDatum(typ) + _, err := tree.AsJSON(d, time.UTC) + if err != nil { + t.Errorf("couldn't convert %s to JSON: %s", d, err) + } + } +} + +// Test default values of many different datum types. +func TestNewDefaultDatum(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + defer evalCtx.Stop(context.Background()) + + testCases := []struct { + t *types.T + expected string + }{ + {t: types.Bool, expected: "false"}, + {t: types.Int, expected: "0:::INT8"}, + {t: types.Int2, expected: "0:::INT8"}, + {t: types.Int4, expected: "0:::INT8"}, + {t: types.Float, expected: "0.0:::FLOAT8"}, + {t: types.Float4, expected: "0.0:::FLOAT8"}, + {t: types.Decimal, expected: "0:::DECIMAL"}, + {t: types.MakeDecimal(10, 5), expected: "0:::DECIMAL"}, + {t: types.Date, expected: "'2000-01-01':::DATE"}, + {t: types.Timestamp, expected: "'0001-01-01 00:00:00':::TIMESTAMP"}, + {t: types.Interval, expected: "'00:00:00':::INTERVAL"}, + {t: types.String, expected: "'':::STRING"}, + {t: types.MakeChar(3), expected: "'':::STRING"}, + {t: types.Bytes, expected: "'\\x':::BYTES"}, + {t: types.TimestampTZ, expected: "'0001-01-01 00:00:00+00:00':::TIMESTAMPTZ"}, + {t: types.MakeCollatedString(types.MakeVarChar(10), "de"), expected: "'' COLLATE de"}, + {t: types.MakeCollatedString(types.VarChar, "en_US"), expected: "'' COLLATE en_US"}, + {t: types.Oid, expected: "26:::OID"}, + {t: types.RegClass, expected: "crdb_internal.create_regclass(2205,'regclass'):::REGCLASS"}, + {t: types.Unknown, expected: "NULL"}, + {t: types.Uuid, expected: "'00000000-0000-0000-0000-000000000000':::UUID"}, + {t: types.MakeArray(types.Int), expected: "ARRAY[]:::INT8[]"}, + {t: types.MakeArray(types.MakeArray(types.String)), expected: "ARRAY[]:::STRING[][]"}, + {t: types.OidVector, expected: "ARRAY[]:::OID[]"}, + {t: types.INet, expected: "'0.0.0.0/0':::INET"}, + {t: types.Time, expected: "'00:00:00':::TIME"}, + {t: types.Jsonb, expected: "'null':::JSONB"}, + {t: types.TimeTZ, expected: "'00:00:00+00:00:00':::TIMETZ"}, + {t: types.MakeTuple([]*types.T{}), expected: "()"}, + {t: types.MakeTuple([]*types.T{types.Int, types.MakeChar(1)}), expected: "(0:::INT8, '':::STRING)"}, + {t: types.MakeTuple([]*types.T{types.OidVector, types.MakeTuple([]*types.T{types.Float})}), expected: "(ARRAY[]:::OID[], (0.0:::FLOAT8,))"}, + {t: types.VarBit, expected: "B''"}, + {t: types.MakeBit(5), expected: "B''"}, + } + + for i, tc := range testCases { + t.Run(fmt.Sprintf("#%d %s", i, tc.t.SQLString()), func(t *testing.T) { + datum, err := tree.NewDefaultDatum(evalCtx, tc.t) + if err != nil { + t.Errorf("unexpected error: %s", err) + } + + actual := tree.AsStringWithFlags(datum, tree.FmtCheckEquivalence) + if actual != tc.expected { + t.Errorf("expected %s, got %s", tc.expected, actual) + } + }) + } +} + +type testParseTimeContext time.Time + +var _ tree.ParseTimeContext = testParseTimeContext{} + +func (t testParseTimeContext) GetRelativeParseTime() time.Time { + return time.Time(t) +} + +func TestGeospatialSize(t *testing.T) { + defer leaktest.AfterTest(t)() + testCases := []struct { + wkt string + expected uintptr + }{ + {"SRID=4004;POINT EMPTY", 73}, + {"SRID=4326;LINESTRING(0 0, 10 0)", 125}, + } + + for _, tc := range testCases { + t.Run(tc.wkt, func(t *testing.T) { + t.Run("geometry", func(t *testing.T) { + g, err := tree.ParseDGeometry(tc.wkt) + require.NoError(t, err) + require.Equal(t, tc.expected, g.Size()) + }) + t.Run("geography", func(t *testing.T) { + g, err := tree.ParseDGeography(tc.wkt) + require.NoError(t, err) + require.Equal(t, tc.expected, g.Size()) + }) + }) + } +} diff --git a/pkg/sql/sem/tree/datum_invariants_test.go b/pkg/sql/sem/tree/datum_invariants_test.go index 35fc83056a88..5d39871c28a7 100644 --- a/pkg/sql/sem/tree/datum_invariants_test.go +++ b/pkg/sql/sem/tree/datum_invariants_test.go @@ -12,15 +12,10 @@ package tree import ( "testing" - "time" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/timeofday" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestAllTypesCastableToString(t *testing.T) { @@ -42,119 +37,3 @@ func TestAllTypesCastableFromString(t *testing.T) { } } } - -func TestCompareTimestamps(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - pacificTimeZone := int32(7 * 60 * 60) - sydneyTimeZone := int32(-10 * 60 * 60) - - sydneyFixedZone := time.FixedZone("otan@sydney", -int(sydneyTimeZone)) - // kiwiFixedZone is 2 hours ahead of Sydney. - kiwiFixedZone := time.FixedZone("otan@auckland", -int(sydneyTimeZone)+2*60*60) - - ddate, err := NewDDateFromTime(time.Date(2019, time.November, 22, 0, 0, 0, 0, time.UTC)) - require.NoError(t, err) - - testCases := []struct { - desc string - left Datum - right Datum - location *time.Location - 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: "DTime and DTimeTZ both UTC, and so are equal", - left: MakeDTime(timeofday.New(12, 0, 0, 0)), - right: NewDTimeTZFromOffset(timeofday.New(12, 0, 0, 0), 0), - expected: 0, - }, - { - desc: "DTime and DTimeTZ both Sydney time, and so are equal", - left: MakeDTime(timeofday.New(12, 0, 0, 0)), - right: NewDTimeTZFromOffset(timeofday.New(12, 0, 0, 0), sydneyTimeZone), - location: sydneyFixedZone, - expected: 0, - }, - { - desc: "DTimestamp and DTimestampTZ (Sydney) equal in Sydney zone", - left: MustMakeDTimestamp(time.Date(2019, time.November, 22, 10, 0, 0, 0, time.UTC), time.Microsecond), - right: MustMakeDTimestampTZ(time.Date(2019, time.November, 22, 10, 0, 0, 0, sydneyFixedZone), time.Microsecond), - location: sydneyFixedZone, - expected: 0, - }, - { - desc: "DTimestamp and DTimestampTZ (Sydney) equal in Sydney+2 zone", - left: MustMakeDTimestamp(time.Date(2019, time.November, 22, 12, 0, 0, 0, time.UTC), time.Microsecond), - right: MustMakeDTimestampTZ(time.Date(2019, time.November, 22, 10, 0, 0, 0, sydneyFixedZone), time.Microsecond), - location: kiwiFixedZone, - expected: 0, - }, - { - desc: "Date and DTimestampTZ (Sydney) equal in Sydney zone", - left: ddate, - right: MustMakeDTimestampTZ(time.Date(2019, time.November, 22, 0, 0, 0, 0, sydneyFixedZone), time.Microsecond), - location: sydneyFixedZone, - expected: 0, - }, - { - desc: "Date and DTimestampTZ (Sydney) equal in Sydney+2 zone", - left: ddate, - right: MustMakeDTimestampTZ(time.Date(2019, time.November, 21, 22, 0, 0, 0, sydneyFixedZone), time.Microsecond), - location: kiwiFixedZone, - 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( - tc.desc, - func(t *testing.T) { - ctx := &EvalContext{ - SessionData: &sessiondata.SessionData{ - Location: tc.location, - }, - } - assert.Equal(t, tc.expected, compareTimestamps(ctx, tc.left, tc.right)) - assert.Equal(t, -tc.expected, compareTimestamps(ctx, tc.right, tc.left)) - }, - ) - } -} diff --git a/pkg/sql/sem/tree/datum_test.go b/pkg/sql/sem/tree/datum_test.go index a4d90c0ccdc6..9150eafa1773 100644 --- a/pkg/sql/sem/tree/datum_test.go +++ b/pkg/sql/sem/tree/datum_test.go @@ -1,4 +1,4 @@ -// Copyright 2016 The Cockroach Authors. +// Copyright 2020 The Cockroach Authors. // // Use of this software is governed by the Business Source License // included in the file licenses/BSL.txt. @@ -8,1148 +8,207 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package tree_test +package tree import ( - "context" - "fmt" - "math" - "strings" "testing" "time" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/parser" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeofday" - "github.com/cockroachdb/cockroach/pkg/util/timetz" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) -func prepareExpr(t *testing.T, datumExpr string) tree.Datum { - expr, err := parser.ParseExpr(datumExpr) - if err != nil { - t.Fatalf("%s: %v", datumExpr, err) - } - // Type checking ensures constant folding is performed and type - // annotations have come into effect. - ctx := context.Background() - sema := tree.MakeSemaContext() - typedExpr, err := tree.TypeCheck(ctx, expr, &sema, types.Any) - if err != nil { - t.Fatalf("%s: %v", datumExpr, err) - } - // Normalization ensures that casts are processed. - evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) - defer evalCtx.Stop(context.Background()) - typedExpr, err = evalCtx.NormalizeExpr(typedExpr) - if err != nil { - t.Fatalf("%s: %v", datumExpr, err) - } - d, err := typedExpr.Eval(evalCtx) - if err != nil { - t.Fatalf("%s: %v", datumExpr, err) - } - return d -} - -func TestDatumOrdering(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - const valIsMin = `min` - const valIsMax = `max` - const noPrev = `` - const noNext = `` - const noMin = `` - const noMax = `` - - testData := []struct { - datumExpr string - prev string - next string - min string - max string - }{ - // Integers - {`1`, `0`, `2`, `-9223372036854775808`, `9223372036854775807`}, - {`-9223372036854775808`, valIsMin, `-9223372036854775807`, `-9223372036854775808`, `9223372036854775807`}, - {`9223372036854775807`, `9223372036854775806`, valIsMax, `-9223372036854775808`, `9223372036854775807`}, - - // Boolean - {`true`, `false`, valIsMax, `false`, `true`}, - {`false`, valIsMin, `true`, `false`, `true`}, - - // Floats - {`3.14:::float`, `3.1399999999999997`, `3.1400000000000006`, `NaN`, `+Inf`}, - {`9.223372036854776e+18:::float`, `9.223372036854775e+18`, `9.223372036854778e+18`, `NaN`, `+Inf`}, - {`'NaN':::float`, valIsMin, `-Inf`, `NaN`, `+Inf`}, - {`-1.7976931348623157e+308:::float`, `-Inf`, `-1.7976931348623155e+308`, `NaN`, `+Inf`}, - {`1.7976931348623157e+308:::float`, `1.7976931348623155e+308`, `+Inf`, `NaN`, `+Inf`}, - - // Decimal - {`1.0:::decimal`, noPrev, noNext, `NaN`, `Infinity`}, - - // Strings and byte arrays - {`'':::string`, valIsMin, `e'\x00'`, `''`, noMax}, - {`e'\x00'`, noPrev, `e'\x00\x00'`, `''`, noMax}, - {`'abc':::string`, noPrev, `e'abc\x00'`, `''`, noMax}, - {`'':::bytes`, valIsMin, `'\x00'`, `'\x'`, noMax}, - {`'abc':::bytes`, noPrev, `'\x61626300'`, `'\x'`, noMax}, - - // Dates - {`'2006-01-02':::date`, `'2006-01-01'`, `'2006-01-03'`, `'-infinity'`, `'infinity'`}, - {`'0001-01-01':::date`, `'0001-12-31 BC'`, `'0001-01-02'`, `'-infinity'`, `'infinity'`}, - {`'4000-01-01 BC':::date`, `'4001-12-31 BC'`, `'4000-01-02 BC'`, `'-infinity'`, `'infinity'`}, - {`'2006-01-02 03:04:05.123123':::timestamp`, - `'2006-01-02 03:04:05.123122'`, `'2006-01-02 03:04:05.123124'`, `'-4713-11-24 00:00:00'`, `'294276-12-31 23:59:59.999999'`}, - - // Geospatial types - {`'BOX(1 2,3 4)'::box2d`, noPrev, noNext, noMin, noMax}, - {`'POINT(1.0 1.0)'::geometry`, noPrev, noNext, noMin, noMax}, - {`'POINT(1.0 1.0)'::geography`, noPrev, noNext, noMin, noMax}, - - // Times - {`'00:00:00':::time`, valIsMin, `'00:00:00.000001'`, - `'00:00:00'`, `'24:00:00'`}, - {`'12:00:00':::time`, `'11:59:59.999999'`, `'12:00:00.000001'`, - `'00:00:00'`, `'24:00:00'`}, - {`'24:00:00':::time`, `'23:59:59.999999'`, valIsMax, `'00:00:00'`, `'24:00:00'`}, - - // Intervals - {`'1 day':::interval`, noPrev, noNext, - `'-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 854775 us':::interval`, - noPrev, valIsMax, - `'-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 -854775 us':::interval`, - valIsMin, noNext, - `'-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, - `'00000000-0000-0000-0000-000000000000'`, `'ffffffff-ffff-ffff-ffff-ffffffffffff'`}, - {`'00000000-0000-0000-0000-000000000000'::uuid`, valIsMin, `'00000000-0000-0000-0000-000000000001'`, - `'00000000-0000-0000-0000-000000000000'`, `'ffffffff-ffff-ffff-ffff-ffffffffffff'`}, - {`'ffffffff-ffff-ffff-0000-000000000000'::uuid`, `'ffffffff-ffff-fffe-ffff-ffffffffffff'`, - `'ffffffff-ffff-ffff-0000-000000000001'`, `'00000000-0000-0000-0000-000000000000'`, - `'ffffffff-ffff-ffff-ffff-ffffffffffff'`}, - {`'00000000-0000-0000-ffff-ffffffffffff'::uuid`, `'00000000-0000-0000-ffff-fffffffffffe'`, - `'00000000-0000-0001-0000-000000000000'`, `'00000000-0000-0000-0000-000000000000'`, - `'ffffffff-ffff-ffff-ffff-ffffffffffff'`}, - - // INETs - {`'0.0.0.0'::inet`, `'255.255.255.255/31'`, `'0.0.0.1'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'0.0.0.0/0'::inet`, noPrev, `'0.0.0.1/0'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'192.168.255.255'::inet`, `'192.168.255.254'`, `'192.169.0.0'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'127.0.0.1'::inet`, `'127.0.0.0'`, `'127.0.0.2'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'192.168.0.1/20'::inet`, `'192.168.0.0/20'`, `'192.168.0.2/20'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'192.168.0.0/20'::inet`, `'192.167.255.255/20'`, `'192.168.0.1/20'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'::ffff:1.2.3.4'::inet`, `'::ffff:1.2.3.3'`, `'::ffff:1.2.3.5'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'::0'::inet`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/127'`, `'::1'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'::0/0'::inet`, `'255.255.255.255'`, `'::1/0'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'255.255.255.255/32'::inet`, `'255.255.255.254'`, `'::/0'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'255.255.255.255/16'::inet`, `'255.255.255.254/16'`, `'0.0.0.0/17'`, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - {`'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/128'::inet`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'`, noNext, `'0.0.0.0/0'`, `'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'`}, - - // NULL - {`NULL`, valIsMin, valIsMax, `NULL`, `NULL`}, - - // Tuples - {`row()`, valIsMin, valIsMax, `()`, `()`}, - - {`(NULL,)`, valIsMin, valIsMax, `(NULL,)`, `(NULL,)`}, - - {`(true,)`, `(false,)`, valIsMax, `(false,)`, `(true,)`}, - {`(false,)`, valIsMin, `(true,)`, `(false,)`, `(true,)`}, - - {`(true, false, false)`, `(false, true, true)`, `(true, false, true)`, - `(false, false, false)`, `(true, true, true)`}, - {`(false, true, true)`, `(false, true, false)`, `(true, NULL, NULL)`, - `(false, false, false)`, `(true, true, true)`}, - - {`(0, 0)`, `(0, -1)`, `(0, 1)`, - `(-9223372036854775808, -9223372036854775808)`, - `(9223372036854775807, 9223372036854775807)`}, - - {`(0, 9223372036854775807)`, - `(0, 9223372036854775806)`, `(1, NULL)`, - `(-9223372036854775808, -9223372036854775808)`, - `(9223372036854775807, 9223372036854775807)`}, - {`(9223372036854775807, 9223372036854775807)`, - `(9223372036854775807, 9223372036854775806)`, valIsMax, - `(-9223372036854775808, -9223372036854775808)`, - `(9223372036854775807, 9223372036854775807)`}, - - {`(0, 0:::decimal)`, noPrev, noNext, - `(-9223372036854775808, NaN)`, - `(9223372036854775807, Infinity)`}, - {`(0:::decimal, 0)`, `(0, -1)`, `(0, 1)`, - `(NaN, -9223372036854775808)`, - `(Infinity, 9223372036854775807)`}, - - {`(10, '')`, noPrev, `(10, e'\x00')`, - `(-9223372036854775808, '')`, noMax}, - {`(-9223372036854775808, '')`, valIsMin, `(-9223372036854775808, e'\x00')`, - `(-9223372036854775808, '')`, noMax}, - {`(-9223372036854775808, 'abc')`, noPrev, `(-9223372036854775808, e'abc\x00')`, - `(-9223372036854775808, '')`, noMax}, - - {`(10, NULL)`, `(9, NULL)`, `(11, NULL)`, - `(-9223372036854775808, NULL)`, `(9223372036854775807, NULL)`}, - {`(NULL, 10)`, `(NULL, 9)`, `(NULL, 11)`, - `(NULL, -9223372036854775808)`, `(NULL, 9223372036854775807)`}, - - {`(true, NULL, false)`, `(false, NULL, true)`, `(true, NULL, true)`, - `(false, NULL, false)`, `(true, NULL, true)`}, - {`(false, NULL, true)`, `(false, NULL, false)`, `(true, NULL, NULL)`, - `(false, NULL, false)`, `(true, NULL, true)`}, - - {`((true,), (false,))`, `((false,), (true,))`, `((true,), (true,))`, - `((false,), (false,))`, `((true,), (true,))`}, - {`((false,), (true,))`, `((false,), (false,))`, `((true,), NULL)`, - `((false,), (false,))`, `((true,), (true,))`}, - - // Arrays - - {`'{}'::INT[]`, valIsMin, `ARRAY[NULL]`, `ARRAY[]`, noMax}, - - {`array[NULL]`, noPrev, `ARRAY[NULL,NULL]`, `ARRAY[]`, noMax}, - {`array[true]`, noPrev, `ARRAY[true,NULL]`, `ARRAY[]`, noMax}, - - // Mixed tuple/array datums. - {`(ARRAY[true], (true,))`, `(ARRAY[true], (false,))`, `(ARRAY[true,NULL], NULL)`, - `(ARRAY[], (false,))`, noMax}, - {`((false,), ARRAY[true])`, noPrev, `((false,), ARRAY[true,NULL])`, - `((false,), ARRAY[])`, noMax}, - } - ctx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) - for _, td := range testData { - d := prepareExpr(t, td.datumExpr) - - prevVal, hasPrev := d.Prev(ctx) - nextVal, hasNext := d.Next(ctx) - if td.prev == noPrev { - if hasPrev { - if !d.IsMin(ctx) { - t.Errorf("%s: value should not have a prev, yet hasPrev true and IsMin() false (expected (!hasPrev || IsMin()))", td.datumExpr) - } - } - } else { - if !hasPrev && td.prev != valIsMin { - t.Errorf("%s: hasPrev: got false, expected true", td.datumExpr) - continue - } - isMin := d.IsMin(ctx) - if isMin != (td.prev == valIsMin) { - t.Errorf("%s: IsMin() %v, expected %v", td.datumExpr, isMin, (td.prev == valIsMin)) - continue - } - if !isMin { - dPrev := prevVal.String() - if dPrev != td.prev { - t.Errorf("%s: Prev(): got %s, expected %s", td.datumExpr, dPrev, td.prev) - } - } - } - if td.next == noNext { - if hasNext { - if !d.IsMax(ctx) { - t.Errorf("%s: value should not have a next, yet hasNext true and IsMax() false (expected (!hasNext || IsMax()))", td.datumExpr) - } - } - } else { - if !hasNext && td.next != valIsMax { - t.Errorf("%s: HasNext(): got false, expected true", td.datumExpr) - continue - } - isMax := d.IsMax(ctx) - if isMax != (td.next == valIsMax) { - t.Errorf("%s: IsMax() %v, expected %v", td.datumExpr, isMax, (td.next == valIsMax)) - continue - } - if !isMax { - dNext := nextVal.String() - if dNext != td.next { - t.Errorf("%s: Next(): got %s, expected %s", td.datumExpr, dNext, td.next) - } - } - } - - minVal, hasMin := d.Min(ctx) - maxVal, hasMax := d.Max(ctx) - - if td.min == noMin { - if hasMin { - t.Errorf("%s: hasMin true, expected false", td.datumExpr) - } - } else { - dMin := minVal.String() - if dMin != td.min { - t.Errorf("%s: min(): got %s, expected %s", td.datumExpr, dMin, td.min) - } - } - if td.max == noMax { - if hasMax { - t.Errorf("%s: hasMax true, expected false", td.datumExpr) - } - } else { - dMax := maxVal.String() - if dMax != td.max { - t.Errorf("%s: max(): got %s, expected %s", td.datumExpr, dMax, td.max) - } - } - } -} - -func TestDFloatCompare(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - values := []tree.Datum{tree.DNull} - for _, x := range []float64{math.NaN(), math.Inf(-1), -1, 0, 1, math.Inf(1)} { - values = append(values, tree.NewDFloat(tree.DFloat(x))) - } - for i, x := range values { - for j, y := range values { - expected := 0 - if i < j { - expected = -1 - } else if i > j { - expected = 1 - } - evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) - defer evalCtx.Stop(context.Background()) - got := x.Compare(evalCtx, y) - if got != expected { - t.Errorf("comparing DFloats %s and %s: expected %d, got %d", x, y, expected, got) - } - } - } -} - -// TestParseDIntervalWithTypeMetadata tests that the additional features available -// to tree.ParseDIntervalWithTypeMetadata beyond those in tree.ParseDInterval behave as expected. -func TestParseDIntervalWithTypeMetadata(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - var ( - second = types.IntervalTypeMetadata{ - DurationField: types.IntervalDurationField{ - DurationType: types.IntervalDurationType_SECOND, - }, - } - minute = types.IntervalTypeMetadata{ - DurationField: types.IntervalDurationField{ - DurationType: types.IntervalDurationType_MINUTE, - }, - } - hour = types.IntervalTypeMetadata{ - DurationField: types.IntervalDurationField{ - DurationType: types.IntervalDurationType_HOUR, - }, - } - day = types.IntervalTypeMetadata{ - DurationField: types.IntervalDurationField{ - DurationType: types.IntervalDurationType_DAY, - }, - } - month = types.IntervalTypeMetadata{ - DurationField: types.IntervalDurationField{ - DurationType: types.IntervalDurationType_MONTH, - }, - } - year = types.IntervalTypeMetadata{ - DurationField: types.IntervalDurationField{ - DurationType: types.IntervalDurationType_YEAR, - }, - } - ) - - testData := []struct { - str string - dtype types.IntervalTypeMetadata - expected string - }{ - // Test cases for raw numbers with fields - {"5", second, "5s"}, - {"5.8", second, "5.8s"}, - {"5", minute, "5m"}, - {"5.8", minute, "5m"}, - {"5", hour, "5h"}, - {"5.8", hour, "5h"}, - {"5", day, "5 day"}, - {"5.8", day, "5 day"}, - {"5", month, "5 month"}, - {"5.8", month, "5 month"}, - {"5", year, "5 year"}, - {"5.8", year, "5 year"}, - // Test cases for truncation based on fields - {"1-2 3 4:56:07", second, "1-2 3 4:56:07"}, - {"1-2 3 4:56:07", minute, "1-2 3 4:56:00"}, - {"1-2 3 4:56:07", hour, "1-2 3 4:00:00"}, - {"1-2 3 4:56:07", day, "1-2 3 0:"}, - {"1-2 3 4:56:07", month, "1-2 0 0:"}, - {"1-2 3 4:56:07", year, "1 year"}, - } - for _, td := range testData { - actual, err := tree.ParseDIntervalWithTypeMetadata(td.str, td.dtype) - if err != nil { - t.Errorf("unexpected error while parsing INTERVAL %s %#v: %s", td.str, td.dtype, err) - continue - } - expected, err := tree.ParseDInterval(td.expected) - if err != nil { - t.Errorf("unexpected error while parsing expected value INTERVAL %s: %s", td.expected, err) - continue - } - evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) - defer evalCtx.Stop(context.Background()) - if expected.Compare(evalCtx, actual) != 0 { - t.Errorf("INTERVAL %s %#v: got %s, expected %s", td.str, td.dtype, actual, expected) - } - } -} - -func TestParseDDate(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := testParseTimeContext( - time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.FixedZone("foo", -18000)), - ) - - testData := []struct { - str string - expected string - expectedDepOnCtx bool - }{ - {"now", "2001-02-03", true}, - {"today", "2001-02-03", true}, - {"tomorrow", "2001-02-04", true}, - {"yesterday", "2001-02-02", true}, - {"2017-03-03 01:00:00.00000", "2017-03-03", false}, - {"2017-03-03 01:00:00.00000-05", "2017-03-03", false}, - {"2017-03-03 01:00:00.00000+05", "2017-03-03", false}, - {"2017-03-03 -01:00:00", "2017-03-03", false}, - {"2017-03-03 -01:00:00 America/New_York", "2017-03-03", false}, - {"2017-03-03 -1:0:0", "2017-03-03", false}, - {"2017-03-03 -01:00", "2017-03-03", false}, - {"2017-03-03 -01", "2017-03-03", false}, - {"2017-03-03 -010000", "2017-03-03", false}, - {"2017-03-03 -0100", "2017-03-03", false}, - {"2017-03-03 -1", "2017-03-03", false}, - {"2017-03-03", "2017-03-03", false}, - {"2017-3-3 -01:00:00", "2017-03-03", false}, - {"2017-3-3 -1:0:0", "2017-03-03", false}, - {"2017-3-3 -01:00", "2017-03-03", false}, - {"2017-3-3 -01", "2017-03-03", false}, - {"2017-3-3 -010000", "2017-03-03", false}, - {"2017-3-3 -0100", "2017-03-03", false}, - {"2017-3-3 -1", "2017-03-03", false}, - {"2017-3-3", "2017-03-03", false}, - } - for _, td := range testData { - actual, depOnCtx, err := tree.ParseDDate(ctx, td.str) - if err != nil { - t.Errorf("unexpected error while parsing DATE %s: %s", td.str, err) - continue - } - expected, _, err := tree.ParseDDate(nil, td.expected) - if err != nil { - t.Errorf("unexpected error while parsing expected value DATE %s: %s", td.expected, err) - continue - } - evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) - defer evalCtx.Stop(context.Background()) - if expected.Compare(evalCtx, actual) != 0 { - t.Errorf("DATE %s: got %s, expected %s", td.str, actual, expected) - } - if td.expectedDepOnCtx != depOnCtx { - t.Errorf("DATE %s: expected depOnCtx=%v", td.str, td.expectedDepOnCtx) - } - } -} - -func TestParseDBool(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - testData := []struct { - str string - expected *tree.DBool - err bool - }{ - {str: "t", expected: tree.DBoolTrue}, - {str: "tr", expected: tree.DBoolTrue}, - {str: "tru", expected: tree.DBoolTrue}, - {str: "true", expected: tree.DBoolTrue}, - {str: "tr", expected: tree.DBoolTrue}, - {str: "TRUE", expected: tree.DBoolTrue}, - {str: "tRUe", expected: tree.DBoolTrue}, - {str: " tRUe ", expected: tree.DBoolTrue}, - {str: " tR ", expected: tree.DBoolTrue}, - {str: "on", expected: tree.DBoolTrue}, - {str: "On", expected: tree.DBoolTrue}, - {str: "oN", expected: tree.DBoolTrue}, - {str: "ON", expected: tree.DBoolTrue}, - {str: "1", expected: tree.DBoolTrue}, - {str: "yes", expected: tree.DBoolTrue}, - {str: "ye", expected: tree.DBoolTrue}, - {str: "y", expected: tree.DBoolTrue}, - - {str: "false", expected: tree.DBoolFalse}, - {str: "FALSE", expected: tree.DBoolFalse}, - {str: "fALse", expected: tree.DBoolFalse}, - {str: "f", expected: tree.DBoolFalse}, - {str: "off", expected: tree.DBoolFalse}, - {str: "Off", expected: tree.DBoolFalse}, - {str: "oFF", expected: tree.DBoolFalse}, - {str: "OFF", expected: tree.DBoolFalse}, - {str: "0", expected: tree.DBoolFalse}, - - {str: "foo", err: true}, - {str: "tr ue", err: true}, - {str: "o", err: true}, - {str: "", err: true}, - {str: " ", err: true}, - {str: " ", err: true}, - } - - for _, td := range testData { - t.Run(td.str, func(t *testing.T) { - result, err := tree.ParseDBool(td.str) - if td.err { - if err == nil { - t.Fatalf("expected parsing %v to error, got %v", td.str, result) - } - return - } - if err != nil { - t.Fatalf("expected parsing %v to be %s, got error: %s", td.str, td.expected, err) - } - if *td.expected != *result { - t.Fatalf("expected parsing %v to be %s, got %s", td.str, td.expected, result) - } - }) - } -} - -func TestParseDTime(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := testParseTimeContext( - time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.FixedZone("foo", -18000)), - ) - // Since ParseDTime shares most of the underlying parsing logic to - // ParseDTimestamp, we only test a subset of the timestamp test cases. - testData := []struct { - str string - precision time.Duration - expected timeofday.TimeOfDay - expectedDepOnCtx bool - }{ - {"now", time.Microsecond, timeofday.New(4, 5, 6, 1), true}, - {" 04:05:06 ", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, - {"04:05:06", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, - {"04:05:06.000001", time.Microsecond, timeofday.New(4, 5, 6, 1), false}, - {"04:05:06.000001+00", time.Microsecond, timeofday.New(4, 5, 6, 1), false}, - {"04:05:06.000001-05", time.Microsecond, timeofday.New(4, 5, 6, 1), false}, - {"04:05:06.000001+05", time.Microsecond, timeofday.New(4, 5, 6, 1), false}, - {"04:05:06.000001", time.Second, timeofday.New(4, 5, 6, 0), false}, - {"04:05:06-07", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, - {"0000-01-01 04:05:06", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, - {"2001-01-01 04:05:06", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, - {"4:5:6", time.Microsecond, timeofday.New(4, 5, 6, 0), false}, - {"24:00:00", time.Microsecond, timeofday.Time2400, false}, - {"24:00:00.000", time.Microsecond, timeofday.Time2400, false}, - {"24:00:00.000000", time.Microsecond, timeofday.Time2400, false}, - {"0000-01-01T24:00:00", time.Microsecond, timeofday.Time2400, false}, - {"0000-01-01T24:00:00.0", time.Microsecond, timeofday.Time2400, false}, - {"0000-01-01 24:00:00", time.Microsecond, timeofday.Time2400, false}, - {"0000-01-01 24:00:00.0", time.Microsecond, timeofday.Time2400, false}, - {" 24:00:00.0", time.Microsecond, timeofday.Time2400, false}, - {" 24:00:00.0 ", time.Microsecond, timeofday.Time2400, false}, - } - for _, td := range testData { - actual, depOnCtx, err := tree.ParseDTime(ctx, td.str, td.precision) - if err != nil { - t.Errorf("unexpected error while parsing TIME %s: %s", td.str, err) - continue - } - if *actual != tree.DTime(td.expected) { - t.Errorf("TIME %s: got %s, expected %s", td.str, actual, td.expected) - } - if td.expectedDepOnCtx != depOnCtx { - t.Errorf("TIME %s: expected depOnCtx=%v", td.str, td.expectedDepOnCtx) - } - } -} - -func TestParseDTimeError(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - testData := []string{ - "", - "foo", - "01", - "today", - "yesterday", - - // TODO(radu): these exceptions seem dubious. They work in postgres. - "24:00:00.000000+00", - "24:00:00.000000-05", - "24:00:00.000000+05", - } - for _, s := range testData { - actual, _, _ := tree.ParseDTime(nil, s, time.Microsecond) - if actual != nil { - t.Errorf("TIME %s: got %s, expected error", s, actual) - } - } -} - -func TestParseDTimeTZ(t *testing.T) { - defer leaktest.AfterTest(t)() - ctx := testParseTimeContext( - time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.FixedZone("foo", 18000)), - ) - - mk := func(hour, min, sec, micro int, offset int32) timetz.TimeTZ { - return timetz.MakeTimeTZ(timeofday.New(hour, min, sec, micro), offset) - } - - testData := []struct { - str string - precision time.Duration - expected timetz.TimeTZ - expectedDepOnCtx bool - }{ - {" 04:05:06 ", time.Microsecond, mk(4, 5, 6, 0, -18000), true}, - {"04:05:06", time.Microsecond, mk(4, 5, 6, 0, -18000), true}, - {"04:05:06.000001", time.Microsecond, mk(4, 5, 6, 1, -18000), true}, - {"04:05:06.000001", time.Second, mk(4, 5, 6, 0, -18000), true}, - {"04:05:06.000001+00", time.Microsecond, mk(4, 5, 6, 1, 0), false}, - {"04:05:06.000001-04", time.Microsecond, mk(4, 5, 6, 1, 4*3600), false}, - {"04:05:06.000001+04", time.Microsecond, mk(4, 5, 6, 1, -4*3600), false}, - {"04:05:06-07", time.Microsecond, mk(4, 5, 6, 0, 7*3600), false}, - {"0000-01-01 04:05:06", time.Microsecond, mk(4, 5, 6, 0, -18000), true}, - {"2001-01-01 04:05:06", time.Microsecond, mk(4, 5, 6, 0, -18000), true}, - {"4:5:6", time.Microsecond, mk(4, 5, 6, 0, -18000), true}, - {"24:00:00", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, - {"24:00:00.000", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, - {"24:00:00.000000", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, - {"24:00:00.000000+00", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, 0), false}, - {"24:00:00.000000-04", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, 4*3600), false}, - {"24:00:00.000000+04", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -4*3600), false}, - {"0000-01-01T24:00:00", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, - {"0000-01-01T24:00:00.0", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, - {"0000-01-01 24:00:00", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, - {"0000-01-01 24:00:00.0", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, - {" 24:00:00.0", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, - {" 24:00:00.0 ", time.Microsecond, timetz.MakeTimeTZ(timeofday.Time2400, -18000), true}, - } - for _, td := range testData { - actual, depOnCtx, err := tree.ParseDTimeTZ(ctx, td.str, td.precision) - if err != nil { - t.Errorf("unexpected error while parsing TIME %s: %s", td.str, err) - continue - } - exp := tree.DTimeTZ{TimeTZ: td.expected} - if *actual != exp { - t.Errorf("TIMETZ %s: got %s, expected %s", td.str, actual, &exp) - } - if td.expectedDepOnCtx != depOnCtx { - t.Errorf("TIME %s: expected depOnCtx=%v", td.str, td.expectedDepOnCtx) - } - } -} - -func TestParseDTimeTZError(t *testing.T) { - defer leaktest.AfterTest(t)() - testData := []string{ - "", - "foo", - "01", - "today", - "yesterday", - - // TODO(radu): this should work. - "now", - } - for _, s := range testData { - actual, _, _ := tree.ParseDTimeTZ(nil, s, time.Microsecond) - if actual != nil { - t.Errorf("TIMETZ %s: got %s, expected error", s, actual) - } - } -} - -func TestParseDTimestamp(t *testing.T) { +func TestCompareTimestamps(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - ctx := testParseTimeContext( - time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.FixedZone("foo", -18000)), - ) - - testData := []struct { - str string - expected time.Time - expectedDepOnCtx bool - }{ - {"now", time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.UTC), true}, - {"today", time.Date(2001, time.February, 3, 0, 0, 0, 0, time.UTC), true}, - {"tomorrow", time.Date(2001, time.February, 4, 0, 0, 0, 0, time.UTC), true}, - {"yesterday", time.Date(2001, time.February, 2, 0, 0, 0, 0, time.UTC), true}, - {"2001-02-03", time.Date(2001, time.February, 3, 0, 0, 0, 0, time.UTC), false}, - {"2001-02-03 04:05:06", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.UTC), false}, - {"2001-02-03 04:05:06.000001", time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.UTC), false}, - {"2001-02-03 04:05:06.00001", time.Date(2001, time.February, 3, 4, 5, 6, 10000, time.UTC), false}, - {"2001-02-03 04:05:06.0001", time.Date(2001, time.February, 3, 4, 5, 6, 100000, time.UTC), false}, - {"2001-02-03 04:05:06.001", time.Date(2001, time.February, 3, 4, 5, 6, 1000000, time.UTC), false}, - {"2001-02-03 04:05:06.01", time.Date(2001, time.February, 3, 4, 5, 6, 10000000, time.UTC), false}, - {"2001-02-03 04:05:06.1", time.Date(2001, time.February, 3, 4, 5, 6, 100000000, time.UTC), false}, - {"2001-02-03 04:05:06.12", time.Date(2001, time.February, 3, 4, 5, 6, 120000000, time.UTC), false}, - {"2001-02-03 04:05:06.123", time.Date(2001, time.February, 3, 4, 5, 6, 123000000, time.UTC), false}, - {"2001-02-03 04:05:06.1234", time.Date(2001, time.February, 3, 4, 5, 6, 123400000, time.UTC), false}, - {"2001-02-03 04:05:06.12345", time.Date(2001, time.February, 3, 4, 5, 6, 123450000, time.UTC), false}, - {"2001-02-03 04:05:06.123456", time.Date(2001, time.February, 3, 4, 5, 6, 123456000, time.UTC), false}, - {"2001-02-03 04:05:06.123-07", time.Date(2001, time.February, 3, 4, 5, 6, 123000000, time.UTC), false}, - {"2001-02-03 04:05:06-07", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.UTC), false}, - {"2001-02-03 04:05:06-07:42", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.UTC), false}, - {"2001-02-03 04:05:06-07:30:09", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.UTC), false}, - {"2001-02-03 04:05:06+07", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.UTC), false}, - {"2001-02-03 04:0:06", time.Date(2001, time.February, 3, 4, 0, 6, 0, time.UTC), false}, - {"2001-02-03 0:0:06", time.Date(2001, time.February, 3, 0, 0, 6, 0, time.UTC), false}, - {"2001-02-03 4:05:0", time.Date(2001, time.February, 3, 4, 5, 0, 0, time.UTC), false}, - {"2001-02-03 4:05:0-07:0:00", time.Date(2001, time.February, 3, 4, 5, 0, 0, time.UTC), false}, - {"2001-02-03 4:0:6 +3:0:0", time.Date(2001, time.February, 3, 4, 0, 6, 0, time.UTC), false}, - } - for _, td := range testData { - actual, depOnCtx, err := tree.ParseDTimestamp(ctx, td.str, time.Nanosecond) - if err != nil { - t.Errorf("unexpected error while parsing TIMESTAMP %s: %s", td.str, err) - continue - } - if !actual.Time.Equal(td.expected) { - t.Errorf("TIMESTAMP %s: got %s, expected %s", td.str, actual, td.expected) - } - if td.expectedDepOnCtx != depOnCtx { - t.Errorf("TIMESTAMP %s: expected depOnCtx=%v", td.str, td.expectedDepOnCtx) - } - } -} - -func TestParseDTimestampTZ(t *testing.T) { - defer leaktest.AfterTest(t)() - - local := time.FixedZone("foo", -18000) - ctx := testParseTimeContext(time.Date(2001, time.February, 3, 4, 5, 6, 1000, local)) - - testData := []struct { - str string - expected time.Time - expectedDepOnCtx bool - }{ - {"now", time.Date(2001, time.February, 3, 4, 5, 6, 1000, local), true}, - {"today", time.Date(2001, time.February, 3, 0, 0, 0, 0, local), true}, - {"tomorrow", time.Date(2001, time.February, 4, 0, 0, 0, 0, local), true}, - {"yesterday", time.Date(2001, time.February, 2, 0, 0, 0, 0, local), true}, - {"2001-02-03", time.Date(2001, time.February, 3, 0, 0, 0, 0, local), true}, - {"2001-02-03 04:05:06", time.Date(2001, time.February, 3, 4, 5, 6, 0, local), true}, - {"2001-02-03 04:05:06.000001", time.Date(2001, time.February, 3, 4, 5, 6, 1000, local), true}, - {"2001-02-03 04:05:06.00001", time.Date(2001, time.February, 3, 4, 5, 6, 10000, local), true}, - {"2001-02-03 04:05:06.0001", time.Date(2001, time.February, 3, 4, 5, 6, 100000, local), true}, - {"2001-02-03 04:05:06.001", time.Date(2001, time.February, 3, 4, 5, 6, 1000000, local), true}, - {"2001-02-03 04:05:06.01", time.Date(2001, time.February, 3, 4, 5, 6, 10000000, local), true}, - {"2001-02-03 04:05:06.1", time.Date(2001, time.February, 3, 4, 5, 6, 100000000, local), true}, - {"2001-02-03 04:05:06.12", time.Date(2001, time.February, 3, 4, 5, 6, 120000000, local), true}, - {"2001-02-03 04:05:06.123", time.Date(2001, time.February, 3, 4, 5, 6, 123000000, local), true}, - {"2001-02-03 04:05:06.1234", time.Date(2001, time.February, 3, 4, 5, 6, 123400000, local), true}, - {"2001-02-03 04:05:06.12345", time.Date(2001, time.February, 3, 4, 5, 6, 123450000, local), true}, - {"2001-02-03 04:05:06.123456", time.Date(2001, time.February, 3, 4, 5, 6, 123456000, local), true}, - {"2001-02-03 04:05:06.123-07", time.Date(2001, time.February, 3, 4, 5, 6, 123000000, time.FixedZone("", -7*3600)), false}, - {"2001-02-03 04:05:06-07", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.FixedZone("", -7*3600)), false}, - {"2001-02-03 04:05:06-07:42", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.FixedZone("", -7*3600-42*60)), false}, - {"2001-02-03 04:05:06-07:30:09", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.FixedZone("", -7*3600-30*60-9)), false}, - {"2001-02-03 04:05:06+07", time.Date(2001, time.February, 3, 4, 5, 6, 0, time.FixedZone("", 7*3600)), false}, - {"2001-02-03 04:0:06", time.Date(2001, time.February, 3, 4, 0, 6, 0, local), true}, - {"2001-02-03 0:0:06", time.Date(2001, time.February, 3, 0, 0, 6, 0, local), true}, - {"2001-02-03 4:05:0", time.Date(2001, time.February, 3, 4, 5, 0, 0, local), true}, - {"2001-02-03 4:05:0-07:0:00", time.Date(2001, time.February, 3, 4, 5, 0, 0, time.FixedZone("", -7*3600)), false}, - {"2001-02-03 4:0:6 +3:0:0", time.Date(2001, time.February, 3, 4, 0, 6, 0, time.FixedZone("", 3*3600)), false}, - } - for _, td := range testData { - actual, depOnCtx, err := tree.ParseDTimestampTZ(ctx, td.str, time.Nanosecond) - if err != nil { - t.Errorf("unexpected error while parsing TIMESTAMP %s: %s", td.str, err) - continue - } - if !actual.Time.Equal(td.expected) { - t.Errorf("TIMESTAMPTZ %s: got %s, expected %s", td.str, actual, td.expected) - } - if td.expectedDepOnCtx != depOnCtx { - t.Errorf("TIMESTAMPTZ %s: expected depOnCtx=%v", td.str, td.expectedDepOnCtx) - } - } -} - -func TestMakeDJSON(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - j1, err := tree.MakeDJSON(1) - if err != nil { - t.Fatal(err) - } - j2, err := tree.MakeDJSON(2) - if err != nil { - t.Fatal(err) - } - if j1.Compare(tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()), j2) != -1 { - t.Fatal("expected JSON 1 < 2") - } -} - -func TestDTimeTZ(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := &tree.EvalContext{ - SessionData: &sessiondata.SessionData{ - Location: time.UTC, - }, - } - - maxTime, depOnCtx, err := tree.ParseDTimeTZ(ctx, "24:00:00-1559", time.Microsecond) - require.NoError(t, err) - require.False(t, depOnCtx) - minTime, depOnCtx, err := tree.ParseDTimeTZ(ctx, "00:00:00+1559", time.Microsecond) - require.NoError(t, err) - require.False(t, depOnCtx) + pacificTimeZone := int32(7 * 60 * 60) + sydneyTimeZone := int32(-10 * 60 * 60) - // These are all the same UTC time equivalents. - utcTime, depOnCtx, err := tree.ParseDTimeTZ(ctx, "11:14:15+0", time.Microsecond) - require.NoError(t, err) - require.False(t, depOnCtx) - sydneyTime, depOnCtx, err := tree.ParseDTimeTZ(ctx, "21:14:15+10", time.Microsecond) - require.NoError(t, err) - require.False(t, depOnCtx) + sydneyFixedZone := time.FixedZone("otan@sydney", -int(sydneyTimeZone)) + // kiwiFixedZone is 2 hours ahead of Sydney. + kiwiFixedZone := time.FixedZone("otan@auckland", -int(sydneyTimeZone)+2*60*60) // No daylight savings in Hawaii! - hawaiiZone, err := timeutil.LoadLocation("Pacific/Honolulu") + ddate, err := NewDDateFromTime(time.Date(2019, time.November, 22, 0, 0, 0, 0, time.UTC)) 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 + desc string + left Datum + right Datum + location *time.Location + expected int }{ { - t: weirdTimeZone, - largerThan: []tree.Datum{minTime, tree.DNull}, - smallerThan: []tree.Datum{maxTime}, - equalTo: []tree.Datum{weirdTimeZone}, - isMax: false, - isMin: false, + desc: "same DTime are equal", + left: MakeDTime(timeofday.New(12, 0, 0, 0)), + right: MakeDTime(timeofday.New(12, 0, 0, 0)), + expected: 0, }, { - t: utcTime, - largerThan: []tree.Datum{minTime, sydneyTime, tree.DNull}, - smallerThan: []tree.Datum{maxTime, hawaiiTime}, - equalTo: []tree.Datum{utcTime}, - isMax: false, - isMin: false, + 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, }, { - t: sydneyTime, - largerThan: []tree.Datum{minTime, tree.DNull}, - smallerThan: []tree.Datum{maxTime, utcTime, hawaiiTime}, - equalTo: []tree.Datum{sydneyTime}, - isMax: false, - isMin: false, + desc: "DTime and DTimeTZ both UTC, and so are equal", + left: MakeDTime(timeofday.New(12, 0, 0, 0)), + right: NewDTimeTZFromOffset(timeofday.New(12, 0, 0, 0), 0), + expected: 0, }, { - t: hawaiiTime, - largerThan: []tree.Datum{minTime, utcTime, sydneyTime, tree.DNull}, - smallerThan: []tree.Datum{maxTime}, - equalTo: []tree.Datum{hawaiiTime}, - isMax: false, - isMin: false, + desc: "DTime and DTimeTZ both Sydney time, and so are equal", + left: MakeDTime(timeofday.New(12, 0, 0, 0)), + right: NewDTimeTZFromOffset(timeofday.New(12, 0, 0, 0), sydneyTimeZone), + location: sydneyFixedZone, + expected: 0, }, { - t: minTime, - largerThan: []tree.Datum{tree.DNull}, - smallerThan: []tree.Datum{maxTime, utcTime, sydneyTime, hawaiiTime}, - equalTo: []tree.Datum{minTime}, - isMax: false, - isMin: true, + desc: "DTimestamp and DTimestampTZ (Sydney) equal in Sydney zone", + left: MustMakeDTimestamp(time.Date(2019, time.November, 22, 10, 0, 0, 0, time.UTC), time.Microsecond), + right: MustMakeDTimestampTZ(time.Date(2019, time.November, 22, 10, 0, 0, 0, sydneyFixedZone), time.Microsecond), + location: sydneyFixedZone, + expected: 0, }, { - t: maxTime, - largerThan: []tree.Datum{minTime, utcTime, sydneyTime, hawaiiTime, tree.DNull}, - smallerThan: []tree.Datum{}, - equalTo: []tree.Datum{maxTime}, - isMax: true, - isMin: false, + desc: "DTimestamp and DTimestampTZ (Sydney) equal in Sydney+2 zone", + left: MustMakeDTimestamp(time.Date(2019, time.November, 22, 12, 0, 0, 0, time.UTC), time.Microsecond), + right: MustMakeDTimestampTZ(time.Date(2019, time.November, 22, 10, 0, 0, 0, sydneyFixedZone), time.Microsecond), + location: kiwiFixedZone, + expected: 0, + }, + { + desc: "Date and DTimestampTZ (Sydney) equal in Sydney zone", + left: ddate, + right: MustMakeDTimestampTZ(time.Date(2019, time.November, 22, 0, 0, 0, 0, sydneyFixedZone), time.Microsecond), + location: sydneyFixedZone, + expected: 0, + }, + { + desc: "Date and DTimestampTZ (Sydney) equal in Sydney+2 zone", + left: ddate, + right: MustMakeDTimestampTZ(time.Date(2019, time.November, 21, 22, 0, 0, 0, sydneyFixedZone), time.Microsecond), + location: kiwiFixedZone, + 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 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(ctx) - 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(ctx, largerThan), "%s > %s", tc.t.String(), largerThan.String()) - } - var smallerThan []tree.Datum - next, ok := tc.t.Next(ctx) - 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(ctx, smallerThan), "%s < %s", tc.t.String(), smallerThan.String()) - } - - for _, equalTo := range tc.equalTo { - assert.Equal(t, 0, tc.t.Compare(ctx, equalTo), "%s = %s", tc.t.String(), equalTo.String()) - } - - assert.Equal(t, tc.isMax, tc.t.IsMax(ctx)) - assert.Equal(t, tc.isMin, tc.t.IsMin(ctx)) - }) + for _, tc := range testCases { + t.Run( + tc.desc, + func(t *testing.T) { + ctx := &EvalContext{ + SessionData: &sessiondata.SessionData{ + Location: tc.location, + }, + } + assert.Equal(t, tc.expected, compareTimestamps(ctx, tc.left, tc.right)) + assert.Equal(t, -tc.expected, compareTimestamps(ctx, tc.right, tc.left)) + }, + ) } } -func TestIsDistinctFrom(t *testing.T) { +func TestCastStringToRegClassTableName(t *testing.T) { defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - testData := []struct { - a string // comma separated list of strings, `NULL` is converted to a NULL - b string // same as a - expected bool + + testCases := []struct { + in string + expected TableName }{ - {"a", "a", false}, - {"a", "b", true}, - {"b", "b", false}, - {"a,a", "a,a", false}, - {"a,a", "a,b", true}, - {"a,a", "b,a", true}, - {"a,a,a", "a,a,a", false}, - {"a,a,a", "a,a,b", true}, - {"a,a,a", "a,b,a", true}, - {"a,a,a", "a,b,b", true}, - {"a,a,a", "b,a,a", true}, - {"a,a,a", "b,a,b", true}, - {"a,a,a", "b,b,a", true}, - {"a,a,a", "b,b,b", true}, - {"NULL", "NULL", false}, - {"a", "NULL", true}, - {"a,a", "a,NULL", true}, - {"a,a", "NULL,a", true}, - {"a,a", "NULL,NULL", true}, - {"a,NULL", "a,a", true}, - {"a,NULL", "a,NULL", false}, - {"a,NULL", "NULL,a", true}, - {"a,NULL", "NULL,NULL", true}, - {"NULL,a", "a,a", true}, - {"NULL,a", "a,NULL", true}, - {"NULL,a", "NULL,a", false}, - {"NULL,a", "NULL,NULL", true}, - {"NULL,NULL", "a,a", true}, - {"NULL,NULL", "a,NULL", true}, - {"NULL,NULL", "NULL,a", true}, - {"NULL,NULL", "NULL,NULL", false}, - {"a,a,a", "a,a,NULL", true}, - {"a,a,a", "a,NULL,a", true}, - {"a,a,a", "a,NULL,NULL", true}, - {"a,a,a", "NULL,a,a", true}, - {"a,a,a", "NULL,a,NULL", true}, - {"a,a,a", "NULL,NULL,a", true}, - {"a,a,a", "NULL,NULL,NULL", true}, - {"a,NULL,a", "a,a,a", true}, - {"a,NULL,a", "a,a,NULL", true}, - {"a,NULL,a", "a,NULL,a", false}, - {"a,NULL,a", "a,NULL,NULL", true}, - {"a,NULL,a", "NULL,a,a", true}, - {"a,NULL,a", "NULL,a,NULL", true}, - {"a,NULL,a", "NULL,NULL,a", true}, - {"a,NULL,a", "NULL,NULL,NULL", true}, - {"NULL,a,NULL", "a,a,a", true}, - {"NULL,a,NULL", "a,a,NULL", true}, - {"NULL,a,NULL", "a,NULL,a", true}, - {"NULL,a,NULL", "a,NULL,NULL", true}, - {"NULL,a,NULL", "NULL,a,a", true}, - {"NULL,a,NULL", "NULL,a,NULL", false}, - {"NULL,a,NULL", "NULL,NULL,a", true}, - {"NULL,a,NULL", "NULL,NULL,NULL", true}, - {"NULL,NULL,NULL", "a,a,a", true}, - {"NULL,NULL,NULL", "a,a,NULL", true}, - {"NULL,NULL,NULL", "a,NULL,a", true}, - {"NULL,NULL,NULL", "a,NULL,NULL", true}, - {"NULL,NULL,NULL", "NULL,a,a", true}, - {"NULL,NULL,NULL", "NULL,a,NULL", true}, - {"NULL,NULL,NULL", "NULL,NULL,a", true}, - {"NULL,NULL,NULL", "NULL,NULL,NULL", false}, + {"a", MakeUnqualifiedTableName("a")}, + {`a"`, MakeUnqualifiedTableName(`a"`)}, + {`"a""".bB."cD" `, MakeTableNameWithSchema(`a"`, "bb", "cD")}, } - convert := func(s string) tree.Datums { - splits := strings.Split(s, ",") - result := make(tree.Datums, len(splits)) - for i, value := range splits { - if value == "NULL" { - result[i] = tree.DNull - continue - } - result[i] = tree.NewDString(value) - } - return result - } - for _, td := range testData { - t.Run(fmt.Sprintf("%s to %s", td.a, td.b), func(t *testing.T) { - datumsA := convert(td.a) - datumsB := convert(td.b) - if e, a := td.expected, datumsA.IsDistinctFrom(&tree.EvalContext{}, datumsB); e != a { - if e { - t.Errorf("expected %s to be distinct from %s, but got %t", datumsA, datumsB, e) - } else { - t.Errorf("expected %s to not be distinct from %s, but got %t", datumsA, datumsB, e) - } - } + + for _, tc := range testCases { + t.Run(tc.in, func(t *testing.T) { + out, err := castStringToRegClassTableName(tc.in) + require.NoError(t, err) + require.Equal(t, tc.expected, out) }) } -} -func TestAllTypesAsJSON(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - for _, typ := range types.Scalar { - d := tree.SampleDatum(typ) - _, err := tree.AsJSON(d, time.UTC) - if err != nil { - t.Errorf("couldn't convert %s to JSON: %s", d, err) - } + errorTestCases := []struct { + in string + expectedError string + }{ + {"a.b.c.d", "too many components: a.b.c.d"}, + {"", `invalid table name: `}, + } + + for _, tc := range errorTestCases { + t.Run(tc.in, func(t *testing.T) { + _, err := castStringToRegClassTableName(tc.in) + require.EqualError(t, err, tc.expectedError) + }) } + } -// Test default values of many different datum types. -func TestNewDefaultDatum(t *testing.T) { +func TestSplitIdentifierList(t *testing.T) { defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) - defer evalCtx.Stop(context.Background()) testCases := []struct { - t *types.T - expected string + in string + expected []string }{ - {t: types.Bool, expected: "false"}, - {t: types.Int, expected: "0:::INT8"}, - {t: types.Int2, expected: "0:::INT8"}, - {t: types.Int4, expected: "0:::INT8"}, - {t: types.Float, expected: "0.0:::FLOAT8"}, - {t: types.Float4, expected: "0.0:::FLOAT8"}, - {t: types.Decimal, expected: "0:::DECIMAL"}, - {t: types.MakeDecimal(10, 5), expected: "0:::DECIMAL"}, - {t: types.Date, expected: "'2000-01-01':::DATE"}, - {t: types.Timestamp, expected: "'0001-01-01 00:00:00':::TIMESTAMP"}, - {t: types.Interval, expected: "'00:00:00':::INTERVAL"}, - {t: types.String, expected: "'':::STRING"}, - {t: types.MakeChar(3), expected: "'':::STRING"}, - {t: types.Bytes, expected: "'\\x':::BYTES"}, - {t: types.TimestampTZ, expected: "'0001-01-01 00:00:00+00:00':::TIMESTAMPTZ"}, - {t: types.MakeCollatedString(types.MakeVarChar(10), "de"), expected: "'' COLLATE de"}, - {t: types.MakeCollatedString(types.VarChar, "en_US"), expected: "'' COLLATE en_US"}, - {t: types.Oid, expected: "26:::OID"}, - {t: types.RegClass, expected: "crdb_internal.create_regclass(2205,'regclass'):::REGCLASS"}, - {t: types.Unknown, expected: "NULL"}, - {t: types.Uuid, expected: "'00000000-0000-0000-0000-000000000000':::UUID"}, - {t: types.MakeArray(types.Int), expected: "ARRAY[]:::INT8[]"}, - {t: types.MakeArray(types.MakeArray(types.String)), expected: "ARRAY[]:::STRING[][]"}, - {t: types.OidVector, expected: "ARRAY[]:::OID[]"}, - {t: types.INet, expected: "'0.0.0.0/0':::INET"}, - {t: types.Time, expected: "'00:00:00':::TIME"}, - {t: types.Jsonb, expected: "'null':::JSONB"}, - {t: types.TimeTZ, expected: "'00:00:00+00:00:00':::TIMETZ"}, - {t: types.MakeTuple([]*types.T{}), expected: "()"}, - {t: types.MakeTuple([]*types.T{types.Int, types.MakeChar(1)}), expected: "(0:::INT8, '':::STRING)"}, - {t: types.MakeTuple([]*types.T{types.OidVector, types.MakeTuple([]*types.T{types.Float})}), expected: "(ARRAY[]:::OID[], (0.0:::FLOAT8,))"}, - {t: types.VarBit, expected: "B''"}, - {t: types.MakeBit(5), expected: "B''"}, + {`abc`, []string{"abc"}}, + {`abc.dEf `, []string{"abc", "def"}}, + {` "aBc" . d ."HeLLo"""`, []string{"aBc", "d", `HeLLo"`}}, } - for i, tc := range testCases { - t.Run(fmt.Sprintf("#%d %s", i, tc.t.SQLString()), func(t *testing.T) { - datum, err := tree.NewDefaultDatum(evalCtx, tc.t) - if err != nil { - t.Errorf("unexpected error: %s", err) - } - - actual := tree.AsStringWithFlags(datum, tree.FmtCheckEquivalence) - if actual != tc.expected { - t.Errorf("expected %s, got %s", tc.expected, actual) - } + for _, tc := range testCases { + t.Run(tc.in, func(t *testing.T) { + out, err := splitIdentifierList(tc.in) + require.NoError(t, err) + require.Equal(t, tc.expected, out) }) } -} - -type testParseTimeContext time.Time -var _ tree.ParseTimeContext = testParseTimeContext{} - -func (t testParseTimeContext) GetRelativeParseTime() time.Time { - return time.Time(t) -} - -func TestGeospatialSize(t *testing.T) { - defer leaktest.AfterTest(t)() - testCases := []struct { - wkt string - expected uintptr + errorTestCases := []struct { + in string + expectedError string }{ - {"SRID=4004;POINT EMPTY", 73}, - {"SRID=4326;LINESTRING(0 0, 10 0)", 125}, + {`"unclosed`, `invalid name: unclosed ": "unclosed`}, + {`"unclosed""`, `invalid name: unclosed ": "unclosed""`}, + {`hello !`, `invalid name: expected separator .: hello !`}, } - for _, tc := range testCases { - t.Run(tc.wkt, func(t *testing.T) { - t.Run("geometry", func(t *testing.T) { - g, err := tree.ParseDGeometry(tc.wkt) - require.NoError(t, err) - require.Equal(t, tc.expected, g.Size()) - }) - t.Run("geography", func(t *testing.T) { - g, err := tree.ParseDGeography(tc.wkt) - require.NoError(t, err) - require.Equal(t, tc.expected, g.Size()) - }) + for _, tc := range errorTestCases { + t.Run(tc.in, func(t *testing.T) { + _, err := splitIdentifierList(tc.in) + require.EqualError(t, err, tc.expectedError) }) } } diff --git a/pkg/sql/sem/tree/fuzz.go b/pkg/sql/sem/tree/fuzz.go index 87b6d0ed86fe..59c8b9b5c559 100644 --- a/pkg/sql/sem/tree/fuzz.go +++ b/pkg/sql/sem/tree/fuzz.go @@ -27,7 +27,7 @@ func FuzzParseDDecimal(data []byte) int { } func FuzzParseDDate(data []byte) int { - _, err := ParseDDate(timeCtx, string(data)) + _, _, err := ParseDDate(timeCtx, string(data)) if err != nil { return 0 }