diff --git a/.bazelignore b/.bazelignore index 2d8c63520c94..687e0e3145cd 100644 --- a/.bazelignore +++ b/.bazelignore @@ -3,4 +3,5 @@ bin build/builder_home lib pkg/ui/node_modules +tmp vendor diff --git a/.gitignore b/.gitignore index b21d86e88b17..af5299ef8433 100644 --- a/.gitignore +++ b/.gitignore @@ -14,6 +14,7 @@ artifacts /bin.* /lib /lib.* +/tmp .buildinfo # cockroach-data, cockroach{,.race}-{darwin,linux,windows}-* /cockroach* diff --git a/dev b/dev index a11278ba46f8..d325cb7e70b2 100755 --- a/dev +++ b/dev @@ -3,7 +3,7 @@ set -euo pipefail # Bump this counter to force rebuilding `dev` on all machines. -DEV_VERSION=34 +DEV_VERSION=35 THIS_DIR=$(cd "$(dirname "$0")" && pwd) BINARY_DIR=$THIS_DIR/bin/dev-versions diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 0a9c86ef6223..2c22a28eed8e 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -677,6 +677,7 @@ func createImportingDescriptors( ) (*restorationDataBase, *mainRestorationData, error) { details := r.job.Details().(jobspb.RestoreDetails) + var allMutableDescs []catalog.MutableDescriptor var databases []catalog.DatabaseDescriptor var writtenTypes []catalog.TypeDescriptor var schemas []*schemadesc.Mutable @@ -706,19 +707,23 @@ func createImportingDescriptors( } tables = append(tables, mut) mutableTables = append(mutableTables, mut) + allMutableDescs = append(allMutableDescs, mut) oldTableIDs = append(oldTableIDs, mut.GetID()) case catalog.DatabaseDescriptor: if _, ok := details.DescriptorRewrites[desc.GetID()]; ok { mut := dbdesc.NewBuilder(desc.DatabaseDesc()).BuildCreatedMutableDatabase() databases = append(databases, mut) mutableDatabases = append(mutableDatabases, mut) + allMutableDescs = append(allMutableDescs, mut) } case catalog.SchemaDescriptor: mut := schemadesc.NewBuilder(desc.SchemaDesc()).BuildCreatedMutableSchema() schemas = append(schemas, mut) + allMutableDescs = append(allMutableDescs, mut) case catalog.TypeDescriptor: mut := typedesc.NewBuilder(desc.TypeDesc()).BuildCreatedMutableType() types = append(types, mut) + allMutableDescs = append(allMutableDescs, mut) } } @@ -814,6 +819,12 @@ func createImportingDescriptors( return nil, nil, err } + // Finally, clean up / update any schema changer state inside descriptors + // globally. + if err := rewrite.MaybeClearSchemaChangerStateInDescs(allMutableDescs); err != nil { + return nil, nil, err + } + // Set the new descriptors' states to offline. for _, desc := range mutableTables { desc.SetOffline("restoring") diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 54dd70707d1b..74cd1d273f9d 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -33,10 +33,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeeddist" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent" - // Imported to allow multi-tenant tests - _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" - // Imported to allow locality-related table mutations - _ "github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl" + _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" // multi-tenant tests + _ "github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl" // locality-related table mutations _ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" _ "github.com/cockroachdb/cockroach/pkg/cloud/impl" // registers cloud storage providers @@ -1292,16 +1290,16 @@ func TestChangefeedLaggingSpanCheckpointing(t *testing.T) { setErr := func(stp kvcoord.SpanTimePair, expectedTS hlc.Timestamp) { incorrectCheckpointErr = errors.Newf( "rangefeed for span %s expected to start @%s, started @%s instead", - stp.Span, expectedTS, stp.TS) + stp.Span, expectedTS, stp.StartAfter) } for _, sp := range spans { if laggingSpans.Encloses(sp.Span) { - if !sp.TS.Equal(cursor) { + if !sp.StartAfter.Equal(cursor) { setErr(sp, cursor) } } else { - if !sp.TS.Equal(checkpointTS) { + if !sp.StartAfter.Equal(checkpointTS) { setErr(sp, checkpointTS) } } diff --git a/pkg/ccl/changefeedccl/kvfeed/kv_feed.go b/pkg/ccl/changefeedccl/kvfeed/kv_feed.go index e50d3730e413..223cf8d18cfd 100644 --- a/pkg/ccl/changefeedccl/kvfeed/kv_feed.go +++ b/pkg/ccl/changefeedccl/kvfeed/kv_feed.go @@ -432,7 +432,7 @@ func (f *kvFeed) runUntilTableEvent( var stps []kvcoord.SpanTimePair resumeFrontier.Entries(func(s roachpb.Span, ts hlc.Timestamp) (done span.OpResult) { - stps = append(stps, kvcoord.SpanTimePair{Span: s, TS: ts}) + stps = append(stps, kvcoord.SpanTimePair{Span: s, StartAfter: ts}) return span.ContinueMatch }) diff --git a/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go b/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go index 9af4bfd349a8..f06741afb227 100644 --- a/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go +++ b/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go @@ -393,10 +393,10 @@ func (f rawEventFeed) run( withDiff bool, eventC chan<- *roachpb.RangeFeedEvent, ) error { - var startFrom hlc.Timestamp + var startAfter hlc.Timestamp for _, s := range spans { - if startFrom.IsEmpty() || s.TS.Less(startFrom) { - startFrom = s.TS + if startAfter.IsEmpty() || s.StartAfter.Less(startAfter) { + startAfter = s.StartAfter } } @@ -405,8 +405,8 @@ func (f rawEventFeed) run( var i int for i = range f { ev := f[i] - if ev.Val != nil && startFrom.LessEq(ev.Val.Value.Timestamp) || - ev.Checkpoint != nil && startFrom.LessEq(ev.Checkpoint.ResolvedTS) { + if ev.Val != nil && startAfter.LessEq(ev.Val.Value.Timestamp) || + ev.Checkpoint != nil && startAfter.LessEq(ev.Checkpoint.ResolvedTS) { break } diff --git a/pkg/cli/interactive_tests/test_timing.tcl b/pkg/cli/interactive_tests/test_timing.tcl index c6d928957f9d..0bf2e5439765 100644 --- a/pkg/cli/interactive_tests/test_timing.tcl +++ b/pkg/cli/interactive_tests/test_timing.tcl @@ -35,9 +35,6 @@ end_test start_test "Check that server times also work if IntervalStyle is different" # regression test for issue #67618. -send "set intervalstyle_enabled = 'on';\r" -eexpect "SET" -eexpect root@ send "set IntervalStyle = 'iso_8601';\r" eexpect "SET" eexpect root@ diff --git a/pkg/cmd/dev/doctor.go b/pkg/cmd/dev/doctor.go index 77a05d99da25..ca4edc01dc62 100644 --- a/pkg/cmd/dev/doctor.go +++ b/pkg/cmd/dev/doctor.go @@ -32,7 +32,7 @@ const ( // doctorStatusVersion is the current "version" of the status checks performed // by `dev doctor``. Increasing it will force doctor to be re-run before other // dev commands can be run. - doctorStatusVersion = 4 + doctorStatusVersion = 5 noCacheFlag = "no-cache" ) @@ -287,6 +287,15 @@ slightly slower and introduce a noticeable delay in first-time build setup.` log.Println(failedNogoTestMsg) } + // Check whether the user has configured a custom tmpdir. + present := d.checkLinePresenceInBazelRcUser(workspace, "test --test_tmpdir=") + if !present { + failures = append(failures, "You haven't configured a tmpdir for your tests.\n"+ + "Please add a `test --test_tmpdir=/PATH/TO/TMPDIR` line to your .bazelrc.user:\n"+ + fmt.Sprintf(" echo \"test --test_tmpdir=%s\" >> .bazelrc.user\n", filepath.Join(workspace, "tmp"))+ + "(You can choose any directory as a tmpdir.)") + } + // We want to make sure there are no other failures before trying to // set up the cache. if !noCache && len(failures) == 0 { @@ -359,3 +368,19 @@ func (d *dev) checkPresenceInBazelRc(expectedBazelRcLine string) (string, error) } return errString, nil } + +// checkLinePresenceInBazelRcUser checks whether the .bazelrc.user file +// contains a line starting with the given prefix. Returns true iff a matching +// line is in the file. Failures to find the file are ignored. +func (d *dev) checkLinePresenceInBazelRcUser(workspace, expectedSubstr string) bool { + contents, err := d.os.ReadFile(filepath.Join(workspace, ".bazelrc.user")) + if err != nil { + return false + } + for _, line := range strings.Split(contents, "\n") { + if strings.HasPrefix(line, expectedSubstr) { + return true + } + } + return false +} diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go index 289d9613721e..3097d82c1983 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go @@ -40,9 +40,9 @@ import ( ) type singleRangeInfo struct { - rs roachpb.RSpan - startFrom hlc.Timestamp - token rangecache.EvictionToken + rs roachpb.RSpan + startAfter hlc.Timestamp + token rangecache.EvictionToken } var useDedicatedRangefeedConnectionClass = settings.RegisterBoolSetting( @@ -75,27 +75,32 @@ func maxConcurrentCatchupScans(sv *settings.Values) int { // // Note that the timestamps in RangeFeedCheckpoint events that are streamed back // may be lower than the timestamp given here. +// +// NB: the given startAfter timestamp is exclusive, i.e. the first possible +// emitted event (including catchup scans) will be at startAfter.Next(). func (ds *DistSender) RangeFeed( ctx context.Context, spans []roachpb.Span, - startFrom hlc.Timestamp, + startAfter hlc.Timestamp, // exclusive withDiff bool, eventCh chan<- *roachpb.RangeFeedEvent, ) error { timedSpans := make([]SpanTimePair, 0, len(spans)) for _, sp := range spans { timedSpans = append(timedSpans, SpanTimePair{ - Span: sp, - TS: startFrom, + Span: sp, + StartAfter: startAfter, }) } return ds.RangeFeedSpans(ctx, timedSpans, withDiff, eventCh) } -// SpanTimePair is a pair of span along with its starting time. +// SpanTimePair is a pair of span along with its starting time. The starting +// time is exclusive, i.e. the first possible emitted event (including catchup +// scans) will be at startAfter.Next(). type SpanTimePair struct { - Span roachpb.Span - TS hlc.Timestamp + Span roachpb.Span + StartAfter hlc.Timestamp // exclusive } // RangeFeedSpans is similar to RangeFeed but allows specification of different @@ -128,7 +133,7 @@ func (ds *DistSender) RangeFeedSpans( case sri := <-rangeCh: // Spawn a child goroutine to process this feed. g.GoCtx(func(ctx context.Context) error { - return ds.partialRangeFeed(ctx, rr, sri.rs, sri.startFrom, sri.token, withDiff, &catchupSem, rangeCh, eventCh) + return ds.partialRangeFeed(ctx, rr, sri.rs, sri.startAfter, sri.token, withDiff, &catchupSem, rangeCh, eventCh) }) case <-ctx.Done(): return ctx.Err() @@ -144,7 +149,7 @@ func (ds *DistSender) RangeFeedSpans( if err != nil { return err } - return ds.divideAndSendRangeFeedToRanges(ctx, rs, stp.TS, rangeCh) + return ds.divideAndSendRangeFeedToRanges(ctx, rs, stp.StartAfter, rangeCh) }) }(s) } @@ -164,7 +169,7 @@ type RangeFeedContext struct { // PartialRangeFeed structure describes the state of currently executing partial range feed. type PartialRangeFeed struct { Span roachpb.Span - StartTS hlc.Timestamp + StartAfter hlc.Timestamp // exclusive NodeID roachpb.NodeID RangeID roachpb.RangeID CreatedTime time.Time @@ -249,7 +254,7 @@ func newRangeFeedRegistry(ctx context.Context, withDiff bool) *rangeFeedRegistry } func (ds *DistSender) divideAndSendRangeFeedToRanges( - ctx context.Context, rs roachpb.RSpan, startFrom hlc.Timestamp, rangeCh chan<- singleRangeInfo, + ctx context.Context, rs roachpb.RSpan, startAfter hlc.Timestamp, rangeCh chan<- singleRangeInfo, ) error { // As RangeIterator iterates, it can return overlapping descriptors (and // during splits, this happens frequently), but divideAndSendRangeFeedToRanges @@ -267,9 +272,9 @@ func (ds *DistSender) divideAndSendRangeFeedToRanges( nextRS.Key = partialRS.EndKey select { case rangeCh <- singleRangeInfo{ - rs: partialRS, - startFrom: startFrom, - token: ri.Token(), + rs: partialRS, + startAfter: startAfter, + token: ri.Token(), }: case <-ctx.Done(): return ctx.Err() @@ -289,7 +294,7 @@ func (ds *DistSender) partialRangeFeed( ctx context.Context, rr *rangeFeedRegistry, rs roachpb.RSpan, - startFrom hlc.Timestamp, + startAfter hlc.Timestamp, token rangecache.EvictionToken, withDiff bool, catchupSem *limit.ConcurrentRequestLimiter, @@ -303,7 +308,7 @@ func (ds *DistSender) partialRangeFeed( active := &activeRangeFeed{ PartialRangeFeed: PartialRangeFeed{ Span: span, - StartTS: startFrom, + StartAfter: startAfter, CreatedTime: timeutil.Now(), }, } @@ -329,16 +334,16 @@ func (ds *DistSender) partialRangeFeed( } // Establish a RangeFeed for a single Range. - maxTS, err := ds.singleRangeFeed(ctx, span, startFrom, withDiff, token.Desc(), + maxTS, err := ds.singleRangeFeed(ctx, span, startAfter, withDiff, token.Desc(), catchupSem, eventCh, active.onRangeEvent) // Forward the timestamp in case we end up sending it again. - startFrom.Forward(maxTS) + startAfter.Forward(maxTS) if err != nil { if log.V(1) { log.Infof(ctx, "RangeFeed %s disconnected with last checkpoint %s ago: %v", - span, timeutil.Since(startFrom.GoTime()), err) + span, timeutil.Since(startAfter.GoTime()), err) } switch { case errors.HasType(err, (*roachpb.StoreNotFoundError)(nil)) || @@ -354,7 +359,7 @@ func (ds *DistSender) partialRangeFeed( case errors.HasType(err, (*roachpb.RangeKeyMismatchError)(nil)): // Evict the descriptor from the cache. token.Evict(ctx) - return ds.divideAndSendRangeFeedToRanges(ctx, rs, startFrom, rangeCh) + return ds.divideAndSendRangeFeedToRanges(ctx, rs, startAfter, rangeCh) case errors.HasType(err, (*roachpb.RangeFeedRetryError)(nil)): var t *roachpb.RangeFeedRetryError if ok := errors.As(err, &t); !ok { @@ -373,7 +378,7 @@ func (ds *DistSender) partialRangeFeed( roachpb.RangeFeedRetryError_REASON_NO_LEASEHOLDER: // Evict the descriptor from the cache. token.Evict(ctx) - return ds.divideAndSendRangeFeedToRanges(ctx, rs, startFrom, rangeCh) + return ds.divideAndSendRangeFeedToRanges(ctx, rs, startAfter, rangeCh) default: return errors.AssertionFailedf("unrecognized retryable error type: %T", err) } @@ -398,7 +403,7 @@ type onRangeEventCb func(nodeID roachpb.NodeID, rangeID roachpb.RangeID, event * func (ds *DistSender) singleRangeFeed( ctx context.Context, span roachpb.Span, - startFrom hlc.Timestamp, + startAfter hlc.Timestamp, withDiff bool, desc *roachpb.RangeDescriptor, catchupSem *limit.ConcurrentRequestLimiter, @@ -412,7 +417,7 @@ func (ds *DistSender) singleRangeFeed( args := roachpb.RangeFeedRequest{ Span: span, Header: roachpb.Header{ - Timestamp: startFrom, + Timestamp: startAfter, RangeID: desc.RangeID, }, WithDiff: withDiff, diff --git a/pkg/kv/kvclient/rangefeed/rangefeed.go b/pkg/kv/kvclient/rangefeed/rangefeed.go index 85e9e8a7e9f1..a1aff99f2f82 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed.go @@ -114,6 +114,11 @@ func newFactory(stopper *stop.Stopper, client DB, knobs *TestingKnobs) *Factory // // The only error which can be returned will indicate that the server is being // shut down. +// +// NB: for the rangefeed itself, initialTimestamp is exclusive, i.e. the first +// possible event emitted by the server (including the catchup scan) is at +// initialTimestamp.Next(). This follows from the gRPC API semantics. However, +// the initial scan (if any) is run at initialTimestamp. func (f *Factory) RangeFeed( ctx context.Context, name string, diff --git a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go index 199127f0a2bb..dbecff59553c 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go @@ -876,3 +876,59 @@ func TestRangefeedWithLabelsOption(t *testing.T) { defer allLabelsCorrect.Unlock() require.True(t, allLabelsCorrect.correct) } + +// TestRangeFeedStartTimeExclusive tests that the start timestamp of the +// rangefeed is in fact exclusive, as specified. +func TestRangeFeedStartTimeExclusive(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + + srv0 := tc.Server(0) + db := srv0.DB() + scratchKey := tc.ScratchRange(t) + scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] + mkKey := func(k string) roachpb.Key { + return encoding.EncodeStringAscending(scratchKey, k) + } + span := roachpb.Span{Key: scratchKey, EndKey: scratchKey.PrefixEnd()} + + // Write three versions of "foo". Get the timestamp of the second version. + require.NoError(t, db.Put(ctx, mkKey("foo"), 1)) + require.NoError(t, db.Put(ctx, mkKey("foo"), 2)) + kv, err := db.Get(ctx, mkKey("foo")) + require.NoError(t, err) + ts2 := kv.Value.Timestamp + require.NoError(t, db.Put(ctx, mkKey("foo"), 3)) + + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + _, err = tc.ServerConn(0).Exec("SET CLUSTER SETTING kv.rangefeed.enabled = true") + require.NoError(t, err) + + f, err := rangefeed.NewFactory(srv0.Stopper(), db, srv0.ClusterSettings(), nil) + require.NoError(t, err) + rows := make(chan *roachpb.RangeFeedValue) + r, err := f.RangeFeed(ctx, "test", []roachpb.Span{span}, ts2, + func(ctx context.Context, value *roachpb.RangeFeedValue) { + select { + case rows <- value: + case <-ctx.Done(): + } + }, + ) + require.NoError(t, err) + defer r.Close() + + // The first emitted version should be 3. + select { + case row := <-rows: + require.Equal(t, mkKey("foo"), row.Key) + v, err := row.Value.GetInt() + require.NoError(t, err) + require.EqualValues(t, 3, v) + case <-time.After(3 * time.Second): + t.Fatal("timed out waiting for event") + } +} diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan.go b/pkg/kv/kvserver/rangefeed/catchup_scan.go index 226881a9a342..f93f6daa4d7c 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan.go @@ -53,6 +53,9 @@ func (i simpleCatchupIterAdapter) NextIgnoringTime() { var _ simpleCatchupIter = simpleCatchupIterAdapter{} // NewCatchUpIterator returns a CatchUpIterator for the given Reader. +// +// NB: The start timestamp given in args.Header.Timestamp is exclusive, i.e. the +// first possible event will be emitted at Timestamp.Next(). func NewCatchUpIterator( reader storage.Reader, args *roachpb.RangeFeedRequest, closer func(), ) *CatchUpIterator { @@ -61,10 +64,8 @@ func NewCatchUpIterator( storage.MVCCIncrementalIterOptions{ EnableTimeBoundIteratorOptimization: true, EndKey: args.Span.EndKey, - // StartTime is exclusive but args.Timestamp - // is inclusive. - StartTime: args.Timestamp.Prev(), - EndTime: hlc.MaxTimestamp, + StartTime: args.Timestamp, + EndTime: hlc.MaxTimestamp, // We want to emit intents rather than error // (the default behavior) so that we can skip // over the provisional values during diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go index b05684b44374..8cc2ace13dee 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go @@ -94,8 +94,7 @@ func TestCatchupScan(t *testing.T) { testutils.RunTrueAndFalse(t, "withDiff", func(t *testing.T, withDiff bool) { iter := NewCatchUpIterator(eng, &roachpb.RangeFeedRequest{ Header: roachpb.Header{ - // Inclusive, so want everything >= ts2 - Timestamp: ts2, + Timestamp: ts1, // exclusive }, Span: roachpb.Span{ EndKey: roachpb.KeyMax, diff --git a/pkg/kv/kvserver/rangefeed/processor.go b/pkg/kv/kvserver/rangefeed/processor.go index 6d2138d57e17..36646fc8299e 100644 --- a/pkg/kv/kvserver/rangefeed/processor.go +++ b/pkg/kv/kvserver/rangefeed/processor.go @@ -436,7 +436,7 @@ func (p *Processor) sendStop(pErr *roachpb.Error) { // provided an error when the registration closes. // // The optionally provided "catch-up" iterator is used to read changes from the -// engine which occurred after the provided start timestamp. +// engine which occurred after the provided start timestamp (exclusive). // // If the method returns false, the processor will have been stopped, so calling // Stop is not necessary. If the method returns true, it will also return an @@ -444,6 +444,8 @@ func (p *Processor) sendStop(pErr *roachpb.Error) { // registration. // // NOT safe to call on nil Processor. +// +// NB: startTS is exclusive; the first possible event will be at startTS.Next(). func (p *Processor) Register( span roachpb.RSpan, startTS hlc.Timestamp, diff --git a/pkg/kv/kvserver/rangefeed/registry.go b/pkg/kv/kvserver/rangefeed/registry.go index 3b0c517b74a4..311447d07bd9 100644 --- a/pkg/kv/kvserver/rangefeed/registry.go +++ b/pkg/kv/kvserver/rangefeed/registry.go @@ -76,7 +76,7 @@ func putPooledSharedEvent(e *sharedEvent) { type registration struct { // Input. span roachpb.Span - catchUpTimestamp hlc.Timestamp + catchUpTimestamp hlc.Timestamp // exclusive withDiff bool metrics *Metrics diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index c98fec3ca9e3..aaaab6295144 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -323,7 +323,7 @@ func logSlowRangefeedRegistration(ctx context.Context) func() { func (r *Replica) registerWithRangefeedRaftMuLocked( ctx context.Context, span roachpb.RSpan, - startTS hlc.Timestamp, + startTS hlc.Timestamp, // exclusive catchUpIter rangefeed.CatchUpIteratorConstructor, withDiff bool, stream rangefeed.Stream, diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 588741cc6c79..d46f95636687 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -2681,7 +2681,11 @@ message RangeLookupResponse { } // RangeFeedRequest is a request that expresses the intention to establish a -// RangeFeed stream over the provided span, starting at the specified timestamp. +// RangeFeed stream over the provided span, starting at the specified timestamp +// (exclusive). +// +// NB: The start timestamp is exclusive, i.e. the first possible event emitted +// will be at Header.Timestamp.Next(). This includes catchup scans. message RangeFeedRequest { Header header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; Span span = 2 [(gogoproto.nullable) = false]; diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index fdfae86d90a3..9ce5f39fdaa1 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -129,6 +129,8 @@ var retiredSettings = map[string]struct{}{ // removed as of 22.2. "kv.rangefeed.catchup_scan_iterator_optimization.enabled": {}, + "sql.defaults.datestyle.enabled": {}, + "sql.defaults.intervalstyle.enabled": {}, } var sqlDefaultSettings = map[string]struct{}{ diff --git a/pkg/sql/catalog/internal/validate/schema_changer_state.go b/pkg/sql/catalog/internal/validate/schema_changer_state.go index 446824f72a8f..281b0a3f5e87 100644 --- a/pkg/sql/catalog/internal/validate/schema_changer_state.go +++ b/pkg/sql/catalog/internal/validate/schema_changer_state.go @@ -48,7 +48,6 @@ func validateSchemaChangerState(d catalog.Descriptor, vea catalog.ValidationErro report(errors.Errorf("target %d corresponds to descriptor %d != %d", i, got, d.GetID())) } - switch t.TargetStatus { case scpb.Status_PUBLIC, scpb.Status_ABSENT, scpb.Status_TRANSIENT_ABSENT: // These are valid target status values. diff --git a/pkg/sql/catalog/post_derserialization_changes.go b/pkg/sql/catalog/post_derserialization_changes.go index 98be157acbf4..b1e7fa74dc86 100644 --- a/pkg/sql/catalog/post_derserialization_changes.go +++ b/pkg/sql/catalog/post_derserialization_changes.go @@ -86,10 +86,4 @@ const ( // dropping a schema, we'd mark the database itself as though it was the // schema which was dropped. RemovedSelfEntryInSchemas - - // FixedDateStyleIntervalStyleCast indicates that the table had at least - // one computed column or partial indexes that had a stable cast within it. - // The stable cast was rewritten so that it would no longer cause - // inconsistencies when DateStyle/IntervalStyle is enabled. - FixedDateStyleIntervalStyleCast ) diff --git a/pkg/sql/catalog/rewrite/rewrite.go b/pkg/sql/catalog/rewrite/rewrite.go index e6d0733b33b9..775cab07b79f 100644 --- a/pkg/sql/catalog/rewrite/rewrite.go +++ b/pkg/sql/catalog/rewrite/rewrite.go @@ -394,6 +394,35 @@ func rewriteIDsInTypesT(typ *types.T, descriptorRewrites jobspb.DescRewriteMap) return nil } +// MaybeClearSchemaChangerStateInDescs goes over all mutable descriptors and +// cleans any state information from descriptors which have no targets associated +// with the corresponding jobs. The state is used to lock a descriptor to ensure +// no concurrent schema change jobs can occur, which needs to be cleared if no +// jobs exist working on *any* targets, since otherwise the descriptor would +// be left locked. +func MaybeClearSchemaChangerStateInDescs(descriptors []catalog.MutableDescriptor) error { + nonEmptyJobs := make(map[jobspb.JobID]struct{}) + // Track all the schema changer states that have a non-empty job associated + // with them. + for _, desc := range descriptors { + if state := desc.GetDeclarativeSchemaChangerState(); state != nil && + len(state.Targets) > 0 { + nonEmptyJobs[state.JobID] = struct{}{} + } + } + // Clean up any schema changer states that have empty jobs that don't have any + // targets associated. + for _, desc := range descriptors { + if state := desc.GetDeclarativeSchemaChangerState(); state != nil && + len(state.Targets) == 0 { + if _, found := nonEmptyJobs[state.JobID]; !found { + desc.SetDeclarativeSchemaChangerState(nil) + } + } + } + return nil +} + // TypeDescs rewrites all ID's in the input slice of TypeDescriptors // using the input ID rewrite mapping. func TypeDescs(types []*typedesc.Mutable, descriptorRewrites jobspb.DescRewriteMap) error { @@ -522,9 +551,6 @@ func rewriteSchemaChangerState( // TODO(ajwerner): Remember to rewrite views when the time comes. Currently // views are not handled by the declarative schema changer. } - if len(state.Targets) == 0 { - d.SetDeclarativeSchemaChangerState(nil) - } return nil } diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index 069dc4d67a82..2a98395c2bea 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -4,7 +4,6 @@ go_library( name = "tabledesc", srcs = [ "column.go", - "fix_cast_for_style_visitor.go", "index.go", "mutation.go", "safe_format.go", @@ -40,7 +39,6 @@ go_library( "//pkg/sql/privilege", "//pkg/sql/rowenc", "//pkg/sql/schemachanger/scpb", - "//pkg/sql/sem/cast", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", @@ -64,7 +62,6 @@ go_test( name = "tabledesc_test", size = "small", srcs = [ - "fix_cast_for_style_visitor_test.go", "helpers_test.go", "index_test.go", "main_test.go", @@ -96,20 +93,15 @@ go_test( "//pkg/sql/catalog/desctestutils", "//pkg/sql/catalog/internal/validate", "//pkg/sql/catalog/nstree", - "//pkg/sql/parser", "//pkg/sql/privilege", - "//pkg/sql/sem/cast", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/catid", - "//pkg/sql/sem/tree", - "//pkg/sql/tests", "//pkg/sql/types", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/util/hlc", "//pkg/util/leaktest", - "//pkg/util/log", "//pkg/util/protoutil", "//pkg/util/randutil", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/catalog/tabledesc/fix_cast_for_style_visitor.go b/pkg/sql/catalog/tabledesc/fix_cast_for_style_visitor.go deleted file mode 100644 index fdd5cb694453..000000000000 --- a/pkg/sql/catalog/tabledesc/fix_cast_for_style_visitor.go +++ /dev/null @@ -1,171 +0,0 @@ -// Copyright 2022 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 tabledesc - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" - "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" - "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/errors" -) - -// fixCastForStyleVisitor is used to rewrite cast expressions that contain casts -// that cause formatting issues when DateStyle/IntervalStyle is enabled. The -// issue is caused by the volatility of the cast being stable. The expression -// passed into fixCastForStyleVisitor must be a TypedExpr. -// fixCastForStyleVisitor replaces these casts with a builtin that is an -// immutable version of the cast. There are casts that are disallowed due to -// them being context dependent, the visitor will still check for them and -// replace them. This is to account for the possibility these casts exist in an -// older cluster. These casts are as follows: string::date, string::timestamp, -// string::timestamptz, string::time, timestamptz::string, string::timetz -type fixCastForStyleVisitor struct { - ctx context.Context - semaCtx *tree.SemaContext - tDesc catalog.TableDescriptor - err error -} - -var _ tree.Visitor = &fixCastForStyleVisitor{} - -// VisitPre implements the Visitor interface. -func (v *fixCastForStyleVisitor) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) { - if v.err != nil { - return false, expr - } - // If the expression is already immutable this will exit early and avoid - // checking the expression further. - _, _, _, err := schemaexpr.DequalifyAndValidateExpr( - v.ctx, - v.tDesc, - expr, - types.Any, - "fixCastForStyleVisitor", - v.semaCtx, - volatility.Immutable, - tree.NewUnqualifiedTableName(tree.Name(v.tDesc.GetName())), - ) - if err == nil { - return false, expr - } - - return true, expr -} - -// VisitPost implements the Visitor interface. -func (v *fixCastForStyleVisitor) VisitPost(expr tree.Expr) tree.Expr { - if v.err != nil { - return expr - } - - if expr, ok := expr.(*tree.CastExpr); ok { - // We only perform type-checking for CastExprs, so that we avoid type- - // checking expressions that contain user-defined types, since those types - // cannot be resolved in this context. - typedExpr, err := schemaexpr.DequalifyAndTypeCheckExpr( - v.ctx, - v.tDesc, - expr, - v.semaCtx, - tree.NewUnqualifiedTableName(tree.Name(v.tDesc.GetName())), - ) - if err != nil { - // Don't return or save the error here. If the expression can't be - // type-checked, then it can't be rewritten, and that should not block - // RESTORE or a database upgrade. - return expr - } - expr = typedExpr.(*tree.CastExpr) - - innerExpr := expr.Expr.(tree.TypedExpr) - outerTyp := expr.ResolvedType() - innerTyp := innerExpr.ResolvedType() - vol, ok := cast.LookupCastVolatility(innerTyp, outerTyp, v.semaCtx.CastSessionOptions) - if !ok { - v.err = errors.AssertionFailedf("Not a valid cast %s -> %s", innerTyp.SQLString(), outerTyp.SQLString()) - } - if vol <= volatility.Immutable { - return expr - } - - var newExpr tree.Expr - switch innerTyp.Family() { - case types.StringFamily: - switch outerTyp.Family() { - case types.IntervalFamily: - newExpr = &tree.CastExpr{ - Expr: &tree.FuncExpr{Func: tree.WrapFunction("parse_interval"), Exprs: tree.Exprs{expr.Expr}}, - Type: expr.Type, - SyntaxMode: tree.CastShort, - } - return newExpr - case types.DateFamily: - newExpr = &tree.CastExpr{ - Expr: &tree.FuncExpr{Func: tree.WrapFunction("parse_date"), Exprs: tree.Exprs{expr.Expr}}, - Type: expr.Type, - SyntaxMode: tree.CastShort, - } - return newExpr - case types.TimeFamily: - newExpr = &tree.CastExpr{ - Expr: &tree.FuncExpr{Func: tree.WrapFunction("parse_time"), Exprs: tree.Exprs{expr.Expr}}, - Type: expr.Type, - SyntaxMode: tree.CastShort, - } - return newExpr - case types.TimeTZFamily: - newExpr = &tree.CastExpr{ - Expr: &tree.FuncExpr{Func: tree.WrapFunction("parse_timetz"), Exprs: tree.Exprs{expr.Expr}}, - Type: expr.Type, - SyntaxMode: tree.CastShort, - } - return newExpr - case types.TimestampFamily: - newExpr = &tree.CastExpr{ - Expr: &tree.FuncExpr{Func: tree.WrapFunction("parse_timestamp"), Exprs: tree.Exprs{expr.Expr}}, - Type: expr.Type, - SyntaxMode: tree.CastShort, - } - return newExpr - } - case types.IntervalFamily, types.DateFamily, types.TimestampFamily, types.TimeFamily, types.TimeTZFamily, types.TimestampTZFamily: - if outerTyp.Family() == types.StringFamily { - newExpr = &tree.CastExpr{ - Expr: &tree.FuncExpr{Func: tree.WrapFunction("to_char"), Exprs: tree.Exprs{expr.Expr}}, - Type: expr.Type, - SyntaxMode: tree.CastShort, - } - return newExpr - } - } - } - return expr -} - -// ResolveCastForStyleUsingVisitor checks expression for stable cast that affect -// DateStyle/IntervalStyle and rewrites them. -func ResolveCastForStyleUsingVisitor( - ctx context.Context, semaCtx *tree.SemaContext, desc *descpb.TableDescriptor, expr tree.Expr, -) (tree.Expr, bool, error) { - v := &fixCastForStyleVisitor{ctx: ctx, semaCtx: semaCtx} - descBuilder := NewBuilder(desc) - tDesc := descBuilder.BuildImmutableTable() - v.tDesc = tDesc - - expr, changed := tree.WalkExpr(v, expr) - return expr, changed, v.err -} diff --git a/pkg/sql/catalog/tabledesc/fix_cast_for_style_visitor_test.go b/pkg/sql/catalog/tabledesc/fix_cast_for_style_visitor_test.go deleted file mode 100644 index 0a36ac783fdf..000000000000 --- a/pkg/sql/catalog/tabledesc/fix_cast_for_style_visitor_test.go +++ /dev/null @@ -1,140 +0,0 @@ -// Copyright 2022 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 tabledesc_test - -import ( - "context" - "testing" - - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" - "github.com/cockroachdb/cockroach/pkg/sql/parser" - "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/tests" - "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/stretchr/testify/require" -) - -func TestFixCastForStyleVisitor(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - params, _ := tests.CreateTestServerParams() - s, sqlDB, kvDB := serverutils.StartServer(t, params) - ctx := context.Background() - var semaCtx tree.SemaContext - defer s.Stopper().Stop(context.Background()) - - if _, err := sqlDB.Exec(` -CREATE DATABASE t; -CREATE TABLE t.ds (it INTERVAL, s STRING, vc VARCHAR, c CHAR, t TIMESTAMP, n NAME, d DATE); -`); err != nil { - t.Fatal(err) - } - - desc := desctestutils.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "public", "ds") - tDesc := desc.TableDesc() - - tests := []struct { - expr string - expect string - }{ - { - expr: "s::INTERVAL", - expect: "parse_interval(s)::INTERVAL", - }, - { - expr: "s::INTERVAL(4)", - expect: "parse_interval(s)::INTERVAL(4)", - }, - { - expr: "vc::DATE", - expect: "parse_date(vc)::DATE", - }, - { - expr: "n::TIME", - expect: "parse_time(n)::TIME", - }, - { - expr: "n::TIME(5)", - expect: "parse_time(n)::TIME(5)", - }, - { - expr: "parse_interval(s)", - expect: "parse_interval(s)", - }, - { - expr: "s::INT", - expect: "s::INT8", - }, - { - expr: "it::TEXT", - expect: "to_char(it)::STRING", - }, - { - expr: "vc::TIMETZ", - expect: "parse_timetz(vc)::TIMETZ", - }, - { - expr: "t::TIME", - expect: "t::TIME", - }, - { - expr: "s::TIME", - expect: "parse_time(s)::TIME", - }, - { - expr: `it::STRING = 'abc'`, - expect: `to_char(it)::STRING = 'abc'`, - }, - { - expr: "lower(it::STRING)", - expect: "lower(to_char(it)::STRING)", - }, - { - expr: "s::TIMESTAMPTZ::STRING", - expect: "to_char(s::TIMESTAMPTZ)::STRING", - }, - { - expr: "extract(epoch from s::TIME)", - expect: "extract('epoch', parse_time(s)::TIME)", - }, - { - expr: "extract(epoch from s::DATE)", - expect: "extract('epoch', parse_date(s)::DATE)", - }, - } - - for _, test := range tests { - t.Run(test.expr, func(t *testing.T) { - semaCtx.CastSessionOptions = cast.SessionOptions{ - IntervalStyleEnabled: true, - DateStyleEnabled: true, - } - expr, err := parser.ParseExpr(test.expr) - require.NoError(t, err) - newExpr, _, err := tabledesc.ResolveCastForStyleUsingVisitor( - ctx, - &semaCtx, - tDesc, - expr, - ) - if err != nil { - require.Equal(t, test.expect, err.Error()) - } else { - require.Equal(t, test.expect, newExpr.String()) - } - }) - } -} diff --git a/pkg/sql/catalog/tabledesc/table_desc_builder.go b/pkg/sql/catalog/tabledesc/table_desc_builder.go index 3f8fd20db848..72a1fa2f95e6 100644 --- a/pkg/sql/catalog/tabledesc/table_desc_builder.go +++ b/pkg/sql/catalog/tabledesc/table_desc_builder.go @@ -11,17 +11,12 @@ package tabledesc import ( - "context" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/privilege" - "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" ) @@ -255,81 +250,9 @@ func maybeFillInDescriptor( set(catalog.UpgradedPrivileges, fixedPrivileges || addedGrantOptions) set(catalog.RemovedDuplicateIDsInRefs, maybeRemoveDuplicateIDsInRefs(desc)) set(catalog.AddedConstraintIDs, maybeAddConstraintIDs(desc)) - - rewrittenCast, err := maybeRewriteCast(desc) - if err != nil { - return changes, err - } - set(catalog.FixedDateStyleIntervalStyleCast, rewrittenCast) return changes, nil } -// maybeRewriteCast rewrites stable cast in computed columns, indexes and -// partial indexes that cause issues with DateStyle/IntervalStyle -func maybeRewriteCast(desc *descpb.TableDescriptor) (hasChanged bool, err error) { - // We skip the system tables due to type checking not working properly during - // init time. - if desc.ParentID == keys.SystemDatabaseID { - return false, nil - } - - ctx := context.Background() - var semaCtx tree.SemaContext - semaCtx.CastSessionOptions = cast.SessionOptions{ - IntervalStyleEnabled: true, - DateStyleEnabled: true, - } - hasChanged = false - - for i, col := range desc.Columns { - if col.IsComputed() { - expr, err := parser.ParseExpr(*col.ComputeExpr) - if err != nil { - return hasChanged, err - } - newExpr, changed, err := ResolveCastForStyleUsingVisitor( - ctx, - &semaCtx, - desc, - expr, - ) - if err != nil { - return hasChanged, err - } - if changed { - hasChanged = true - s := tree.Serialize(newExpr) - desc.Columns[i].ComputeExpr = &s - } - } - } - - for i, idx := range desc.Indexes { - if idx.IsPartial() { - expr, err := parser.ParseExpr(idx.Predicate) - - if err != nil { - return hasChanged, err - } - newExpr, changed, err := ResolveCastForStyleUsingVisitor( - ctx, - &semaCtx, - desc, - expr, - ) - if err != nil { - return hasChanged, err - } - if changed { - hasChanged = true - s := tree.Serialize(newExpr) - desc.Indexes[i].Predicate = s - } - } - } - return hasChanged, nil -} - // maybeRemoveDefaultExprFromComputedColumns removes DEFAULT expressions on // computed columns. Although we now have a descriptor validation check to // prevent this, this hasn't always been the case, so it's theoretically diff --git a/pkg/sql/colexec/colexecbase/cast_test.go b/pkg/sql/colexec/colexecbase/cast_test.go index 967379e8b77d..1c13cc734393 100644 --- a/pkg/sql/colexec/colexecbase/cast_test.go +++ b/pkg/sql/colexec/colexecbase/cast_test.go @@ -54,7 +54,7 @@ func TestRandomizedCast(t *testing.T) { // below). continue } - if _, ok := cast.LookupCastVolatility(from, to, cast.SessionOptions{}); ok { + if _, ok := cast.LookupCastVolatility(from, to); ok { if colexecbase.IsCastSupported(from, to) { return from, to } diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 36e4df5da257..cde74ea25512 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -21,7 +21,6 @@ import ( "time" "unicode/utf8" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" @@ -43,7 +42,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun" "github.com/cockroachdb/cockroach/pkg/sql/sem/asof" - "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -2797,17 +2795,6 @@ func (ex *connExecutor) resetPlanner( p.cancelChecker.Reset(ctx) p.semaCtx = tree.MakeSemaContext() - if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.DateStyleIntervalStyleCastRewrite) { - p.semaCtx.CastSessionOptions = cast.SessionOptions{ - IntervalStyleEnabled: true, - DateStyleEnabled: true, - } - } else { - p.semaCtx.CastSessionOptions = cast.SessionOptions{ - IntervalStyleEnabled: ex.sessionData().IntervalStyleEnabled, - DateStyleEnabled: ex.sessionData().DateStyleEnabled, - } - } p.semaCtx.SearchPath = &ex.sessionData().SearchPath p.semaCtx.Annotations = nil p.semaCtx.TypeResolver = p diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 328ffdffb350..ad3790475e0f 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -5502,6 +5502,7 @@ GROUP BY var crdbInternalActiveRangeFeedsTable = virtualSchemaTable{ comment: `node-level table listing all currently running range feeds`, + // NB: startTS is exclusive; consider renaming to startAfter. schema: ` CREATE TABLE crdb_internal.active_range_feeds ( id INT, @@ -5529,7 +5530,7 @@ CREATE TABLE crdb_internal.active_range_feeds ( return addRow( tree.NewDInt(tree.DInt(rfCtx.ID)), tree.NewDString(rfCtx.CtxTags), - tree.NewDString(rf.StartTS.AsOfSystemTime()), + tree.NewDString(rf.StartAfter.AsOfSystemTime()), tree.MakeDBool(tree.DBool(rfCtx.WithDiff)), tree.NewDInt(tree.DInt(rf.NodeID)), tree.NewDInt(tree.DInt(rf.RangeID)), diff --git a/pkg/sql/discard.go b/pkg/sql/discard.go index 258987d26048..3ea8e35382f0 100644 --- a/pkg/sql/discard.go +++ b/pkg/sql/discard.go @@ -47,15 +47,6 @@ func (p *planner) Discard(ctx context.Context, s *tree.Discard) (planNode, error } func resetSessionVars(ctx context.Context, m sessionDataMutator) error { - // Always do intervalstyle_enabled and datestyle_enabled first so that - // IntervalStyle and DateStyle which depend on these flags are correctly - // configured. - if err := resetSessionVar(ctx, m, "datestyle_enabled"); err != nil { - return err - } - if err := resetSessionVar(ctx, m, "intervalstyle_enabled"); err != nil { - return err - } for _, varName := range varNames { if err := resetSessionVar(ctx, m, varName); err != nil { return err diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index e93550561fda..f15d5d1e85d7 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -638,28 +638,6 @@ var dateStyle = settings.RegisterEnumSetting( dateStyleEnumMap, ).WithPublic() -const intervalStyleEnabledClusterSetting = "sql.defaults.intervalstyle.enabled" - -// intervalStyleEnabled controls intervals representation. -// TODO(sql-exp): retire this setting in 22.2. -var intervalStyleEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, - intervalStyleEnabledClusterSetting, - "default value for intervalstyle_enabled session setting", - false, -) - -const dateStyleEnabledClusterSetting = "sql.defaults.datestyle.enabled" - -// dateStyleEnabled controls dates representation. -// TODO(sql-exp): retire this setting in 22.2. -var dateStyleEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, - dateStyleEnabledClusterSetting, - "default value for datestyle_enabled session setting", - false, -) - var txnRowsWrittenLog = settings.RegisterIntSetting( settings.TenantWritable, "sql.defaults.transaction_rows_written_log", @@ -3173,16 +3151,6 @@ func (m *sessionDataMutator) SetDateStyle(style pgdate.DateStyle) { m.bufferParamStatusUpdate("DateStyle", style.SQLString()) } -// SetIntervalStyleEnabled sets the IntervalStyleEnabled for the given session. -func (m *sessionDataMutator) SetIntervalStyleEnabled(enabled bool) { - m.data.IntervalStyleEnabled = enabled -} - -// SetDateStyleEnabled sets the DateStyleEnabled for the given session. -func (m *sessionDataMutator) SetDateStyleEnabled(enabled bool) { - m.data.DateStyleEnabled = enabled -} - // SetStubCatalogTablesEnabled sets default value for stub_catalog_tables. func (m *sessionDataMutator) SetStubCatalogTablesEnabled(enabled bool) { m.data.StubCatalogTablesEnabled = enabled diff --git a/pkg/sql/logictest/testdata/logic_test/as_of b/pkg/sql/logictest/testdata/logic_test/as_of index ef020468f763..1718f5ac1161 100644 --- a/pkg/sql/logictest/testdata/logic_test/as_of +++ b/pkg/sql/logictest/testdata/logic_test/as_of @@ -18,7 +18,7 @@ SELECT * FROM t AS OF SYSTEM TIME INTERVAL '-1us' # Verify that we can use computed expressions. query I -SELECT * FROM t AS OF SYSTEM TIME -( ('1000' || 'us')::INTERVAL ) +SELECT * FROM t AS OF SYSTEM TIME -( parse_interval('1000' || 'us') ) ---- 2 diff --git a/pkg/sql/logictest/testdata/logic_test/datestyle_intervalstyle_mixed_version b/pkg/sql/logictest/testdata/logic_test/datestyle_intervalstyle_mixed_version deleted file mode 100644 index 29f1413faa2d..000000000000 --- a/pkg/sql/logictest/testdata/logic_test/datestyle_intervalstyle_mixed_version +++ /dev/null @@ -1,93 +0,0 @@ -# LogicTest: local-mixed-21.2-22.1 - -statement error setting IntervalStyle is not enabled -SET intervalstyle = 'iso_8601' - -statement error setting IntervalStyle is not enabled -SET intervalstyle = 'sql_standard' - -statement ok -SET intervalstyle_enabled = 'on' - -statement ok -SET intervalstyle = 'iso_8601' - -statement ok -SET intervalstyle = 'sql_standard' - -statement ok -RESET intervalstyle - -statement error context-dependent operators are not allowed in computed column\nHINT: INTERVAL to STRING casts depend on IntervalStyle; consider using to_char\(interval\) -CREATE TABLE invalid_table ( - invalid_col string AS ('1 hour'::interval::string) STORED -) - -statement error string::interval: context-dependent operators are not allowed in computed column\nHINT: STRING to INTERVAL casts depend on session IntervalStyle; use parse_interval\(string\) instead -CREATE TABLE invalid_table ( - invalid_col interval AS ('1 hour'::string::interval) STORED -) - -statement error setting DateStyle is not enabled -SET datestyle = 'dmy' - -statement error setting DateStyle is not enabled -SET datestyle = 'ymd' - -statement ok -SET datestyle_enabled = true - -statement ok -SET datestyle = 'dmy' - -statement ok -SET datestyle = 'ymd' - -statement ok -RESET datestyle - -statement error context-dependent operators are not allowed in computed column\nHINT: TIMESTAMP to STRING casts are dependent on DateStyle; consider using to_char\(timestamp\) instead\. -CREATE TABLE invalid_table ( - invalid_col string AS ('2020-05-12 10:12:13'::timestamp::string) STORED -) - -statement error context-dependent operators are not allowed in computed column\nHINT: DATE to STRING casts are dependent on DateStyle; consider using to_char\(date\) instead\. -CREATE TABLE invalid_table ( - invalid_col string AS ('2020-05-12 10:12:13'::date::string) STORED -) - -statement error context-dependent operators are not allowed in computed column\nHINT: STRING to TIMESTAMP casts are context-dependent because of relative timestamp strings like 'now' and session settings such as DateStyle; use parse_timestamp\(string\) instead\. -CREATE TABLE invalid_table ( - invalid_col timestamp AS ('2020-05-12 10:12:13'::string::timestamp) STORED -) - -statement error context-dependent operators are not allowed in computed column\nHINT: STRING to DATE casts depend on session DateStyle; use parse_date\(string\) instead -CREATE TABLE invalid_table ( - invalid_col date AS ('2020-05-12 10:12:13'::string::date) STORED -) - -statement error context-dependent operators are not allowed in computed column\nHINT: STRING to TIME casts depend on session DateStyle; use parse_time\(string\) instead -CREATE TABLE invalid_table ( - invalid_col time AS ('2020-05-12 10:12:13'::string::time) STORED -) - -statement error context-dependent operators are not allowed in computed column\nHINT: STRING to TIMETZ casts depend on session DateStyle; use parse_timetz\(string\) instead -CREATE TABLE invalid_table ( - invalid_col timetz AS ('2020-05-12 10:12:13'::string::timetz) STORED -) - -statement ok -SET datestyle_enabled = false - -statement ok -CREATE TABLE invalid_timestamp_table ( - invalid_col string AS ('2020-05-12 10:12:13'::timestamp::string) STORED -) - -statement ok -SET intervalstyle_enabled = false - -statement ok -CREATE TABLE invalid_interval_table ( - invalid_col interval AS ('1 hour'::string::interval) STORED -) diff --git a/pkg/sql/logictest/testdata/logic_test/datetime b/pkg/sql/logictest/testdata/logic_test/datetime index 89173026e0b3..972c01485673 100644 --- a/pkg/sql/logictest/testdata/logic_test/datetime +++ b/pkg/sql/logictest/testdata/logic_test/datetime @@ -1752,11 +1752,6 @@ true subtest datestyle_order -query T noticetrace -set datestyle_enabled = false ----- -NOTICE: ignoring datestyle_enabled setting; it is always true - # Setting datestyle is always allowed in 22.1. statement ok set datestyle = 'dmy' @@ -1988,11 +1983,6 @@ ORDER BY pk # Regression test for #71776 -- intervalstyle should apply to pg_catalog. -query T noticetrace -SET intervalstyle_enabled = false; ----- -NOTICE: ignoring intervalstyle_enabled setting; it is always true - statement ok SET intervalstyle = iso_8601; CREATE TABLE table_71776 (interval_col interval DEFAULT 'P3Y') diff --git a/pkg/sql/logictest/testdata/logic_test/interval b/pkg/sql/logictest/testdata/logic_test/interval index 0694526061f4..62dc1f6dc569 100644 --- a/pkg/sql/logictest/testdata/logic_test/interval +++ b/pkg/sql/logictest/testdata/logic_test/interval @@ -400,11 +400,6 @@ SELECT i / 2 FROM ( VALUES subtest interval_session -query T noticetrace -SET intervalstyle_enabled = 'off' ----- -NOTICE: ignoring intervalstyle_enabled setting; it is always true - statement ok SET intervalstyle = 'iso_8601' diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index cadc54544f27..f835d97a2296 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -4118,7 +4118,6 @@ client_min_messages notice NULL cost_scans_with_default_col_size off NULL NULL NULL string database test NULL NULL NULL string datestyle ISO, MDY NULL NULL NULL string -datestyle_enabled on NULL NULL NULL string default_int_size 8 NULL NULL NULL string default_table_access_method heap NULL NULL NULL string default_tablespace · NULL NULL NULL string @@ -4159,7 +4158,6 @@ index_recommendations_enabled off NULL inject_retry_errors_enabled off NULL NULL NULL string integer_datetimes on NULL NULL NULL string intervalstyle postgres NULL NULL NULL string -intervalstyle_enabled on NULL NULL NULL string is_superuser on NULL NULL NULL string join_reader_ordering_strategy_batch_size 10 KiB NULL NULL NULL string large_full_scan_rows 1000 NULL NULL NULL string @@ -4241,7 +4239,6 @@ client_min_messages notice NULL cost_scans_with_default_col_size off NULL user NULL off off database test NULL user NULL · test datestyle ISO, MDY NULL user NULL ISO, MDY ISO, MDY -datestyle_enabled on NULL user NULL on on default_int_size 8 NULL user NULL 8 8 default_table_access_method heap NULL user NULL heap heap default_tablespace · NULL user NULL · · @@ -4282,7 +4279,6 @@ index_recommendations_enabled off NULL inject_retry_errors_enabled off NULL user NULL off off integer_datetimes on NULL user NULL on on intervalstyle postgres NULL user NULL postgres postgres -intervalstyle_enabled on NULL user NULL on on is_superuser on NULL user NULL on on join_reader_ordering_strategy_batch_size 10 KiB NULL user NULL 10 KiB 10 KiB large_full_scan_rows 1000 NULL user NULL 1000 1000 @@ -4359,7 +4355,6 @@ cost_scans_with_default_col_size NULL NULL NULL crdb_version NULL NULL NULL NULL NULL database NULL NULL NULL NULL NULL datestyle NULL NULL NULL NULL NULL -datestyle_enabled NULL NULL NULL NULL NULL default_int_size NULL NULL NULL NULL NULL default_table_access_method NULL NULL NULL NULL NULL default_tablespace NULL NULL NULL NULL NULL @@ -4401,7 +4396,6 @@ index_recommendations_enabled NULL NULL NULL inject_retry_errors_enabled NULL NULL NULL NULL NULL integer_datetimes NULL NULL NULL NULL NULL intervalstyle NULL NULL NULL NULL NULL -intervalstyle_enabled NULL NULL NULL NULL NULL is_superuser NULL NULL NULL NULL NULL join_reader_ordering_strategy_batch_size NULL NULL NULL NULL NULL large_full_scan_rows NULL NULL NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/set b/pkg/sql/logictest/testdata/logic_test/set index f466e320758f..4112d2d82048 100644 --- a/pkg/sql/logictest/testdata/logic_test/set +++ b/pkg/sql/logictest/testdata/logic_test/set @@ -171,9 +171,6 @@ SHOW datestyle ---- ISO, MDY -statement ok -SET datestyle_enabled = true - statement ok SET datestyle = 'ymd' @@ -205,9 +202,6 @@ SET datestyle = 'other' statement ok SET intervalstyle = 'postgres' -statement ok -SET intervalstyle_enabled = true - statement ok SET intervalstyle = 'iso_8601' diff --git a/pkg/sql/logictest/testdata/logic_test/set_local b/pkg/sql/logictest/testdata/logic_test/set_local index 3e1489d78c29..324b8d813d6f 100644 --- a/pkg/sql/logictest/testdata/logic_test/set_local +++ b/pkg/sql/logictest/testdata/logic_test/set_local @@ -16,9 +16,6 @@ SELECT 1; SET LOCAL TIME ZONE +6 ---- WARNING: SET LOCAL can only be used in transaction blocks -statement ok -SET intervalstyle_enabled = 'on' - # Set TIME ZONE local to the transaction, but IntervalStyle globally. statement ok BEGIN; diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index 192ba628c933..32345c99c02c 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -37,7 +37,6 @@ client_min_messages notice cost_scans_with_default_col_size off database test datestyle ISO, MDY -datestyle_enabled on default_int_size 8 default_table_access_method heap default_tablespace · @@ -78,7 +77,6 @@ index_recommendations_enabled off inject_retry_errors_enabled off integer_datetimes on intervalstyle postgres -intervalstyle_enabled on is_superuser on join_reader_ordering_strategy_batch_size 10 KiB large_full_scan_rows 1000 diff --git a/pkg/sql/logictest/testdata/logic_test/txn b/pkg/sql/logictest/testdata/logic_test/txn index 4ca24882f9e3..0a9aff71701f 100644 --- a/pkg/sql/logictest/testdata/logic_test/txn +++ b/pkg/sql/logictest/testdata/logic_test/txn @@ -1119,7 +1119,6 @@ SET SESSION CHARACTERISTICS AS TRANSACTION DEFERRABLE # Test retry rewinds correctly. statement ok -SET intervalstyle_enabled = on; SET intervalstyle = 'postgres'; CREATE TABLE rewind_session_test (s string primary key); diff --git a/pkg/sql/opt/memo/logical_props_builder.go b/pkg/sql/opt/memo/logical_props_builder.go index 499bc705c7eb..70a1992c9208 100644 --- a/pkg/sql/opt/memo/logical_props_builder.go +++ b/pkg/sql/opt/memo/logical_props_builder.go @@ -1620,7 +1620,7 @@ func BuildSharedProps(e opt.Expr, shared *props.Shared, evalCtx *eval.Context) { case *CastExpr, *AssignmentCastExpr: from := e.Child(0).(opt.ScalarExpr).DataType() to := e.Private().(*types.T) - volatility, ok := cast.LookupCastVolatility(from, to, evalCtx.CastSessionOptions()) + volatility, ok := cast.LookupCastVolatility(from, to) if !ok { panic(errors.AssertionFailedf("no volatility for cast %s::%s", from, to)) } diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go index 3bf6fdfad883..fa8d1e7d2bd8 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -143,8 +143,6 @@ type Memo struct { safeUpdates bool preferLookupJoinsForFKs bool saveTablesPrefix string - dateStyleEnabled bool - intervalStyleEnabled bool dateStyle pgdate.DateStyle intervalStyle duration.IntervalStyle propagateInputOrdering bool @@ -190,8 +188,6 @@ func (m *Memo) Init(evalCtx *eval.Context) { safeUpdates: evalCtx.SessionData().SafeUpdates, preferLookupJoinsForFKs: evalCtx.SessionData().PreferLookupJoinsForFKs, saveTablesPrefix: evalCtx.SessionData().SaveTablesPrefix, - intervalStyleEnabled: evalCtx.SessionData().IntervalStyleEnabled, - dateStyleEnabled: evalCtx.SessionData().DateStyleEnabled, dateStyle: evalCtx.SessionData().GetDateStyle(), intervalStyle: evalCtx.SessionData().GetIntervalStyle(), propagateInputOrdering: evalCtx.SessionData().PropagateInputOrdering, @@ -315,8 +311,6 @@ func (m *Memo) IsStale( m.safeUpdates != evalCtx.SessionData().SafeUpdates || m.preferLookupJoinsForFKs != evalCtx.SessionData().PreferLookupJoinsForFKs || m.saveTablesPrefix != evalCtx.SessionData().SaveTablesPrefix || - m.intervalStyleEnabled != evalCtx.SessionData().IntervalStyleEnabled || - m.dateStyleEnabled != evalCtx.SessionData().DateStyleEnabled || m.dateStyle != evalCtx.SessionData().GetDateStyle() || m.intervalStyle != evalCtx.SessionData().GetIntervalStyle() || m.propagateInputOrdering != evalCtx.SessionData().PropagateInputOrdering || diff --git a/pkg/sql/opt/memo/memo_test.go b/pkg/sql/opt/memo/memo_test.go index c156b89e69cd..3e3471f4cd75 100644 --- a/pkg/sql/opt/memo/memo_test.go +++ b/pkg/sql/opt/memo/memo_test.go @@ -219,18 +219,6 @@ func TestMemoIsStale(t *testing.T) { evalCtx.SessionData().SafeUpdates = false notStale() - // Stale intervalStyleEnabled. - evalCtx.SessionData().IntervalStyleEnabled = true - stale() - evalCtx.SessionData().IntervalStyleEnabled = false - notStale() - - // Stale dateStyleEnabled. - evalCtx.SessionData().DateStyleEnabled = true - stale() - evalCtx.SessionData().DateStyleEnabled = false - notStale() - // Stale DateStyle. evalCtx.SessionData().DataConversionConfig.DateStyle = pgdate.DateStyle{Order: pgdate.Order_YMD} stale() diff --git a/pkg/sql/opt/norm/fold_constants_funcs.go b/pkg/sql/opt/norm/fold_constants_funcs.go index c2be2b949d07..b678460969c1 100644 --- a/pkg/sql/opt/norm/fold_constants_funcs.go +++ b/pkg/sql/opt/norm/fold_constants_funcs.go @@ -359,9 +359,7 @@ func (c *CustomFuncs) FoldCast(input opt.ScalarExpr, typ *types.T) (_ opt.Scalar return nil, false } - volatility, ok := cast.LookupCastVolatility( - input.DataType(), typ, c.f.evalCtx.CastSessionOptions(), - ) + volatility, ok := cast.LookupCastVolatility(input.DataType(), typ) if !ok || !c.CanFoldOperator(volatility) { return nil, false } @@ -389,9 +387,7 @@ func (c *CustomFuncs) FoldCast(input opt.ScalarExpr, typ *types.T) (_ opt.Scalar func (c *CustomFuncs) FoldAssignmentCast( input opt.ScalarExpr, typ *types.T, ) (_ opt.ScalarExpr, ok bool) { - volatility, ok := cast.LookupCastVolatility( - input.DataType(), typ, c.f.evalCtx.CastSessionOptions(), - ) + volatility, ok := cast.LookupCastVolatility(input.DataType(), typ) if !ok || !c.CanFoldOperator(volatility) { return nil, false } diff --git a/pkg/sql/pgwire/conn_test.go b/pkg/sql/pgwire/conn_test.go index 6269a93aa33c..74a0bc874b7c 100644 --- a/pkg/sql/pgwire/conn_test.go +++ b/pkg/sql/pgwire/conn_test.go @@ -1549,11 +1549,6 @@ func TestSetSessionArguments(t *testing.T) { defer s.Stopper().Stop(ctx) defer db.Close() - _, err := db.Exec("SET CLUSTER SETTING sql.defaults.datestyle.enabled = true") - require.NoError(t, err) - _, err = db.Exec("SET CLUSTER SETTING sql.defaults.intervalstyle.enabled = true") - require.NoError(t, err) - pgURL, cleanupFunc := sqlutils.PGUrl( t, s.ServingSQLAddr(), "testConnClose" /* prefix */, url.User(username.RootUser), ) diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 69f8039f2585..ae67eabaf02a 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -14,7 +14,6 @@ import ( "context" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -32,7 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/querycache" - "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/transform" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -350,17 +348,6 @@ func newInternalPlanner( p.isInternalPlanner = true p.semaCtx = tree.MakeSemaContext() - if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.DateStyleIntervalStyleCastRewrite) { - p.semaCtx.CastSessionOptions = cast.SessionOptions{ - IntervalStyleEnabled: true, - DateStyleEnabled: true, - } - } else { - p.semaCtx.CastSessionOptions = cast.SessionOptions{ - IntervalStyleEnabled: sd.IntervalStyleEnabled, - DateStyleEnabled: sd.DateStyleEnabled, - } - } p.semaCtx.SearchPath = &sd.SearchPath p.semaCtx.TypeResolver = p p.semaCtx.DateStyle = sd.GetDateStyle() diff --git a/pkg/sql/randgen/expr.go b/pkg/sql/randgen/expr.go index 3eaaf6d6866d..e3a0bc82253c 100644 --- a/pkg/sql/randgen/expr.go +++ b/pkg/sql/randgen/expr.go @@ -218,7 +218,7 @@ func randExpr( } default: - vol, ok := cast.LookupCastVolatility(xTyp, types.String, cast.SessionOptions{}) + vol, ok := cast.LookupCastVolatility(xTyp, types.String) if ok && vol <= volatility.Immutable && !typeToStringCastHasIncorrectVolatility(xTyp) { // We can cast to string; use lower(x::string) diff --git a/pkg/sql/schemachanger/scbuild/BUILD.bazel b/pkg/sql/schemachanger/scbuild/BUILD.bazel index ae900c99816a..6c8ada563432 100644 --- a/pkg/sql/schemachanger/scbuild/BUILD.bazel +++ b/pkg/sql/schemachanger/scbuild/BUILD.bazel @@ -13,7 +13,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild", visibility = ["//visibility:public"], deps = [ - "//pkg/clusterversion", "//pkg/keys", "//pkg/kv", "//pkg/security/username", @@ -38,7 +37,6 @@ go_library( "//pkg/sql/schemachanger/scerrors", "//pkg/sql/schemachanger/scpb", "//pkg/sql/schemachanger/screl", - "//pkg/sql/sem/cast", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go index 5181df3741c6..e699f367da6d 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go @@ -167,11 +167,6 @@ func alterTableAddColumn( } if desc.HasDefault() { expression := b.WrapExpression(tbl.TableID, cdd.DefaultExpr) - // Sequence references inside expressions are unsupported, since these will - // hit errors during backfill. - if len(expression.UsesSequenceIDs) > 0 { - panic(scerrors.NotImplementedErrorf(t, "sequence default expression %s", cdd.DefaultExpr)) - } spec.def = &scpb.ColumnDefaultExpression{ TableID: tbl.TableID, ColumnID: spec.col.ColumnID, diff --git a/pkg/sql/schemachanger/scbuild/tree_context_builder.go b/pkg/sql/schemachanger/scbuild/tree_context_builder.go index 2d80a5b3a7ea..592de89ecabf 100644 --- a/pkg/sql/schemachanger/scbuild/tree_context_builder.go +++ b/pkg/sql/schemachanger/scbuild/tree_context_builder.go @@ -13,10 +13,8 @@ package scbuild import ( "context" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/sql/faketreeeval" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild/internal/scbuildstmt" - "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -33,17 +31,6 @@ func newSemaCtx(d Dependencies) *tree.SemaContext { semaCtx := tree.MakeSemaContext() semaCtx.Annotations = nil semaCtx.SearchPath = &d.SessionData().SearchPath - if d.ClusterSettings().Version.IsActive(context.Background(), clusterversion.IncrementalBackupSubdir) { - semaCtx.CastSessionOptions = cast.SessionOptions{ - IntervalStyleEnabled: true, - DateStyleEnabled: true, - } - } else { - semaCtx.CastSessionOptions = cast.SessionOptions{ - IntervalStyleEnabled: d.SessionData().IntervalStyleEnabled, - DateStyleEnabled: d.SessionData().DateStyleEnabled, - } - } semaCtx.TypeResolver = d.CatalogReader() semaCtx.TableNameResolver = d.CatalogReader() semaCtx.DateStyle = d.SessionData().GetDateStyle() diff --git a/pkg/sql/schemachanger/scdeps/exec_deps.go b/pkg/sql/schemachanger/scdeps/exec_deps.go index 3935f37df2f0..c0ac579f90b5 100644 --- a/pkg/sql/schemachanger/scdeps/exec_deps.go +++ b/pkg/sql/schemachanger/scdeps/exec_deps.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -30,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -455,13 +457,23 @@ func (d *execDeps) getTablesForStatsRefresh() []descpb.ID { return d.tableStatsToRefresh } -// StatsRefreshQueue implements scexec.Dependencies +// StatsRefresher implements scexec.Dependencies func (d *execDeps) StatsRefresher() scexec.StatsRefreshQueue { return d } -// NewNoOpBackfillerTracker constructs a backfiller tracker which does not do -// anything. It will always return backfillProgress for a given backfill which +// Telemetry implements the scexec.Dependencies interface. +func (d *execDeps) Telemetry() scexec.Telemetry { + return d +} + +// IncrementSchemaChangeErrorType implemented the scexec.Telemetry interface. +func (d *execDeps) IncrementSchemaChangeErrorType(typ string) { + telemetry.Inc(sqltelemetry.SchemaChangeErrorCounter(typ)) +} + +// NewNoOpBackfillerTracker constructs a backfill tracker which does not do +// anything. It will always return progress for a given backfill which // contains a full set of CompletedSpans corresponding to the source index // span and an empty MinimumWriteTimestamp. Similarly for merges. func NewNoOpBackfillerTracker(codec keys.SQLCodec) scexec.BackfillerTracker { diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index 6e50f1e2ef71..dfa33d3dc93c 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -1076,6 +1076,16 @@ func (s *TestState) StatsRefresher() scexec.StatsRefreshQueue { return s } +// Telemetry implement scexec.Dependencies. +func (s *TestState) Telemetry() scexec.Telemetry { + return s +} + +// IncrementSchemaChangeErrorType implements scexec.Telemetry +func (s *TestState) IncrementSchemaChangeErrorType(typ string) { + s.LogSideEffectf("incrementing schema change error type metric %s", typ) +} + // GetTestingKnobs implement scexec.Dependencies. func (s *TestState) GetTestingKnobs() *scexec.TestingKnobs { return &scexec.TestingKnobs{} diff --git a/pkg/sql/schemachanger/scexec/dependencies.go b/pkg/sql/schemachanger/scexec/dependencies.go index 5981b7341650..99ec651bf3d4 100644 --- a/pkg/sql/schemachanger/scexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/dependencies.go @@ -42,6 +42,7 @@ type Dependencies interface { DescriptorMetadataUpdater(ctx context.Context) DescriptorMetadataUpdater StatsRefresher() StatsRefreshQueue GetTestingKnobs() *TestingKnobs + Telemetry() Telemetry // Statements returns the statements behind this schema change. Statements() []string @@ -83,6 +84,13 @@ type EventLogger interface { ) error } +// Telemetry encapsulates metrics gather for the declarative schema changer. +type Telemetry interface { + // IncrementSchemaChangeErrorType increments the number of errors of a given + // type observed by the schema changer. + IncrementSchemaChangeErrorType(typ string) +} + // CatalogChangeBatcher encapsulates batched updates to the catalog: descriptor // updates, namespace operations, etc. type CatalogChangeBatcher interface { diff --git a/pkg/sql/schemachanger/scexec/exec_backfill.go b/pkg/sql/schemachanger/scexec/exec_backfill.go index b9b8ae4c5d43..8c9a56764680 100644 --- a/pkg/sql/schemachanger/scexec/exec_backfill.go +++ b/pkg/sql/schemachanger/scexec/exec_backfill.go @@ -329,6 +329,8 @@ func runBackfiller( return im.MergeIndexes(ctx, *p, tracker, tables[p.TableID]) } if err := forEachProgressConcurrently(ctx, op, backfillProgresses, mergeProgresses, bf, mf); err != nil { + // We ran into an uncategorized schema change error. + deps.Telemetry().IncrementSchemaChangeErrorType("uncategorized") return scerrors.SchemaChangerUserError(err) } if err := stop(); err != nil { diff --git a/pkg/sql/schemachanger/scexec/mocks_generated_test.go b/pkg/sql/schemachanger/scexec/mocks_generated_test.go index d83f68808abd..92f550f47b26 100644 --- a/pkg/sql/schemachanger/scexec/mocks_generated_test.go +++ b/pkg/sql/schemachanger/scexec/mocks_generated_test.go @@ -332,6 +332,20 @@ func (mr *MockDependenciesMockRecorder) StatsRefresher() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StatsRefresher", reflect.TypeOf((*MockDependencies)(nil).StatsRefresher)) } +// Telemetry mocks base method. +func (m *MockDependencies) Telemetry() scexec.Telemetry { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Telemetry") + ret0, _ := ret[0].(scexec.Telemetry) + return ret0 +} + +// Telemetry indicates an expected call of Telemetry. +func (mr *MockDependenciesMockRecorder) Telemetry() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Telemetry", reflect.TypeOf((*MockDependencies)(nil).Telemetry)) +} + // TransactionalJobRegistry mocks base method. func (m *MockDependencies) TransactionalJobRegistry() scexec.TransactionalJobRegistry { m.ctrl.T.Helper() diff --git a/pkg/sql/schemachanger/screl/scalars.go b/pkg/sql/schemachanger/screl/scalars.go index bef7637e3516..4d7dc7c01298 100644 --- a/pkg/sql/schemachanger/screl/scalars.go +++ b/pkg/sql/schemachanger/screl/scalars.go @@ -63,6 +63,8 @@ func AllDescIDs(e scpb.Element) (ids catalog.DescriptorIDSet) { if e == nil { return ids } + // For certain elements the references needed will not be attributes, so manually + // include these. _ = WalkDescIDs(e, func(id *catid.DescID) error { ids.Add(*id) return nil diff --git a/pkg/sql/schemachanger/testdata/alter_table_add_column b/pkg/sql/schemachanger/testdata/alter_table_add_column index 87636a5f5003..9fcd33aff2f5 100644 --- a/pkg/sql/schemachanger/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/testdata/alter_table_add_column @@ -1,11 +1,14 @@ setup CREATE DATABASE db; -CREATE TABLE db.public.tbl (i INT PRIMARY KEY) +CREATE TABLE db.public.tbl (i INT PRIMARY KEY); +CREATE SEQUENCE db.public.sq1; ---- ... +database {0 0 db} -> 104 +schema {104 0 public} -> 105 +object {104 105 tbl} -> 106 ++object {104 105 sq1} -> 107 + test ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAULT 42 @@ -763,3 +766,1671 @@ update progress of schema change job #1: "all stages completed" commit transaction #11 notified job registry to adopt jobs: [2] # end PostCommitPhase + +test +ALTER TABLE db.public.tbl ADD COLUMN k INT NOT NULL DEFAULT 42 +---- +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +increment telemetry for sql.schema.alter_table.add_column +increment telemetry for sql.schema.qualifcation.default_expr +increment telemetry for sql.schema.new_column_type.int8 +begin transaction #1 +# begin StatementPhase +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 9 MutationType ops +upsert descriptor #106 + ... + createAsOfTime: + wallTime: "1" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - PUBLIC + + - PUBLIC + + - DELETE_ONLY + + - PUBLIC + + - PUBLIC + + - PUBLIC + + - BACKFILL_ONLY + + - ABSENT + + - DELETE_ONLY + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹k› INT8 NOT + + NULL DEFAULT ‹42› + + statement: ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42 + + statementTag: ALTER TABLE + + revertible: true + + targetRanks: + + - 0 + + - 1 + + - 2 + + - 3 + + - 4 + + - 5 + + - 6 + + - 7 + + - 8 + + targets: + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 2 + + indexId: 2 + + isCreatedExplicitly: true + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + storingColumnIds: + + - 2 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexName: + + indexId: 2 + + name: tbl_pkey + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 3 + + pgAttributeNum: 3 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnName: + + columnId: 3 + + name: k + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnType: + + columnId: 3 + + embeddedTypeT: + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnDefaultExpression: + + columnId: 3 + + embeddedExpr: + + expr: 42:::INT8 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 2 + + indexId: 4 + + isCreatedExplicitly: true + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + sourceIndexId: 2 + + storingColumnIds: + + - 2 + + - 3 + + tableId: 106 + + temporaryIndexId: 5 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + indexName: + + indexId: 4 + + name: tbl_pkey + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + temporaryIndex: + + embeddedIndex: + + constraintId: 2 + + indexId: 5 + + isCreatedExplicitly: true + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + sourceIndexId: 2 + + storingColumnIds: + + - 2 + + - 3 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: TRANSIENT_ABSENT + families: + - columnIds: + - 1 + - 2 + + - 3 + columnNames: + - i + - j + + - k + defaultColumnId: 2 + name: primary + ... + id: 106 + modificationTime: {} + + mutations: + + - column: + + defaultExpr: 42:::INT8 + + id: 3 + + name: k + + pgAttributeNum: 3 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: ADD + + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 4 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 4 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_4_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + - 3 + + storeColumnNames: + + - j + + - k + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 5 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 5 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_5_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + - 3 + + storeColumnNames: + + - j + + - k + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: tbl + - nextColumnId: 3 + - nextConstraintId: 4 + + nextColumnId: 4 + + nextConstraintId: 6 + nextFamilyId: 1 + - nextIndexId: 4 + + nextIndexId: 6 + nextMutationId: 1 + parentId: 104 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "8" + + version: "9" +write *eventpb.AlterTable to event log for descriptor #106: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹k› INT8 NOT NULL DEFAULT ‹42› +create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42" + descriptor IDs: [106] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitPhase stage 1 of 7 with 4 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - DELETE_ONLY + + - WRITE_ONLY + - PUBLIC + - PUBLIC + ... + - BACKFILL_ONLY + - ABSENT + - - DELETE_ONLY + + - WRITE_ONLY + jobId: "1" + relevantStatements: + ... + direction: ADD + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + name: tbl + nextColumnId: 4 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "9" + + version: "10" +update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" +commit transaction #3 +begin transaction #4 +## PostCommitPhase stage 2 of 7 with 1 BackfillType op +backfill indexes [4] from index #2 in table #106 +commit transaction #4 +begin transaction #5 +## PostCommitPhase stage 3 of 7 with 3 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - BACKFILL_ONLY + + - DELETE_ONLY + - ABSENT + - WRITE_ONLY + ... + version: 4 + mutationId: 1 + - state: BACKFILLING + + state: DELETE_ONLY + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "10" + + version: "11" +update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" +commit transaction #5 +begin transaction #6 +## PostCommitPhase stage 4 of 7 with 3 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - DELETE_ONLY + + - MERGE_ONLY + - ABSENT + - WRITE_ONLY + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "11" + + version: "12" +update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" +commit transaction #6 +begin transaction #7 +## PostCommitPhase stage 5 of 7 with 1 BackfillType op +merge temporary indexes [5] into backfilled indexes [4] in table #106 +commit transaction #7 +begin transaction #8 +## PostCommitPhase stage 6 of 7 with 1 ValidationType op +validate forward indexes [4] in table #106 +commit transaction #8 +begin transaction #9 +## PostCommitPhase stage 7 of 7 with 8 MutationType ops +upsert descriptor #106 + ... + oid: 20 + width: 64 + + - defaultExpr: 42:::INT8 + + id: 3 + + name: k + + pgAttributeNum: 3 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + createAsOfTime: + wallTime: "1" + ... + userName: root + currentStatuses: + + - VALIDATED + + - ABSENT + - PUBLIC + - PUBLIC + - - WRITE_ONLY + - PUBLIC + - PUBLIC + - PUBLIC + - - MERGE_ONLY + - - ABSENT + + - PUBLIC + - WRITE_ONLY + jobId: "1" + ... + statement: ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42 + statementTag: ALTER TABLE + - revertible: true + targetRanks: + - 0 + ... + modificationTime: {} + mutations: + - - column: + - defaultExpr: 42:::INT8 + - id: 3 + - name: k + - pgAttributeNum: 3 + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - direction: ADD + - mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + - constraintId: 4 + + constraintId: 5 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 4 + + id: 5 + interleave: {} + keyColumnDirections: + ... + keyColumnNames: + - i + - name: crdb_internal_index_4_name_placeholder + + name: crdb_internal_index_5_name_placeholder + partitioning: {} + sharded: {} + ... + - k + unique: true + + useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + state: DELETE_AND_WRITE_ONLY + - - direction: ADD + + - direction: DROP + index: + - constraintId: 5 + + constraintId: 2 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 5 + + id: 2 + interleave: {} + keyColumnDirections: + ... + keyColumnNames: + - i + - name: crdb_internal_index_5_name_placeholder + + name: tbl_pkey + partitioning: {} + sharded: {} + storeColumnIds: + - 2 + - - 3 + storeColumnNames: + - j + - - k + unique: true + - useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + ... + parentId: 104 + primaryIndex: + - constraintId: 2 + + constraintId: 4 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 2 + + id: 4 + interleave: {} + keyColumnDirections: + ... + storeColumnIds: + - 2 + + - 3 + storeColumnNames: + - j + + - k + unique: true + version: 4 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "12" + + version: "13" +adding table for stats refresh: 106 +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 1 of 2 with 2 MutationType ops pending" +set schema change job #1 to non-cancellable +commit transaction #9 +begin transaction #10 +## PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops +upsert descriptor #106 + ... + userName: root + currentStatuses: + - - VALIDATED + + - DELETE_ONLY + - ABSENT + - PUBLIC + ... + - PUBLIC + - PUBLIC + - - WRITE_ONLY + + - TRANSIENT_DELETE_ONLY + jobId: "1" + relevantStatements: + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: DELETE_ONLY + - direction: DROP + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: DELETE_ONLY + name: tbl + nextColumnId: 4 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "13" + + version: "14" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending" +commit transaction #10 +begin transaction #11 +## PostCommitNonRevertiblePhase stage 2 of 2 with 6 MutationType ops +upsert descriptor #106 + ... + createAsOfTime: + wallTime: "1" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - - DELETE_ONLY + - - ABSENT + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - TRANSIENT_DELETE_ONLY + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹k› INT8 NOT + - NULL DEFAULT ‹42› + - statement: ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42 + - statementTag: ALTER TABLE + - targetRanks: + - - 0 + - - 1 + - - 2 + - - 3 + - - 4 + - - 5 + - - 6 + - - 7 + - - 8 + - targets: + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 2 + - indexId: 2 + - isCreatedExplicitly: true + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - storingColumnIds: + - - 2 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexName: + - indexId: 2 + - name: tbl_pkey + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 3 + - pgAttributeNum: 3 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnName: + - columnId: 3 + - name: k + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnType: + - columnId: 3 + - embeddedTypeT: + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnDefaultExpression: + - columnId: 3 + - embeddedExpr: + - expr: 42:::INT8 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 2 + - indexId: 4 + - isCreatedExplicitly: true + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - sourceIndexId: 2 + - storingColumnIds: + - - 2 + - - 3 + - tableId: 106 + - temporaryIndexId: 5 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - indexName: + - indexId: 4 + - name: tbl_pkey + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - temporaryIndex: + - embeddedIndex: + - constraintId: 2 + - indexId: 5 + - isCreatedExplicitly: true + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - sourceIndexId: 2 + - storingColumnIds: + - - 2 + - - 3 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: TRANSIENT_ABSENT + families: + - columnIds: + ... + id: 106 + modificationTime: {} + - mutations: + - - direction: ADD + - index: + - constraintId: 5 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 5 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: crdb_internal_index_5_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 2 + - - 3 + - storeColumnNames: + - - j + - - k + - unique: true + - useDeletePreservingEncoding: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + - - direction: DROP + - index: + - constraintId: 2 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 2 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: tbl_pkey + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 2 + - storeColumnNames: + - - j + - unique: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + + mutations: [] + name: tbl + nextColumnId: 4 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "14" + + version: "15" +write *eventpb.FinishSchemaChange to event log for descriptor 106 +create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42" + descriptor IDs: [106] +update progress of schema change job #1: "all stages completed" +commit transaction #11 +notified job registry to adopt jobs: [2] +# end PostCommitPhase + + +test +ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAULT nextval('db.public.sq1') +---- +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +increment telemetry for sql.schema.alter_table.add_column +increment telemetry for sql.schema.qualifcation.default_expr +increment telemetry for sql.schema.new_column_type.int8 +begin transaction #1 +# begin StatementPhase +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 11 MutationType ops +upsert descriptor #106 + ... + createAsOfTime: + wallTime: "1" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - PUBLIC + + - PUBLIC + + - DELETE_ONLY + + - PUBLIC + + - PUBLIC + + - PUBLIC + + - BACKFILL_ONLY + + - ABSENT + + - DELETE_ONLY + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT + + NULL DEFAULT nextval(‹'db.public.sq1'›) + + statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') + + statementTag: ALTER TABLE + + revertible: true + + targetRanks: + + - 0 + + - 1 + + - 2 + + - 3 + + - 4 + + - 5 + + - 6 + + - 7 + + - 8 + + targets: + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 4 + + indexId: 4 + + isCreatedExplicitly: true + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + storingColumnIds: + + - 2 + + - 3 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexName: + + indexId: 4 + + name: tbl_pkey + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 4 + + pgAttributeNum: 4 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnName: + + columnId: 4 + + name: l + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnType: + + columnId: 4 + + embeddedTypeT: + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnDefaultExpression: + + columnId: 4 + + embeddedExpr: + + expr: nextval(107:::REGCLASS) + + usesSequenceIds: + + - 107 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 4 + + indexId: 6 + + isCreatedExplicitly: true + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + sourceIndexId: 4 + + storingColumnIds: + + - 2 + + - 3 + + - 4 + + tableId: 106 + + temporaryIndexId: 7 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + indexName: + + indexId: 6 + + name: tbl_pkey + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + temporaryIndex: + + embeddedIndex: + + constraintId: 4 + + indexId: 7 + + isCreatedExplicitly: true + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + sourceIndexId: 4 + + storingColumnIds: + + - 2 + + - 3 + + - 4 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: TRANSIENT_ABSENT + families: + - columnIds: + ... + - 2 + - 3 + + - 4 + columnNames: + - i + - j + - k + + - l + defaultColumnId: 2 + name: primary + ... + id: 106 + modificationTime: {} + + mutations: + + - column: + + defaultExpr: nextval(107:::REGCLASS) + + id: 4 + + name: l + + pgAttributeNum: 4 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + usesSequenceIds: + + - 107 + + direction: ADD + + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 6 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 6 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_6_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + - 3 + + - 4 + + storeColumnNames: + + - j + + - k + + - l + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 7 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 7 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_7_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + - 3 + + - 4 + + storeColumnNames: + + - j + + - k + + - l + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: tbl + - nextColumnId: 4 + - nextConstraintId: 6 + + nextColumnId: 5 + + nextConstraintId: 8 + nextFamilyId: 1 + - nextIndexId: 6 + + nextIndexId: 8 + nextMutationId: 1 + parentId: 104 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "15" + + version: "16" +upsert descriptor #107 + ... + createAsOfTime: + wallTime: "1" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + jobId: "1" + + revertible: true + + dependedOnBy: + + - byId: true + + columnIds: + + - 4 + + id: 106 + families: + - columnIds: + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "1" + + version: "2" +write *eventpb.AlterTable to event log for descriptor #106: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT NULL DEFAULT nextval(‹'db.public.sq1'›) +create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1')" + descriptor IDs: [106 107] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitPhase stage 1 of 7 with 5 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - DELETE_ONLY + + - WRITE_ONLY + - PUBLIC + - PUBLIC + ... + - BACKFILL_ONLY + - ABSENT + - - DELETE_ONLY + + - WRITE_ONLY + jobId: "1" + relevantStatements: + ... + direction: ADD + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + name: tbl + nextColumnId: 5 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "16" + + version: "17" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "2" + + version: "3" +update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" +commit transaction #3 +begin transaction #4 +## PostCommitPhase stage 2 of 7 with 1 BackfillType op +backfill indexes [6] from index #4 in table #106 +commit transaction #4 +begin transaction #5 +## PostCommitPhase stage 3 of 7 with 4 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - BACKFILL_ONLY + + - DELETE_ONLY + - ABSENT + - WRITE_ONLY + ... + version: 4 + mutationId: 1 + - state: BACKFILLING + + state: DELETE_ONLY + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "17" + + version: "18" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "3" + + version: "4" +update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" +commit transaction #5 +begin transaction #6 +## PostCommitPhase stage 4 of 7 with 4 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - DELETE_ONLY + + - MERGE_ONLY + - ABSENT + - WRITE_ONLY + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "18" + + version: "19" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "4" + + version: "5" +update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" +commit transaction #6 +begin transaction #7 +## PostCommitPhase stage 5 of 7 with 1 BackfillType op +merge temporary indexes [7] into backfilled indexes [6] in table #106 +commit transaction #7 +begin transaction #8 +## PostCommitPhase stage 6 of 7 with 1 ValidationType op +validate forward indexes [6] in table #106 +commit transaction #8 +begin transaction #9 +## PostCommitPhase stage 7 of 7 with 9 MutationType ops +upsert descriptor #106 + ... + oid: 20 + width: 64 + + - defaultExpr: nextval(107:::REGCLASS) + + id: 4 + + name: l + + pgAttributeNum: 4 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + usesSequenceIds: + + - 107 + createAsOfTime: + wallTime: "1" + ... + userName: root + currentStatuses: + + - VALIDATED + + - ABSENT + - PUBLIC + - PUBLIC + - - WRITE_ONLY + - PUBLIC + - PUBLIC + - PUBLIC + - - MERGE_ONLY + - - ABSENT + + - PUBLIC + - WRITE_ONLY + jobId: "1" + ... + statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') + statementTag: ALTER TABLE + - revertible: true + targetRanks: + - 0 + ... + modificationTime: {} + mutations: + - - column: + - defaultExpr: nextval(107:::REGCLASS) + - id: 4 + - name: l + - pgAttributeNum: 4 + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - usesSequenceIds: + - - 107 + - direction: ADD + - mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + - constraintId: 6 + + constraintId: 7 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 6 + + id: 7 + interleave: {} + keyColumnDirections: + ... + keyColumnNames: + - i + - name: crdb_internal_index_6_name_placeholder + + name: crdb_internal_index_7_name_placeholder + partitioning: {} + sharded: {} + ... + - l + unique: true + + useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + state: DELETE_AND_WRITE_ONLY + - - direction: ADD + + - direction: DROP + index: + - constraintId: 7 + + constraintId: 4 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 7 + + id: 4 + interleave: {} + keyColumnDirections: + ... + keyColumnNames: + - i + - name: crdb_internal_index_7_name_placeholder + + name: tbl_pkey + partitioning: {} + sharded: {} + ... + - 2 + - 3 + - - 4 + storeColumnNames: + - j + - k + - - l + unique: true + - useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + ... + parentId: 104 + primaryIndex: + - constraintId: 4 + + constraintId: 6 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 4 + + id: 6 + interleave: {} + keyColumnDirections: + ... + - 2 + - 3 + + - 4 + storeColumnNames: + - j + - k + + - l + unique: true + version: 4 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "19" + + version: "20" +upsert descriptor #107 + ... + userName: root + jobId: "1" + - revertible: true + dependedOnBy: + - byId: true + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "5" + + version: "6" +adding table for stats refresh: 106 +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 1 of 2 with 2 MutationType ops pending" +set schema change job #1 to non-cancellable +commit transaction #9 +begin transaction #10 +## PostCommitNonRevertiblePhase stage 1 of 2 with 5 MutationType ops +upsert descriptor #106 + ... + userName: root + currentStatuses: + - - VALIDATED + + - DELETE_ONLY + - ABSENT + - PUBLIC + ... + - PUBLIC + - PUBLIC + - - WRITE_ONLY + + - TRANSIENT_DELETE_ONLY + jobId: "1" + relevantStatements: + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: DELETE_ONLY + - direction: DROP + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: DELETE_ONLY + name: tbl + nextColumnId: 5 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "20" + + version: "21" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "6" + + version: "7" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending" +commit transaction #10 +begin transaction #11 +## PostCommitNonRevertiblePhase stage 2 of 2 with 7 MutationType ops +upsert descriptor #106 + ... + createAsOfTime: + wallTime: "1" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - - DELETE_ONLY + - - ABSENT + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - TRANSIENT_DELETE_ONLY + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT + - NULL DEFAULT nextval(‹'db.public.sq1'›) + - statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') + - statementTag: ALTER TABLE + - targetRanks: + - - 0 + - - 1 + - - 2 + - - 3 + - - 4 + - - 5 + - - 6 + - - 7 + - - 8 + - targets: + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 4 + - indexId: 4 + - isCreatedExplicitly: true + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - storingColumnIds: + - - 2 + - - 3 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexName: + - indexId: 4 + - name: tbl_pkey + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 4 + - pgAttributeNum: 4 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnName: + - columnId: 4 + - name: l + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnType: + - columnId: 4 + - embeddedTypeT: + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnDefaultExpression: + - columnId: 4 + - embeddedExpr: + - expr: nextval(107:::REGCLASS) + - usesSequenceIds: + - - 107 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 4 + - indexId: 6 + - isCreatedExplicitly: true + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - sourceIndexId: 4 + - storingColumnIds: + - - 2 + - - 3 + - - 4 + - tableId: 106 + - temporaryIndexId: 7 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - indexName: + - indexId: 6 + - name: tbl_pkey + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - temporaryIndex: + - embeddedIndex: + - constraintId: 4 + - indexId: 7 + - isCreatedExplicitly: true + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - sourceIndexId: 4 + - storingColumnIds: + - - 2 + - - 3 + - - 4 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: TRANSIENT_ABSENT + families: + - columnIds: + ... + id: 106 + modificationTime: {} + - mutations: + - - direction: ADD + - index: + - constraintId: 7 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 7 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: crdb_internal_index_7_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 2 + - - 3 + - - 4 + - storeColumnNames: + - - j + - - k + - - l + - unique: true + - useDeletePreservingEncoding: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + - - direction: DROP + - index: + - constraintId: 4 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 4 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: tbl_pkey + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 2 + - - 3 + - storeColumnNames: + - - j + - - k + - unique: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + + mutations: [] + name: tbl + nextColumnId: 5 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "21" + + version: "22" +upsert descriptor #107 + ... + createAsOfTime: + wallTime: "1" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - jobId: "1" + dependedOnBy: + - byId: true + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "7" + + version: "8" +write *eventpb.FinishSchemaChange to event log for descriptor 106 +create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1')" + descriptor IDs: [106] +update progress of schema change job #1: "all stages completed" +commit transaction #11 +notified job registry to adopt jobs: [2] +# end PostCommitPhase diff --git a/pkg/sql/sem/cast/cast.go b/pkg/sql/sem/cast/cast.go index d5c7553e3848..325d24e7833e 100644 --- a/pkg/sql/sem/cast/cast.go +++ b/pkg/sql/sem/cast/cast.go @@ -20,12 +20,6 @@ import ( "github.com/lib/pq/oid" ) -// SessionOptions configures options for how cast volatility is defined. -type SessionOptions struct { - IntervalStyleEnabled bool - DateStyleEnabled bool -} - // Context represents the contexts in which a cast can be performed. There // are three types of cast contexts: explicit, assignment, and implicit. Not all // casts can be performed in all contexts. See the description of each context @@ -136,13 +130,6 @@ type Cast struct { // set, it is used as an error hint suggesting a possible workaround when // stable casts are not allowed. VolatilityHint string - // intervalStyleAffected is true if the cast is a stable cast when - // SemaContext.IntervalStyleEnabled is true, and an immutable cast - // otherwise. - intervalStyleAffected bool - // dateStyleAffected is true if the cast is a stable cast when - // SemaContext.DateStyleEnabled is true, and an immutable cast otherwise. - dateStyleAffected bool } // ForEachCast calls fn for every valid cast from a source type to a target @@ -194,7 +181,7 @@ func ValidCast(src, tgt *types.T, ctx Context) bool { // If src and tgt are not both array or tuple types, check castMap for a // valid cast. - c, ok := LookupCast(src, tgt, SessionOptions{}) + c, ok := LookupCast(src, tgt) if ok { return c.MaxContext >= ctx } @@ -204,7 +191,7 @@ func ValidCast(src, tgt *types.T, ctx Context) bool { // LookupCast returns a cast that describes the cast from src to tgt if it // exists. If it does not exist, ok=false is returned. -func LookupCast(src, tgt *types.T, so SessionOptions) (Cast, bool) { +func LookupCast(src, tgt *types.T) (Cast, bool) { srcFamily := src.Family() tgtFamily := tgt.Family() srcFamily.Name() @@ -301,10 +288,6 @@ func LookupCast(src, tgt *types.T, so SessionOptions) (Cast, bool) { if tgts, ok := castMap[src.Oid()]; ok { if c, ok := tgts[tgt.Oid()]; ok { - if so.IntervalStyleEnabled && c.intervalStyleAffected || - so.DateStyleEnabled && c.dateStyleAffected { - c.Volatility = volatility.Stable - } return c, true } } @@ -323,12 +306,12 @@ func LookupCast(src, tgt *types.T, so SessionOptions) (Cast, bool) { } // LookupCastVolatility returns the Volatility of a valid cast. -func LookupCastVolatility(from, to *types.T, opts SessionOptions) (_ volatility.V, ok bool) { +func LookupCastVolatility(from, to *types.T) (_ volatility.V, ok bool) { fromFamily := from.Family() toFamily := to.Family() // Special case for casting between arrays. if fromFamily == types.ArrayFamily && toFamily == types.ArrayFamily { - return LookupCastVolatility(from.ArrayContents(), to.ArrayContents(), opts) + return LookupCastVolatility(from.ArrayContents(), to.ArrayContents()) } // Special case for casting between tuples. if fromFamily == types.TupleFamily && toFamily == types.TupleFamily { @@ -343,7 +326,7 @@ func LookupCastVolatility(from, to *types.T, opts SessionOptions) (_ volatility. } maxVolatility := volatility.LeakProof for i := range fromTypes { - v, lookupOk := LookupCastVolatility(fromTypes[i], toTypes[i], opts) + v, lookupOk := LookupCastVolatility(fromTypes[i], toTypes[i]) if !lookupOk { return 0, false } @@ -354,7 +337,7 @@ func LookupCastVolatility(from, to *types.T, opts SessionOptions) (_ volatility. return maxVolatility, true } - cast, ok := LookupCast(from, to, opts) + cast, ok := LookupCast(from, to) if !ok { return 0, false } diff --git a/pkg/sql/sem/cast/cast_map.go b/pkg/sql/sem/cast/cast_map.go index 5a5d58a5680a..b1fadf0b9e34 100644 --- a/pkg/sql/sem/cast/cast_map.go +++ b/pkg/sql/sem/cast/cast_map.go @@ -77,11 +77,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oidext.T_box2d: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_bytea: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_date: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "CHAR to DATE casts depend on session DateStyle; use parse_date(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "CHAR to DATE casts depend on session DateStyle; use parse_date(string) instead", }, oid.T_float4: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_float8: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -92,12 +91,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_int4: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_int8: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_interval: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, - VolatilityHint: "CHAR to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead", - intervalStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "CHAR to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead", }, oid.T_jsonb: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_numeric: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -110,11 +107,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_regrole: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Stable}, oid.T_regtype: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Stable}, oid.T_time: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "CHAR to TIME casts depend on session DateStyle; use parse_time(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "CHAR to TIME casts depend on session DateStyle; use parse_time(string) instead", }, oid.T_timestamp: { MaxContext: ContextExplicit, @@ -122,7 +118,6 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ Volatility: volatility.Stable, VolatilityHint: "CHAR to TIMESTAMP casts are context-dependent because of relative timestamp strings " + "like 'now' and session settings such as DateStyle; use parse_timestamp(string) instead.", - dateStyleAffected: true, }, oid.T_timestamptz: { MaxContext: ContextExplicit, @@ -130,11 +125,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ Volatility: volatility.Stable, }, oid.T_timetz: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "CHAR to TIMETZ casts depend on session DateStyle; use parse_timetz(char) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "CHAR to TIMETZ casts depend on session DateStyle; use parse_timetz(char) instead", }, oid.T_uuid: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_varbit: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -166,11 +160,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oidext.T_box2d: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_bytea: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_date: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: `"char" to DATE casts depend on session DateStyle; use parse_date(string) instead`, - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: `"char" to DATE casts depend on session DateStyle; use parse_date(string) instead`, }, oid.T_float4: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_float8: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -180,12 +173,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_int2: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_int8: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_interval: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, - VolatilityHint: `"char" to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead`, - intervalStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: `"char" to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead`, }, oid.T_jsonb: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_numeric: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -198,11 +189,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_regrole: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Stable}, oid.T_regtype: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Stable}, oid.T_time: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: `"char" to TIME casts depend on session DateStyle; use parse_time(string) instead`, - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: `"char" to TIME casts depend on session DateStyle; use parse_time(string) instead`, }, oid.T_timestamp: { MaxContext: ContextExplicit, @@ -210,7 +200,6 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ Volatility: volatility.Stable, VolatilityHint: `"char" to TIMESTAMP casts are context-dependent because of relative timestamp strings ` + "like 'now' and session settings such as DateStyle; use parse_timestamp(string) instead.", - dateStyleAffected: true, }, oid.T_timestamptz: { MaxContext: ContextExplicit, @@ -218,11 +207,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ Volatility: volatility.Stable, }, oid.T_timetz: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: `"char" to TIMETZ casts depend on session DateStyle; use parse_timetz(string) instead`, - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: `"char" to TIMETZ casts depend on session DateStyle; use parse_timetz(string) instead`, }, oid.T_uuid: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_varbit: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -241,47 +229,37 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_bpchar: { MaxContext: ContextAssignment, origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, + Volatility: volatility.Stable, VolatilityHint: "DATE to CHAR casts are dependent on DateStyle; consider " + "using to_char(date) instead.", - dateStyleAffected: true, }, oid.T_char: { MaxContext: ContextAssignment, origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, + Volatility: volatility.Stable, VolatilityHint: `DATE to "char" casts are dependent on DateStyle; consider ` + "using to_char(date) instead.", - dateStyleAffected: true, }, oid.T_name: { MaxContext: ContextAssignment, origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, + Volatility: volatility.Stable, VolatilityHint: "DATE to NAME casts are dependent on DateStyle; consider " + "using to_char(date) instead.", - dateStyleAffected: true, }, oid.T_text: { MaxContext: ContextAssignment, origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, + Volatility: volatility.Stable, VolatilityHint: "DATE to STRING casts are dependent on DateStyle; consider " + "using to_char(date) instead.", - dateStyleAffected: true, }, oid.T_varchar: { MaxContext: ContextAssignment, origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, + Volatility: volatility.Stable, VolatilityHint: "DATE to VARCHAR casts are dependent on DateStyle; consider " + "using to_char(date) instead.", - dateStyleAffected: true, }, }, oid.T_float4: { @@ -439,39 +417,34 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_time: {MaxContext: ContextAssignment, origin: ContextOriginPgCast, Volatility: volatility.Immutable}, // Automatic I/O conversions to string types. oid.T_bpchar: { - MaxContext: ContextAssignment, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Immutable, - VolatilityHint: "INTERVAL to CHAR casts depend on IntervalStyle; consider using to_char(interval)", - intervalStyleAffected: true, + MaxContext: ContextAssignment, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "INTERVAL to CHAR casts depend on IntervalStyle; consider using to_char(interval)", }, oid.T_char: { - MaxContext: ContextAssignment, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Immutable, - VolatilityHint: `INTERVAL to "char" casts depend on IntervalStyle; consider using to_char(interval)`, - intervalStyleAffected: true, + MaxContext: ContextAssignment, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: `INTERVAL to "char" casts depend on IntervalStyle; consider using to_char(interval)`, }, oid.T_name: { - MaxContext: ContextAssignment, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Immutable, - VolatilityHint: "INTERVAL to NAME casts depend on IntervalStyle; consider using to_char(interval)", - intervalStyleAffected: true, + MaxContext: ContextAssignment, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "INTERVAL to NAME casts depend on IntervalStyle; consider using to_char(interval)", }, oid.T_text: { - MaxContext: ContextAssignment, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Immutable, - VolatilityHint: "INTERVAL to STRING casts depend on IntervalStyle; consider using to_char(interval)", - intervalStyleAffected: true, + MaxContext: ContextAssignment, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "INTERVAL to STRING casts depend on IntervalStyle; consider using to_char(interval)", }, oid.T_varchar: { - MaxContext: ContextAssignment, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Immutable, - VolatilityHint: "INTERVAL to VARCHAR casts depend on IntervalStyle; consider using to_char(interval)", - intervalStyleAffected: true, + MaxContext: ContextAssignment, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "INTERVAL to VARCHAR casts depend on IntervalStyle; consider using to_char(interval)", }, }, oid.T_jsonb: { @@ -503,11 +476,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oidext.T_box2d: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_bytea: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_date: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "NAME to DATE casts depend on session DateStyle; use parse_date(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "NAME to DATE casts depend on session DateStyle; use parse_date(string) instead", }, oid.T_float4: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_float8: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -518,12 +490,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_int4: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_int8: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_interval: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, - VolatilityHint: "NAME to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead", - intervalStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "NAME to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead", }, oid.T_jsonb: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_numeric: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -536,11 +506,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_regrole: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Stable}, oid.T_regtype: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Stable}, oid.T_time: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "NAME to TIME casts depend on session DateStyle; use parse_time(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "NAME to TIME casts depend on session DateStyle; use parse_time(string) instead", }, oid.T_timestamp: { MaxContext: ContextExplicit, @@ -548,7 +517,6 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ Volatility: volatility.Stable, VolatilityHint: "NAME to TIMESTAMP casts are context-dependent because of relative timestamp strings " + "like 'now' and session settings such as DateStyle; use parse_timestamp(string) instead.", - dateStyleAffected: true, }, oid.T_timestamptz: { MaxContext: ContextExplicit, @@ -556,11 +524,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ Volatility: volatility.Stable, }, oid.T_timetz: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "NAME to TIMETZ casts depend on session DateStyle; use parse_timetz(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "NAME to TIMETZ casts depend on session DateStyle; use parse_timetz(string) instead", }, oid.T_uuid: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_varbit: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -741,11 +708,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oidext.T_box2d: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_bytea: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_date: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "STRING to DATE casts depend on session DateStyle; use parse_date(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "STRING to DATE casts depend on session DateStyle; use parse_date(string) instead", }, oid.T_float4: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_float8: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -755,12 +721,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_int4: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_int8: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_interval: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, - VolatilityHint: "STRING to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead", - intervalStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "STRING to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead", }, oid.T_jsonb: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_numeric: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -772,11 +736,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_regrole: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Stable}, oid.T_regtype: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Stable}, oid.T_time: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "STRING to TIME casts depend on session DateStyle; use parse_time(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "STRING to TIME casts depend on session DateStyle; use parse_time(string) instead", }, oid.T_timestamp: { MaxContext: ContextExplicit, @@ -784,7 +747,6 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ Volatility: volatility.Stable, VolatilityHint: "STRING to TIMESTAMP casts are context-dependent because of relative timestamp strings " + "like 'now' and session settings such as DateStyle; use parse_timestamp(string) instead.", - dateStyleAffected: true, }, oid.T_timestamptz: { MaxContext: ContextExplicit, @@ -792,11 +754,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ Volatility: volatility.Stable, }, oid.T_timetz: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "STRING to TIMETZ casts depend on session DateStyle; use parse_timetz(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "STRING to TIMETZ casts depend on session DateStyle; use parse_timetz(string) instead", }, oid.T_uuid: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_varbit: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -828,47 +789,37 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_bpchar: { MaxContext: ContextAssignment, origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, + Volatility: volatility.Stable, VolatilityHint: "TIMESTAMP to CHAR casts are dependent on DateStyle; consider " + "using to_char(timestamp) instead.", - dateStyleAffected: true, }, oid.T_char: { MaxContext: ContextAssignment, origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, + Volatility: volatility.Stable, VolatilityHint: `TIMESTAMP to "char" casts are dependent on DateStyle; consider ` + "using to_char(timestamp) instead.", - dateStyleAffected: true, }, oid.T_name: { MaxContext: ContextAssignment, origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, + Volatility: volatility.Stable, VolatilityHint: "TIMESTAMP to NAME casts are dependent on DateStyle; consider " + "using to_char(timestamp) instead.", - dateStyleAffected: true, }, oid.T_text: { MaxContext: ContextAssignment, origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, + Volatility: volatility.Stable, VolatilityHint: "TIMESTAMP to STRING casts are dependent on DateStyle; consider " + "using to_char(timestamp) instead.", - dateStyleAffected: true, }, oid.T_varchar: { MaxContext: ContextAssignment, origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, + Volatility: volatility.Stable, VolatilityHint: "TIMESTAMP to VARCHAR casts are dependent on DateStyle; consider " + "using to_char(timestamp) instead.", - dateStyleAffected: true, }, }, oid.T_timestamptz: { @@ -970,11 +921,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oidext.T_box2d: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_bytea: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_date: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "VARCHAR to DATE casts depend on session DateStyle; use parse_date(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "VARCHAR to DATE casts depend on session DateStyle; use parse_date(string) instead", }, oid.T_float4: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_float8: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -985,12 +935,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_int4: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_int8: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_interval: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - // TODO(mgartner): This should be stable. - Volatility: volatility.Immutable, - VolatilityHint: "VARCHAR to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead", - intervalStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "VARCHAR to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead", }, oid.T_jsonb: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_numeric: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, @@ -1002,11 +950,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ oid.T_regrole: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Stable}, oid.T_regtype: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Stable}, oid.T_time: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "VARCHAR to TIME casts depend on session DateStyle; use parse_time(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "VARCHAR to TIME casts depend on session DateStyle; use parse_time(string) instead", }, oid.T_timestamp: { MaxContext: ContextExplicit, @@ -1014,7 +961,6 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ Volatility: volatility.Stable, VolatilityHint: "VARCHAR to TIMESTAMP casts are context-dependent because of relative timestamp strings " + "like 'now' and session settings such as DateStyle; use parse_timestamp(string) instead.", - dateStyleAffected: true, }, oid.T_timestamptz: { MaxContext: ContextExplicit, @@ -1022,11 +968,10 @@ var castMap = map[oid.Oid]map[oid.Oid]Cast{ Volatility: volatility.Stable, }, oid.T_timetz: { - MaxContext: ContextExplicit, - origin: ContextOriginAutomaticIOConversion, - Volatility: volatility.Stable, - VolatilityHint: "VARCHAR to TIMETZ casts depend on session DateStyle; use parse_timetz(string) instead", - dateStyleAffected: true, + MaxContext: ContextExplicit, + origin: ContextOriginAutomaticIOConversion, + Volatility: volatility.Stable, + VolatilityHint: "VARCHAR to TIMETZ casts depend on session DateStyle; use parse_timetz(string) instead", }, oid.T_uuid: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, oid.T_varbit: {MaxContext: ContextExplicit, origin: ContextOriginAutomaticIOConversion, Volatility: volatility.Immutable}, diff --git a/pkg/sql/sem/cast/cast_test.go b/pkg/sql/sem/cast/cast_test.go index 0765855cbb2a..08145cfa332f 100644 --- a/pkg/sql/sem/cast/cast_test.go +++ b/pkg/sql/sem/cast/cast_test.go @@ -175,7 +175,7 @@ func TestCastsFromUnknown(t *testing.T) { defer log.Scope(t).Close(t) for _, typ := range types.OidToType { - _, ok := LookupCast(types.Unknown, typ, SessionOptions{}) + _, ok := LookupCast(types.Unknown, typ) if !ok { t.Errorf("cast from Unknown to %s does not exist", typ.String()) } @@ -227,7 +227,7 @@ func TestTupleCastVolatility(t *testing.T) { from.InternalType.TupleContents = tc.from to := *types.EmptyTuple to.InternalType.TupleContents = tc.to - v, ok := LookupCastVolatility(&from, &to, SessionOptions{}) + v, ok := LookupCastVolatility(&from, &to) res := "error" if ok { res = v.String() diff --git a/pkg/sql/sem/eval/const.go b/pkg/sql/sem/eval/const.go index c676ccc17178..ebac64748623 100644 --- a/pkg/sql/sem/eval/const.go +++ b/pkg/sql/sem/eval/const.go @@ -28,7 +28,7 @@ var _ tree.Visitor = &isConstVisitor{} func (v *isConstVisitor) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) { if v.isConst { - if !tree.OperatorIsImmutable(expr, v.ctx.CastSessionOptions()) || + if !tree.OperatorIsImmutable(expr) || IsVar(v.ctx, expr, true /*allowConstPlaceholders*/) { v.isConst = false return false, expr diff --git a/pkg/sql/sem/eval/context.go b/pkg/sql/sem/eval/context.go index 32df55b4314b..26eff83ebd06 100644 --- a/pkg/sql/sem/eval/context.go +++ b/pkg/sql/sem/eval/context.go @@ -24,7 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" @@ -292,17 +291,6 @@ func (ec *Context) SessionData() *sessiondata.SessionData { return ec.SessionDataStack.Top() } -// CastSessionOptions returns the SessionOptions for casts. -func (ec *Context) CastSessionOptions() cast.SessionOptions { - if ec.SessionData() == nil { - return cast.SessionOptions{} - } - return cast.SessionOptions{ - IntervalStyleEnabled: ec.SessionData().IntervalStyleEnabled, - DateStyleEnabled: ec.SessionData().DateStyleEnabled, - } -} - // Copy returns a deep copy of ctx. func (ec *Context) Copy() *Context { ctxCopy := *ec diff --git a/pkg/sql/sem/normalize/visitor.go b/pkg/sql/sem/normalize/visitor.go index 6881652c5504..e52c48fa60ea 100644 --- a/pkg/sql/sem/normalize/visitor.go +++ b/pkg/sql/sem/normalize/visitor.go @@ -198,7 +198,7 @@ func (v *fastIsConstVisitor) VisitPre(expr tree.Expr) (recurse bool, newExpr tre // If the parent expression is a variable or non-immutable operator, we know // that it is not constant. - if !tree.OperatorIsImmutable(expr, v.ctx.CastSessionOptions()) || + if !tree.OperatorIsImmutable(expr) || eval.IsVar(v.ctx, expr, true /*allowConstPlaceholders*/) { v.isConst = false return false, expr diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index de446c463c12..68746fec834a 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -204,7 +204,6 @@ go_test( "//pkg/sql/parser", "//pkg/sql/randgen", "//pkg/sql/sem/builtins", - "//pkg/sql/sem/cast", "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", "//pkg/sql/sem/normalize", diff --git a/pkg/sql/sem/tree/constant_eval.go b/pkg/sql/sem/tree/constant_eval.go index 4cb8a741545c..a807a1ea48fa 100644 --- a/pkg/sql/sem/tree/constant_eval.go +++ b/pkg/sql/sem/tree/constant_eval.go @@ -19,13 +19,13 @@ import ( // constant operator. Note importantly that this will return true for all // expr types other than FuncExpr, CastExpr, UnaryExpr, BinaryExpr, and // ComparisonExpr. It does not do any recursive searching. -func OperatorIsImmutable(expr Expr, opts cast.SessionOptions) bool { +func OperatorIsImmutable(expr Expr) bool { switch t := expr.(type) { case *FuncExpr: return t.fnProps.Class == NormalClass && t.fn.Volatility <= volatility.Immutable case *CastExpr: - v, ok := cast.LookupCastVolatility(t.Expr.(TypedExpr).ResolvedType(), t.typ, opts) + v, ok := cast.LookupCastVolatility(t.Expr.(TypedExpr).ResolvedType(), t.typ) return ok && v <= volatility.Immutable case *UnaryExpr: diff --git a/pkg/sql/sem/tree/datum_invariants_test.go b/pkg/sql/sem/tree/datum_invariants_test.go index ec531d5285f8..f0270d872d1b 100644 --- a/pkg/sql/sem/tree/datum_invariants_test.go +++ b/pkg/sql/sem/tree/datum_invariants_test.go @@ -13,7 +13,6 @@ package tree import ( "testing" - "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -23,13 +22,7 @@ func TestAllTypesCastableToString(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) for _, typ := range types.Scalar { - if err := resolveCast( - "", - typ, - types.String, - true, /* allowStable */ - cast.SessionOptions{}, - ); err != nil { + if err := resolveCast("", typ, types.String, true); err != nil { t.Errorf("%s is not castable to STRING, all types should be", typ) } } @@ -39,13 +32,7 @@ func TestAllTypesCastableFromString(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) for _, typ := range types.Scalar { - if err := resolveCast( - "", - types.String, - typ, - true, /* allowStable */ - cast.SessionOptions{}, - ); err != nil { + if err := resolveCast("", types.String, typ, true); err != nil { t.Errorf("%s is not castable from STRING, all types should be", typ) } } diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index 730379ebee7b..5e01859075c6 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -62,10 +62,6 @@ type SemaContext struct { // name of a table given its ID. TableNameResolver QualifiedNameResolver - // IntervalStyleEnabled determines whether IntervalStyle is enabled. - // TODO(sql-exp): remove this field in 22.2, since it will always be true. - CastSessionOptions cast.SessionOptions - Properties SemaProperties // DateStyle refers to the DateStyle to parse as. @@ -432,20 +428,12 @@ func invalidCastError(castFrom, castTo *types.T) error { // is false, it also checks that the cast has Immutable. // // On success, any relevant telemetry counters are incremented. -func resolveCast( - context string, castFrom, castTo *types.T, allowStable bool, opts cast.SessionOptions, -) error { +func resolveCast(context string, castFrom, castTo *types.T, allowStable bool) error { toFamily := castTo.Family() fromFamily := castFrom.Family() switch { case toFamily == types.ArrayFamily && fromFamily == types.ArrayFamily: - err := resolveCast( - context, - castFrom.ArrayContents(), - castTo.ArrayContents(), - allowStable, - opts, - ) + err := resolveCast(context, castFrom.ArrayContents(), castTo.ArrayContents(), allowStable) if err != nil { return err } @@ -476,13 +464,7 @@ func resolveCast( } for i, from := range fromTuple { to := toTuple[i] - err := resolveCast( - context, - from, - to, - allowStable, - opts, - ) + err := resolveCast(context, from, to, allowStable) if err != nil { return err } @@ -491,7 +473,7 @@ func resolveCast( return nil default: - cast, ok := cast.LookupCast(castFrom, castTo, opts) + cast, ok := cast.LookupCast(castFrom, castTo) if !ok { return invalidCastError(castFrom, castTo) } @@ -604,17 +586,7 @@ func (expr *CastExpr) TypeCheck( allowStable = false context = semaCtx.Properties.required.context } - var castOpts cast.SessionOptions - if semaCtx != nil { - castOpts = semaCtx.CastSessionOptions - } - err = resolveCast( - context, - castFrom, - exprType, - allowStable, - castOpts, - ) + err = resolveCast(context, castFrom, exprType, allowStable) if err != nil { return nil, err } diff --git a/pkg/sql/sessiondatapb/session_data.proto b/pkg/sql/sessiondatapb/session_data.proto index 78f9e866e157..aba668768a74 100644 --- a/pkg/sql/sessiondatapb/session_data.proto +++ b/pkg/sql/sessiondatapb/session_data.proto @@ -63,12 +63,8 @@ message SessionData { // WorkMemLimit determines how much RAM (in bytes) a single operation of a // single query can use before it has to spill to disk. int64 WorkMemLimit = 12; - // IntervalStyleEnabled allows the setting of IntervalStyle to different - // values. - bool interval_style_enabled = 13; - // DateStyleEnabled allows the setting of DateStyle to different - // values. - bool date_style_enabled = 14; + reserved 13; + reserved 14; // LockTimeout is the maximum amount of time that a query will wait while // attempting to acquire a lock on a key or while blocking on an existing // lock in order to perform a non-locking read on a key. diff --git a/pkg/sql/stats/automatic_stats_test.go b/pkg/sql/stats/automatic_stats_test.go index 8bace90b90f7..bda2420a8207 100644 --- a/pkg/sql/stats/automatic_stats_test.go +++ b/pkg/sql/stats/automatic_stats_test.go @@ -681,7 +681,8 @@ func TestDefaultColumns(t *testing.T) { sqlRun := sqlutils.MakeSQLRunner(sqlDB) sqlRun.Exec(t, `CREATE DATABASE t; - CREATE TABLE t.a (c0 INT PRIMARY KEY);`) + CREATE TABLE t.a (c0 INT PRIMARY KEY) + WITH (sql_stats_automatic_collection_enabled = false);`) for i := 1; i < 110; i++ { // Add more columns than we will collect stats on. @@ -691,17 +692,6 @@ func TestDefaultColumns(t *testing.T) { sqlRun.Exec(t, `CREATE STATISTICS s FROM t.a`) - // TODO(rytaft): this extra logging was added to help debug issue #81513. - // Remove it once that issue is resolved. - // === BEGINNING OF EXTRA LOGGING === - res := sqlRun.QueryStr(t, `SHOW CREATE TABLE t.a`) - t.Log(sqlutils.MatrixToStr(res)) - res = sqlRun.QueryStr(t, `EXPLAIN (DISTSQL) CREATE STATISTICS s FROM t.a`) - t.Log(sqlutils.MatrixToStr(res)) - res = sqlRun.QueryStr(t, `SHOW STATISTICS FOR TABLE t.a`) - t.Log(sqlutils.MatrixToStr(res)) - // === END OF EXTRA LOGGING === - // There should be 101 stats. One for the primary index, plus 100 other // columns. sqlRun.CheckQueryResults(t, diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index c930240e27ce..59ee4c0b5dcc 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -21,7 +21,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/build" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -317,28 +316,6 @@ var varGen = map[string]sessionVar{ if ds.Style != pgdate.Style_ISO { return unimplemented.NewWithIssue(41773, "only ISO style is supported") } - allowed := m.data.DateStyleEnabled - if m.settings.Version.IsActive(ctx, clusterversion.DateStyleIntervalStyleCastRewrite) { - allowed = true - } - if ds.Order != pgdate.Order_MDY && !allowed { - return errors.WithDetailf( - errors.WithHintf( - pgerror.Newf( - pgcode.FeatureNotSupported, - "setting DateStyle is not enabled", - ), - "You can enable DateStyle customization for all sessions with the cluster setting %s, or per session using SET datestyle_enabled = true.", - dateStyleEnabledClusterSetting, - ), - "Setting DateStyle changes the volatility of timestamp/timestamptz/date::string "+ - "and string::timestamp/timestamptz/date/time/timetz casts from immutable to stable. "+ - "No computed columns, partial indexes, partitions and check constraints can "+ - "use these casts. "+ - "Use to_char_with_style or parse_{timestamp,timestamptz,date,time,timetz} "+ - "instead if you need these casts to work in the aforementioned cases.", - ) - } m.SetDateStyle(ds) return nil }, @@ -356,38 +333,10 @@ var varGen = map[string]sessionVar{ }, }, - // TODO(sql-exp): remove this setting in 22.2 by turning it into a no-op. - `datestyle_enabled`: { - Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) { - if evalCtx.Settings.Version.IsActive(evalCtx.Ctx(), clusterversion.DateStyleIntervalStyleCastRewrite) { - return formatBoolAsPostgresSetting(true), nil - } - return formatBoolAsPostgresSetting(evalCtx.SessionData().DateStyleEnabled), nil - }, - GetStringVal: makePostgresBoolGetStringValFn("datestyle_enabled"), - SetWithPlanner: func(ctx context.Context, p *planner, local bool, s string) error { - b, err := paramparse.ParseBoolVar(`datestyle_enabled`, s) - if err != nil { - return err - } - if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.DateStyleIntervalStyleCastRewrite) { - p.BufferClientNotice(ctx, pgnotice.Newf("ignoring datestyle_enabled setting; it is always true")) - b = true - } - applyFunc := func(m sessionDataMutator) error { - m.SetDateStyleEnabled(b) - return nil - } - if local { - return p.sessionDataMutatorIterator.applyOnTopMutator(applyFunc) - } - return p.sessionDataMutatorIterator.applyOnEachMutatorError(applyFunc) - - }, - GlobalDefault: func(sv *settings.Values) string { - return formatBoolAsPostgresSetting(dateStyleEnabled.Get(sv)) - }, - }, + // This is only kept for backwards compatibility and no longer has any effect. + `datestyle_enabled`: makeBackwardsCompatBoolVar( + "datestyle_enabled", true, + ), // Controls the subsequent parsing of a "naked" INT type. // TODO(bob): Remove or no-op this in v2.4: https://github.com/cockroachdb/cockroach/issues/32844 @@ -957,27 +906,6 @@ var varGen = map[string]sessionVar{ return newVarValueError(`IntervalStyle`, s, validIntervalStyles...) } style := duration.IntervalStyle(styleVal) - allowed := m.data.IntervalStyleEnabled - if m.settings.Version.IsActive(ctx, clusterversion.DateStyleIntervalStyleCastRewrite) { - allowed = true - } - if style != duration.IntervalStyle_POSTGRES && !allowed { - return errors.WithDetailf( - errors.WithHintf( - pgerror.Newf( - pgcode.FeatureNotSupported, - "setting IntervalStyle is not enabled", - ), - "You can enable IntervalStyle customization for all sessions with the cluster setting %s, or per session using SET intervalstyle_enabled = true.", - intervalStyleEnabledClusterSetting, - ), - "Setting IntervalStyle changes the volatility of string::interval or interval::string "+ - "casts from immutable to stable. No computed columns, partial indexes, partitions "+ - "and check constraints can use these casts. "+ - "Use to_char_with_style or parse_interval instead if you need these casts to work "+ - "in the aforementioned cases.", - ) - } m.SetIntervalStyle(style) return nil }, @@ -994,37 +922,10 @@ var varGen = map[string]sessionVar{ return a.GetIntervalStyle() == b.GetIntervalStyle() }, }, - // TODO(sql-exp): remove this in 22.2, possibly by converting it into a no-op. - `intervalstyle_enabled`: { - Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) { - if evalCtx.Settings.Version.IsActive(evalCtx.Ctx(), clusterversion.DateStyleIntervalStyleCastRewrite) { - return formatBoolAsPostgresSetting(true), nil - } - return formatBoolAsPostgresSetting(evalCtx.SessionData().IntervalStyleEnabled), nil - }, - GetStringVal: makePostgresBoolGetStringValFn("intervalstyle_enabled"), - SetWithPlanner: func(ctx context.Context, p *planner, local bool, s string) error { - b, err := paramparse.ParseBoolVar(`intervalstyle_enabled`, s) - if err != nil { - return err - } - if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.DateStyleIntervalStyleCastRewrite) { - p.BufferClientNotice(ctx, pgnotice.Newf("ignoring intervalstyle_enabled setting; it is always true")) - b = true - } - applyFunc := func(m sessionDataMutator) error { - m.SetIntervalStyleEnabled(b) - return nil - } - if local { - return p.sessionDataMutatorIterator.applyOnTopMutator(applyFunc) - } - return p.sessionDataMutatorIterator.applyOnEachMutatorError(applyFunc) - }, - GlobalDefault: func(sv *settings.Values) string { - return formatBoolAsPostgresSetting(intervalStyleEnabled.Get(sv)) - }, - }, + // This is only kept for backwards compatibility and no longer has any effect. + `intervalstyle_enabled`: makeBackwardsCompatBoolVar( + "intervalstyle_enabled", true, + ), `is_superuser`: { Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) { diff --git a/pkg/ui/workspaces/cluster-ui/package.json b/pkg/ui/workspaces/cluster-ui/package.json index f05828074e92..011fac080fb5 100644 --- a/pkg/ui/workspaces/cluster-ui/package.json +++ b/pkg/ui/workspaces/cluster-ui/package.json @@ -1,6 +1,6 @@ { "name": "@cockroachlabs/cluster-ui", - "version": "22.1.0-prerelease-3", + "version": "22.2.0-prerelease-1", "description": "Cluster UI is a library of large features shared between CockroachDB and CockroachCloud", "repository": { "type": "git", diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 0bb149c3dccc..ffe2157863d3 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -9,7 +9,6 @@ go_library( "comment_on_index_migration.go", "descriptor_utils.go", "ensure_no_draining_names.go", - "fix_cast_for_style_migration.go", "grant_option_migration.go", "insert_missing_public_schema_namespace_entry.go", "migrate_span_configs.go", @@ -71,7 +70,6 @@ go_test( "descriptor_utils_test.go", "ensure_constraint_id_test.go", "ensure_no_draining_names_external_test.go", - "fix_cast_for_style_migration_test.go", "grant_option_migration_external_test.go", "helpers_test.go", "main_test.go", diff --git a/pkg/upgrade/upgrades/fix_cast_for_style_migration.go b/pkg/upgrade/upgrades/fix_cast_for_style_migration.go deleted file mode 100644 index 1adcb4b2c0cd..000000000000 --- a/pkg/upgrade/upgrades/fix_cast_for_style_migration.go +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright 2022 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 upgrades - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/upgrade" -) - -func fixCastForStyleMigration( - ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, -) error { - - return runPostDeserializationChangesOnAllDescriptors(ctx, d) - -} diff --git a/pkg/upgrade/upgrades/fix_cast_for_style_migration_test.go b/pkg/upgrade/upgrades/fix_cast_for_style_migration_test.go deleted file mode 100644 index 6e0d89715ab6..000000000000 --- a/pkg/upgrade/upgrades/fix_cast_for_style_migration_test.go +++ /dev/null @@ -1,117 +0,0 @@ -// Copyright 2022 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 upgrades_test - -import ( - "context" - "fmt" - "testing" - - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/stretchr/testify/require" -) - -func TestFixCastForStyle(t *testing.T) { - defer leaktest.AfterTest(t)() - ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - DisableAutomaticVersionUpgrade: make(chan struct{}), - BinaryVersionOverride: clusterversion.ByKey(clusterversion.DateStyleIntervalStyleCastRewrite - 1), - }, - }, - }, - }) - defer tc.Stopper().Stop(ctx) - - sqlDB := tc.ServerConn(0) - tdb := sqlutils.MakeSQLRunner(sqlDB) - - tdb.Exec(t, `CREATE TABLE ds ( -it interval, -s string, -t timestamp, -c string AS ((it + interval '2 minutes')::string) STORED, -c2 interval AS ((s)::interval) STORED, -c3 string AS (t::string) STORED, -c4 interval AS ((s)::INTERVAL(4)) STORED -)`) - tdb.Exec(t, `CREATE INDEX rw ON ds ((it::text))`) - tdb.Exec(t, `CREATE INDEX partial ON ds(it) WHERE (it::text) > 'abc';`) - - tdb.Exec(t, `CREATE TABLE ds2 ( - ch char, - d date, - c string AS (d::string) STORED, - c1 string AS (lower(d::STRING)) STORED, - c2 interval AS (ch::interval) STORED -)`) - - upgrades.Upgrade( - t, - sqlDB, - clusterversion.DateStyleIntervalStyleCastRewrite, - nil, - false, - ) - - tests := []struct { - tName string - expect string - }{ - { - tName: "ds", - expect: `CREATE TABLE public.ds ( - it INTERVAL NULL, - s STRING NULL, - t TIMESTAMP NULL, - c STRING NULL AS (to_char(it + '00:02:00':::INTERVAL)) STORED, - c2 INTERVAL NULL AS (parse_interval(s)) STORED, - c3 STRING NULL AS (to_char(t)) STORED, - c4 INTERVAL NULL AS (parse_interval(s)::INTERVAL(4)) STORED, - rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), - CONSTRAINT ds_pkey PRIMARY KEY (rowid ASC), - INDEX rw (to_char(it) ASC), - INDEX partial (it ASC) WHERE to_char(it) > 'abc':::STRING -)`, - }, - { - tName: "ds2", - expect: `CREATE TABLE public.ds2 ( - ch CHAR NULL, - d DATE NULL, - c STRING NULL AS (to_char(d)) STORED, - c1 STRING NULL AS (lower(to_char(d))) STORED, - c2 INTERVAL NULL AS (parse_interval(ch)) STORED, - rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), - CONSTRAINT ds2_pkey PRIMARY KEY (rowid ASC) -)`, - }, - } - - for _, test := range tests { - t.Run(test.tName, func(t *testing.T) { - row := sqlDB.QueryRow(fmt.Sprintf("SHOW CREATE %s", test.tName)) - var other, create string - err := row.Scan(&other, &create) - require.NoError(t, err) - require.Equal(t, test.expect, create) - }) - } -} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index 83570ba44525..f64baf64057b 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -125,12 +125,6 @@ var upgrades = []upgrade.Upgrade{ NoPrecondition, tenantSettingsTableMigration, ), - upgrade.NewTenantUpgrade( - "Rewrites cast that are negatively affected by DateStyle/IntervalStyle", - toCV(clusterversion.DateStyleIntervalStyleCastRewrite), - NoPrecondition, - fixCastForStyleMigration, - ), upgrade.NewTenantUpgrade( "add the system.span_count table", toCV(clusterversion.SpanCountTable),