diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index d23e49d81f03..d5df08e5caaf 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -1566,6 +1566,8 @@ func remapPublicSchemas( // In CockroachDB, root is our substitute for the postgres user. publicSchemaPrivileges := descpb.NewBasePrivilegeDescriptor(security.AdminRoleName()) // By default, everyone has USAGE and CREATE on the public schema. + // Once https://github.com/cockroachdb/cockroach/issues/70266 is resolved, + // the public role will no longer have CREATE privilege. publicSchemaPrivileges.Grant(security.PublicRoleName(), privilege.List{privilege.CREATE, privilege.USAGE}, false) publicSchemaDesc := schemadesc.NewBuilder(&descpb.SchemaDescriptor{ ParentID: db.GetID(), diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index 610161725256..3f752681e7b5 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -90,7 +90,7 @@ func TestCanSendToFollower(t *testing.T) { future := clock.Now().Add(2*clock.MaxOffset().Nanoseconds(), 0) txn := func(ts hlc.Timestamp) *roachpb.Transaction { - txn := roachpb.MakeTransaction("txn", nil, 0, ts, 0) + txn := roachpb.MakeTransaction("txn", nil, 0, ts, 0, 1) return &txn } withWriteTimestamp := func(txn *roachpb.Transaction, ts hlc.Timestamp) *roachpb.Transaction { diff --git a/pkg/ccl/sqlproxyccl/tenant/directory_test.go b/pkg/ccl/sqlproxyccl/tenant/directory_test.go index d480f809e3ea..8585a09476b8 100644 --- a/pkg/ccl/sqlproxyccl/tenant/directory_test.go +++ b/pkg/ccl/sqlproxyccl/tenant/directory_test.go @@ -101,9 +101,10 @@ func TestWatchPods(t *testing.T) { // Ensure that all addresses have been cleared from the directory, since // it should only return RUNNING addresses. - addrs, err := dir.LookupTenantAddrs(ctx, tenantID) - require.NoError(t, err) - require.Empty(t, addrs) + require.Eventually(t, func() bool { + addrs, _ := dir.LookupTenantAddrs(ctx, tenantID) + return len(addrs) == 0 + }, 10*time.Second, 100*time.Millisecond) // Now shut the tenant directory down. processes := tds.Get(tenantID) @@ -120,10 +121,12 @@ func TestWatchPods(t *testing.T) { require.Equal(t, addr, pod.Addr) require.Equal(t, tenant.DELETING, pod.State) - require.Eventually(t, func() bool { - addrs, _ := dir.LookupTenantAddrs(ctx, tenantID) - return len(addrs) == 0 - }, 10*time.Second, 100*time.Millisecond) + // We know that the directory should have been emptied earlier since we + // don't add DRAINING pods to the directory, so putting the pod into the + // DELETING state should not make a difference. + addrs, err := dir.LookupTenantAddrs(ctx, tenantID) + require.NoError(t, err) + require.Empty(t, addrs) // Resume tenant again by a direct call to the directory server _, err = tds.EnsurePod(ctx, &tenant.EnsurePodRequest{tenantID.ToUint64()}) diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index a531d90467fa..4463378d06c7 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -970,6 +970,8 @@ func init() { varFlag(f, addrSetter{&serverHTTPAddr, &serverHTTPPort}, cliflags.ListenHTTPAddr) varFlag(f, addrSetter{&serverAdvertiseAddr, &serverAdvertisePort}, cliflags.AdvertiseAddr) + varFlag(f, &serverCfg.Locality, cliflags.Locality) + varFlag(f, &serverCfg.MaxOffset, cliflags.MaxOffset) varFlag(f, &storeSpecs, cliflags.Store) stringFlag(f, &startCtx.pidFile, cliflags.PIDFile) stringFlag(f, &startCtx.geoLibsDir, cliflags.GeoLibsDir) diff --git a/pkg/internal/sqlsmith/relational.go b/pkg/internal/sqlsmith/relational.go index a13aaa3d3740..300d8bd91d41 100644 --- a/pkg/internal/sqlsmith/relational.go +++ b/pkg/internal/sqlsmith/relational.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/randgen" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/lib/pq/oid" ) func (s *Smither) makeStmt() (stmt tree.Statement, ok bool) { @@ -235,6 +236,10 @@ func makeEquiJoinExpr(s *Smither, refs colRefs, forJoin bool) (tree.TableExpr, c for (cond == nil || s.coin()) && len(available) > 0 { v := available[0] available = available[1:] + // You can't compare voids. + if v[0].ResolvedType().Oid() == oid.T_void || v[1].ResolvedType().Oid() == oid.T_void { + continue + } expr := tree.NewTypedComparisonExpr(tree.MakeComparisonOperator(tree.EQ), v[0], v[1]) if cond == nil { cond = expr diff --git a/pkg/internal/sqlsmith/scalar.go b/pkg/internal/sqlsmith/scalar.go index 5dffa3108ef9..1f59d8db37bc 100644 --- a/pkg/internal/sqlsmith/scalar.go +++ b/pkg/internal/sqlsmith/scalar.go @@ -277,7 +277,7 @@ var compareOps = [...]tree.ComparisonOperatorSymbol{ } func makeCompareOp(s *Smither, typ *types.T, refs colRefs) (tree.TypedExpr, bool) { - if f := typ.Family(); f != types.BoolFamily && f != types.AnyFamily { + if f := typ.Family(); f != types.BoolFamily && f != types.AnyFamily && f != types.VoidFamily { return nil, false } typ = s.randScalarType() diff --git a/pkg/kv/client_test.go b/pkg/kv/client_test.go index 227ecee4e513..2b66cb9110c5 100644 --- a/pkg/kv/client_test.go +++ b/pkg/kv/client_test.go @@ -937,7 +937,7 @@ func TestNodeIDAndObservedTimestamps(t *testing.T) { t.Run(fmt.Sprintf("direct-txn-%d", i), func(t *testing.T) { db := setup(test.nodeID) now := db.Clock().NowAsClockTimestamp() - kvTxn := roachpb.MakeTransaction("unnamed", nil /*baseKey*/, roachpb.NormalUserPriority, now.ToTimestamp(), db.Clock().MaxOffset().Nanoseconds()) + kvTxn := roachpb.MakeTransaction("unnamed", nil /* baseKey */, roachpb.NormalUserPriority, now.ToTimestamp(), db.Clock().MaxOffset().Nanoseconds(), int32(test.nodeID)) txn := kv.NewTxnFromProto(ctx, db, test.nodeID, now, test.typ, &kvTxn) ots := txn.TestingCloneTxn().ObservedTimestamps if (len(ots) == 1 && ots[0].NodeID == test.nodeID) != test.expObserved { diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go index afbe9fbee7ea..7b6549741beb 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go @@ -75,7 +75,7 @@ func TestRangeLookupWithOpenTransaction(t *testing.T) { // engine. key := testutils.MakeKey(keys.Meta1Prefix, roachpb.KeyMax) now := s.Clock().Now() - txn := roachpb.MakeTransaction("txn", roachpb.Key("foobar"), 0, now, 0) + txn := roachpb.MakeTransaction("txn", roachpb.Key("foobar"), 0, now, 0, int32(s.SQLInstanceID())) if err := storage.MVCCPutProto( context.Background(), s.Engines()[0], nil, key, now, &txn, &roachpb.RangeDescriptor{}); err != nil { @@ -1218,7 +1218,7 @@ func TestMultiRangeScanDeleteRange(t *testing.T) { } now := s.Clock().NowAsClockTimestamp() - txnProto := roachpb.MakeTransaction("MyTxn", nil, 0, now.ToTimestamp(), 0) + txnProto := roachpb.MakeTransaction("MyTxn", nil, 0, now.ToTimestamp(), 0, int32(s.SQLInstanceID())) txn := kv.NewTxnFromProto(ctx, db, s.NodeID(), now, kv.RootTxn, &txnProto) scan := roachpb.NewScan(writes[0], writes[len(writes)-1].Next(), false) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index d21ff6c34857..f15100f3aff9 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -546,7 +546,7 @@ func TestImmutableBatchArgs(t *testing.T) { txn := roachpb.MakeTransaction( "test", nil /* baseKey */, roachpb.NormalUserPriority, - clock.Now(), clock.MaxOffset().Nanoseconds(), + clock.Now(), clock.MaxOffset().Nanoseconds(), int32(ds.getNodeID()), ) origTxnTs := txn.WriteTimestamp @@ -2167,7 +2167,14 @@ func TestMultiRangeGapReverse(t *testing.T) { ds := NewDistSender(cfg) - txn := roachpb.MakeTransaction("foo", nil, 1.0, clock.Now(), 0) + txn := roachpb.MakeTransaction( + "foo", + nil, // baseKey + 1.0, // userPriority + clock.Now(), + 0, // maxOffsetNs + 1, // coordinatorNodeID + ) var ba roachpb.BatchRequest ba.Txn = &txn @@ -2965,7 +2972,7 @@ func TestParallelCommitsDetectIntentMissingCause(t *testing.T) { key := roachpb.Key("a") txn := roachpb.MakeTransaction( "test", key, roachpb.NormalUserPriority, - clock.Now(), clock.MaxOffset().Nanoseconds(), + clock.Now(), clock.MaxOffset().Nanoseconds(), 1, /* coordinatorNodeID */ ) txnRecordPresent := true @@ -3300,7 +3307,7 @@ func TestMultipleErrorsMerged(t *testing.T) { txn := roachpb.MakeTransaction( "test", nil /* baseKey */, roachpb.NormalUserPriority, - clock.Now(), clock.MaxOffset().Nanoseconds(), + clock.Now(), clock.MaxOffset().Nanoseconds(), 1, /* coordinatorNodeID */ ) // We're also going to check that the highest bumped WriteTimestamp makes it // to the merged error. diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go index 1a1b0782fda0..136ec5234745 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go @@ -788,6 +788,7 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { roachpb.UserPriority(0), now.ToTimestamp(), clock.MaxOffset().Nanoseconds(), + 0, /* coordinatorNodeID */ ) // TODO(andrei): I've monkeyed with the priorities on this initial // Transaction to keep the test happy from a previous version in which the diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go index 8c798d2f37ce..539b5b6c9cb0 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go @@ -101,7 +101,8 @@ func makeMockTxnPipeliner(iter condensableSpanSetRangeIterator) (txnPipeliner, * } func makeTxnProto() roachpb.Transaction { - return roachpb.MakeTransaction("test", []byte("key"), 0, hlc.Timestamp{WallTime: 10}, 0) + return roachpb.MakeTransaction("test", []byte("key"), 0, hlc.Timestamp{WallTime: 10}, + 0 /* maxOffsetNs */, 0 /* coordinatorNodeID */) } // TestTxnPipeliner1PCTransaction tests that the writes performed by 1PC diff --git a/pkg/kv/kvnemesis/validator_test.go b/pkg/kv/kvnemesis/validator_test.go index 4ad5efc28c0b..5f8eb2db3f98 100644 --- a/pkg/kv/kvnemesis/validator_test.go +++ b/pkg/kv/kvnemesis/validator_test.go @@ -34,7 +34,8 @@ func withTimestamp(op Operation, ts int) Operation { nil, // baseKey roachpb.NormalUserPriority, hlc.Timestamp{WallTime: int64(ts)}, - 0, + 0, // maxOffsetNs + 0, // coordinatorNodeID ) switch o := op.GetValue().(type) { case *ClosureTxnOperation: diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 639aa8e0cc09..1344a30b13ab 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -623,7 +623,7 @@ func TestEvalAddSSTable(t *testing.T) { defer engine.Close() // Write initial data. - intentTxn := roachpb.MakeTransaction("intentTxn", nil, 0, hlc.Timestamp{WallTime: intentTS}, 0) + intentTxn := roachpb.MakeTransaction("intentTxn", nil, 0, hlc.Timestamp{WallTime: intentTS}, 0, 1) b := engine.NewBatch() for i := len(tc.data) - 1; i >= 0; i-- { // reverse, older timestamps first kv := tc.data[i] diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go index bfda0a576553..04b095bdff12 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -46,7 +46,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { k, k2 := roachpb.Key("a"), roachpb.Key("b") ts, ts2, ts3 := hlc.Timestamp{WallTime: 1}, hlc.Timestamp{WallTime: 2}, hlc.Timestamp{WallTime: 3} - txn := roachpb.MakeTransaction("test", k, 0, ts, 0) + txn := roachpb.MakeTransaction("test", k, 0, ts, 0, 1) writes := []roachpb.SequencedWrite{{Key: k, Sequence: 0}} intents := []roachpb.Span{{Key: k2}} @@ -983,7 +983,7 @@ func TestPartialRollbackOnEndTransaction(t *testing.T) { k := roachpb.Key("a") ts := hlc.Timestamp{WallTime: 1} ts2 := hlc.Timestamp{WallTime: 2} - txn := roachpb.MakeTransaction("test", k, 0, ts, 0) + txn := roachpb.MakeTransaction("test", k, 0, ts, 0, 1) endKey := roachpb.Key("z") desc := roachpb.RangeDescriptor{ RangeID: 99, @@ -1138,7 +1138,7 @@ func TestCommitWaitBeforeIntentResolutionIfCommitTrigger(t *testing.T) { now := clock.Now() commitTS := cfg.commitTS(now) - txn := roachpb.MakeTransaction("test", desc.StartKey.AsRawKey(), 0, now, 0) + txn := roachpb.MakeTransaction("test", desc.StartKey.AsRawKey(), 0, now, 0, 1) txn.ReadTimestamp = commitTS txn.WriteTimestamp = commitTS diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go index adfc31eaae77..04a56500cf89 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go @@ -45,7 +45,7 @@ func TestQueryResolvedTimestamp(t *testing.T) { require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), nil)) } writeIntent := func(k string, ts int64) { - txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, makeTS(ts), 0) + txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, makeTS(ts), 0, 1) require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), &txn)) } writeInline := func(k string) { diff --git a/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go b/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go index cbbf27fa4bca..97964b5b0729 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go @@ -36,7 +36,7 @@ func TestRecoverTxn(t *testing.T) { ctx := context.Background() k, k2 := roachpb.Key("a"), roachpb.Key("b") ts := hlc.Timestamp{WallTime: 1} - txn := roachpb.MakeTransaction("test", k, 0, ts, 0) + txn := roachpb.MakeTransaction("test", k, 0, ts, 0, 1) txn.Status = roachpb.STAGING txn.LockSpans = []roachpb.Span{{Key: k}} txn.InFlightWrites = []roachpb.SequencedWrite{{Key: k2, Sequence: 0}} @@ -103,7 +103,7 @@ func TestRecoverTxnRecordChanged(t *testing.T) { ctx := context.Background() k := roachpb.Key("a") ts := hlc.Timestamp{WallTime: 1} - txn := roachpb.MakeTransaction("test", k, 0, ts, 0) + txn := roachpb.MakeTransaction("test", k, 0, ts, 0, 1) txn.Status = roachpb.STAGING testCases := []struct { diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go index 75605b79791b..aecd442272ec 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go @@ -149,7 +149,7 @@ func TestResolveIntentAfterPartialRollback(t *testing.T) { ts := hlc.Timestamp{WallTime: 1} ts2 := hlc.Timestamp{WallTime: 2} endKey := roachpb.Key("z") - txn := roachpb.MakeTransaction("test", k, 0, ts, 0) + txn := roachpb.MakeTransaction("test", k, 0, ts, 0, 1) desc := roachpb.RangeDescriptor{ RangeID: 99, StartKey: roachpb.RKey(k), diff --git a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go index a6eb546db687..6ed545af52f1 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go @@ -187,7 +187,7 @@ func TestCmdRevertRange(t *testing.T) { }) } - txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, tsC, 1) + txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, tsC, 1, 1) if err := storage.MVCCPut( ctx, eng, &stats, []byte("0012"), tsC, roachpb.MakeValueFromBytes([]byte("i")), &txn, ); err != nil { diff --git a/pkg/kv/kvserver/batcheval/intent_test.go b/pkg/kv/kvserver/batcheval/intent_test.go index 2925b0424103..f287e11da403 100644 --- a/pkg/kv/kvserver/batcheval/intent_test.go +++ b/pkg/kv/kvserver/batcheval/intent_test.go @@ -126,7 +126,7 @@ func TestCollectIntentsUsesSameIterator(t *testing.T) { // Write an intent. val := roachpb.MakeValueFromBytes([]byte("val")) - txn := roachpb.MakeTransaction("test", key, roachpb.NormalUserPriority, ts, 0) + txn := roachpb.MakeTransaction("test", key, roachpb.NormalUserPriority, ts, 0, 1) var err error if delete { err = storage.MVCCDelete(ctx, db, nil, key, ts, &txn) diff --git a/pkg/kv/kvserver/batcheval/transaction_test.go b/pkg/kv/kvserver/batcheval/transaction_test.go index 557b7ceded40..b7481b86738d 100644 --- a/pkg/kv/kvserver/batcheval/transaction_test.go +++ b/pkg/kv/kvserver/batcheval/transaction_test.go @@ -62,7 +62,7 @@ func TestUpdateAbortSpan(t *testing.T) { } as := abortspan.New(desc.RangeID) - txn := roachpb.MakeTransaction("test", txnKey, 0, hlc.Timestamp{WallTime: 1}, 0) + txn := roachpb.MakeTransaction("test", txnKey, 0, hlc.Timestamp{WallTime: 1}, 0, 1) newTxnAbortSpanEntry := roachpb.AbortSpanEntry{ Key: txn.Key, Timestamp: txn.WriteTimestamp, diff --git a/pkg/kv/kvserver/below_raft_protos_test.go b/pkg/kv/kvserver/below_raft_protos_test.go index 8513f62b0b07..ab2f072d9c26 100644 --- a/pkg/kv/kvserver/below_raft_protos_test.go +++ b/pkg/kv/kvserver/below_raft_protos_test.go @@ -71,7 +71,7 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{ return m }, emptySum: 7551962144604783939, - populatedSum: 6784975417727259950, + populatedSum: 6170112718709472849, }, reflect.TypeOf(&enginepb.RangeAppliedState{}): { populatedConstructor: func(r *rand.Rand) protoutil.Message { diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 92e709e58be5..42e951e61210 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -573,8 +573,8 @@ func mergeCheckingTimestampCaches( // Simulate a txn abort on the RHS from a node with a newer clock. Because // the transaction record for the pushee was not yet written, this will bump // the timestamp cache to record the abort. - pushee := roachpb.MakeTransaction("pushee", rhsKey, roachpb.MinUserPriority, readTS, 0) - pusher := roachpb.MakeTransaction("pusher", rhsKey, roachpb.MaxUserPriority, readTS, 0) + pushee := roachpb.MakeTransaction("pushee", rhsKey, roachpb.MinUserPriority, readTS, 0, 0) + pusher := roachpb.MakeTransaction("pusher", rhsKey, roachpb.MaxUserPriority, readTS, 0, 0) ba = roachpb.BatchRequest{} ba.Timestamp = readTS.Next() ba.RangeID = rhsDesc.RangeID @@ -4833,7 +4833,7 @@ func sendWithTxn( args roachpb.Request, ) error { txn := roachpb.MakeTransaction("test txn", desc.StartKey.AsRawKey(), - 0, ts, maxOffset.Nanoseconds()) + 0, ts, maxOffset.Nanoseconds(), 0) _, pErr := kv.SendWrappedWith(context.Background(), store.TestSender(), roachpb.Header{Txn: &txn}, args) return pErr.GoError() } diff --git a/pkg/kv/kvserver/client_metrics_test.go b/pkg/kv/kvserver/client_metrics_test.go index 10f1808d0ed0..46d10ed649df 100644 --- a/pkg/kv/kvserver/client_metrics_test.go +++ b/pkg/kv/kvserver/client_metrics_test.go @@ -187,7 +187,7 @@ func TestStoreResolveMetrics(t *testing.T) { require.NoError(t, err) span := roachpb.Span{Key: key, EndKey: key.Next()} - txn := roachpb.MakeTransaction("foo", span.Key, roachpb.MinUserPriority, hlc.Timestamp{WallTime: 123}, 999) + txn := roachpb.MakeTransaction("foo", span.Key, roachpb.MinUserPriority, hlc.Timestamp{WallTime: 123}, 999, int32(s.NodeID())) const resolveCommitCount = int64(200) const resolveAbortCount = int64(800) diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index c8833609b194..eb9b0d799374 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -544,7 +544,7 @@ func TestTxnReadWithinUncertaintyInterval(t *testing.T) { now := s.Clock().Now() maxOffset := s.Clock().MaxOffset().Nanoseconds() require.NotZero(t, maxOffset) - txn := roachpb.MakeTransaction("test", key, 1, now, maxOffset) + txn := roachpb.MakeTransaction("test", key, 1, now, maxOffset, int32(s.SQLInstanceID())) require.True(t, txn.ReadTimestamp.Less(txn.GlobalUncertaintyLimit)) require.Len(t, txn.ObservedTimestamps, 0) @@ -641,7 +641,7 @@ func TestTxnReadWithinUncertaintyIntervalAfterLeaseTransfer(t *testing.T) { now := clocks[1].Now() maxOffset := clocks[1].MaxOffset().Nanoseconds() require.NotZero(t, maxOffset) - txn := roachpb.MakeTransaction("test", keyB, 1, now, maxOffset) + txn := roachpb.MakeTransaction("test", keyB, 1, now, maxOffset, int32(tc.Servers[1].SQLInstanceID())) require.True(t, txn.ReadTimestamp.Less(txn.GlobalUncertaintyLimit)) require.Len(t, txn.ObservedTimestamps, 0) @@ -1374,7 +1374,7 @@ func TestRangeLocalUncertaintyLimitAfterNewLease(t *testing.T) { } // Start a transaction using node2 as a gateway. - txn := roachpb.MakeTransaction("test", keyA, 1, tc.Servers[1].Clock().Now(), tc.Servers[1].Clock().MaxOffset().Nanoseconds() /* maxOffsetNs */) + txn := roachpb.MakeTransaction("test", keyA, 1 /* userPriority */, tc.Servers[1].Clock().Now(), tc.Servers[1].Clock().MaxOffset().Nanoseconds() /* maxOffsetNs */, int32(tc.Servers[1].SQLInstanceID())) // Simulate a read to another range on node2 by setting the observed timestamp. txn.UpdateObservedTimestamp(2, tc.Servers[1].Clock().NowAsClockTimestamp()) diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index 74fdf1d2dac0..f47826a89688 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -131,7 +131,7 @@ func TestStoreSplitAbortSpan(t *testing.T) { left, middle, right := roachpb.Key("a"), roachpb.Key("b"), roachpb.Key("c") txn := func(key roachpb.Key, ts hlc.Timestamp) *roachpb.Transaction { - txn := roachpb.MakeTransaction("test", key, 0, ts, 0) + txn := roachpb.MakeTransaction("test", key, 0, ts, 0, int32(s.SQLInstanceID())) return &txn } @@ -556,7 +556,7 @@ func TestStoreRangeSplitIdempotency(t *testing.T) { // Increments are a good way of testing idempotency. Up here, we // address them to the original range, then later to the one that // contains the key. - txn := roachpb.MakeTransaction("test", []byte("c"), 10, store.Clock().Now(), 0) + txn := roachpb.MakeTransaction("test", []byte("c"), 10, store.Clock().Now(), 0, int32(s.SQLInstanceID())) lIncArgs := incrementArgs([]byte("apoptosis"), 100) lTxn := txn lTxn.Sequence++ @@ -3168,7 +3168,8 @@ func TestRangeLookupAsyncResolveIntent(t *testing.T) { t.Fatal(err) } txn := roachpb.MakeTransaction("test", key2, 1, - store.Clock().Now(), store.Clock().MaxOffset().Nanoseconds()) + store.Clock().Now(), store.Clock().MaxOffset().Nanoseconds(), + int32(s.SQLInstanceID())) // Officially begin the transaction. If not for this, the intent resolution // machinery would simply remove the intent we write below, see #3020. // We send directly to Replica throughout this test, so there's no danger diff --git a/pkg/kv/kvserver/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index 4a52b1f37251..d27de3a8c9e1 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -103,7 +103,7 @@ func TestClosedTimestampCanServe(t *testing.T) { var baWrite roachpb.BatchRequest r := &roachpb.DeleteRequest{} r.Key = desc.StartKey.AsRawKey() - txn := roachpb.MakeTransaction("testwrite", r.Key, roachpb.NormalUserPriority, ts, 100) + txn := roachpb.MakeTransaction("testwrite", r.Key, roachpb.NormalUserPriority, ts, 100, int32(tc.Server(0).SQLInstanceID())) baWrite.Txn = &txn baWrite.Add(r) baWrite.RangeID = repls[0].RangeID @@ -265,7 +265,7 @@ func TestClosedTimestampCantServeWithConflictingIntent(t *testing.T) { // replica. txnKey := desc.StartKey.AsRawKey() txnKey = txnKey[:len(txnKey):len(txnKey)] // avoid aliasing - txn := roachpb.MakeTransaction("txn", txnKey, 0, tc.Server(0).Clock().Now(), 0) + txn := roachpb.MakeTransaction("txn", txnKey, 0, tc.Server(0).Clock().Now(), 0, int32(tc.Server(0).SQLInstanceID())) var keys []roachpb.Key for i := range repls { key := append(txnKey, []byte(strconv.Itoa(i))...) @@ -1321,7 +1321,7 @@ func verifyCanReadFromAllRepls( } func makeTxnReadBatchForDesc(desc roachpb.RangeDescriptor, ts hlc.Timestamp) roachpb.BatchRequest { - txn := roachpb.MakeTransaction("txn", nil, 0, ts, 0) + txn := roachpb.MakeTransaction("txn", nil, 0, ts, 0, 0) var baRead roachpb.BatchRequest baRead.Header.RangeID = desc.RangeID diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index f3b35552eb95..2ca3be973790 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -125,7 +125,7 @@ func setupLockTableWaiterTest() ( } func makeTxnProto(name string) roachpb.Transaction { - return roachpb.MakeTransaction(name, []byte("key"), 0, hlc.Timestamp{WallTime: 10}, 0) + return roachpb.MakeTransaction(name, []byte("key"), 0, hlc.Timestamp{WallTime: 10}, 0, 6) } // TestLockTableWaiterWithTxn tests the lockTableWaiter's behavior under diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index b95fcfd079de..4e216b0c8d1e 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -119,7 +119,7 @@ func TestIntentAgeThresholdSetting(t *testing.T) { intentHlc := hlc.Timestamp{ WallTime: intentTs.Nanoseconds(), } - txn := roachpb.MakeTransaction("txn", key, roachpb.NormalUserPriority, intentHlc, 1000) + txn := roachpb.MakeTransaction("txn", key, roachpb.NormalUserPriority, intentHlc, 1000, 0) require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, value, &txn)) require.NoError(t, eng.Flush()) @@ -169,7 +169,7 @@ func TestIntentCleanupBatching(t *testing.T) { } for _, prefix := range txnPrefixes { key := []byte{prefix, objectKeys[0]} - txn := roachpb.MakeTransaction("txn", key, roachpb.NormalUserPriority, intentHlc, 1000) + txn := roachpb.MakeTransaction("txn", key, roachpb.NormalUserPriority, intentHlc, 1000, 0) for _, suffix := range objectKeys { key := []byte{prefix, suffix} require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, value, &txn)) diff --git a/pkg/kv/kvserver/intent_resolver_integration_test.go b/pkg/kv/kvserver/intent_resolver_integration_test.go index b4b7aecae968..280989696bdc 100644 --- a/pkg/kv/kvserver/intent_resolver_integration_test.go +++ b/pkg/kv/kvserver/intent_resolver_integration_test.go @@ -468,6 +468,7 @@ func TestReliableIntentCleanup(t *testing.T) { roachpb.MaxUserPriority, now.ToTimestamp(), srv.Clock().MaxOffset().Nanoseconds(), + int32(srv.SQLInstanceID()), ) pusher := kv.NewTxnFromProto(ctx, db, srv.NodeID(), now, kv.RootTxn, &pusherProto) if err := pusher.Put(ctx, txnKey, []byte("pushit")); err != nil { diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver_test.go b/pkg/kv/kvserver/intentresolver/intent_resolver_test.go index 0e1b153f9f7a..f764bb82d8b9 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver_test.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver_test.go @@ -765,7 +765,7 @@ func newTransaction( offset = clock.MaxOffset().Nanoseconds() now = clock.Now() } - txn := roachpb.MakeTransaction(name, baseKey, userPriority, now, offset) + txn := roachpb.MakeTransaction(name, baseKey, userPriority, now, offset, 1 /* coordinatorNodeID */) return &txn } diff --git a/pkg/kv/kvserver/replica_closedts_internal_test.go b/pkg/kv/kvserver/replica_closedts_internal_test.go index 64178d6087e3..2243aaa5cd59 100644 --- a/pkg/kv/kvserver/replica_closedts_internal_test.go +++ b/pkg/kv/kvserver/replica_closedts_internal_test.go @@ -657,7 +657,7 @@ func TestQueryResolvedTimestamp(t *testing.T) { tc.StartWithStoreConfig(t, stopper, cfg) // Write an intent. - txn := roachpb.MakeTransaction("test", intentKey, 0, intentTS, 0) + txn := roachpb.MakeTransaction("test", intentKey, 0, intentTS, 0, 0) pArgs := putArgs(intentKey, []byte("val")) assignSeqNumsForReqs(&txn, &pArgs) _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: &txn}, &pArgs) @@ -697,7 +697,7 @@ func TestQueryResolvedTimestampResolvesAbandonedIntents(t *testing.T) { // Write an intent. key := roachpb.Key("a") - txn := roachpb.MakeTransaction("test", key, 0, ts10, 0) + txn := roachpb.MakeTransaction("test", key, 0, ts10, 0, 0) pArgs := putArgs(key, []byte("val")) assignSeqNumsForReqs(&txn, &pArgs) _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: &txn}, &pArgs) @@ -958,7 +958,7 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { tc.StartWithStoreConfig(t, stopper, cfg) // Write an intent. - txn := roachpb.MakeTransaction("test", intentKey, 0, intentTS, 0) + txn := roachpb.MakeTransaction("test", intentKey, 0, intentTS, 0, 0) pArgs := putArgs(intentKey, []byte("val")) assignSeqNumsForReqs(&txn, &pArgs) _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: &txn}, &pArgs) @@ -1044,7 +1044,7 @@ func TestServerSideBoundedStalenessNegotiationWithResumeSpan(t *testing.T) { send(roachpb.Header{Timestamp: makeTS(ts)}, &pArgs) } writeIntent := func(k string, ts int64) { - txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, makeTS(ts), 0) + txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, makeTS(ts), 0, 0) pArgs := putArgs(roachpb.Key(k), val) assignSeqNumsForReqs(&txn, &pArgs) send(roachpb.Header{Txn: &txn}, &pArgs) diff --git a/pkg/kv/kvserver/replica_closedts_test.go b/pkg/kv/kvserver/replica_closedts_test.go index dc90522b5b1b..67261059102e 100644 --- a/pkg/kv/kvserver/replica_closedts_test.go +++ b/pkg/kv/kvserver/replica_closedts_test.go @@ -774,7 +774,7 @@ func TestNonBlockingReadsAtResolvedTimestamp(t *testing.T) { scan := roachpb.ScanRequest{ RequestHeader: roachpb.RequestHeaderFromSpan(keySpan), } - txn := roachpb.MakeTransaction("test", keySpan.Key, 0, resTS, 0) + txn := roachpb.MakeTransaction("test", keySpan.Key, 0, resTS, 0, 0) scanHeader := roachpb.Header{ RangeID: rangeID, ReadConsistency: roachpb.CONSISTENT, diff --git a/pkg/kv/kvserver/replica_evaluate_test.go b/pkg/kv/kvserver/replica_evaluate_test.go index da99b42eb2c0..8d5c0f2e92ec 100644 --- a/pkg/kv/kvserver/replica_evaluate_test.go +++ b/pkg/kv/kvserver/replica_evaluate_test.go @@ -36,7 +36,7 @@ func TestEvaluateBatch(t *testing.T) { defer log.Scope(t).Close(t) ts := hlc.Timestamp{WallTime: 1} - txn := roachpb.MakeTransaction("test", roachpb.Key("a"), 0, ts, 0) + txn := roachpb.MakeTransaction("test", roachpb.Key("a"), 0, ts, 0, 0) tcs := []testCase{ // diff --git a/pkg/kv/kvserver/replica_follower_read_test.go b/pkg/kv/kvserver/replica_follower_read_test.go index 83ff292abc11..43fdc891c500 100644 --- a/pkg/kv/kvserver/replica_follower_read_test.go +++ b/pkg/kv/kvserver/replica_follower_read_test.go @@ -79,9 +79,12 @@ func TestCanServeFollowerRead(t *testing.T) { gArgs := getArgs(key) txn := roachpb.MakeTransaction( - "test", key, roachpb.NormalUserPriority, + "test", + key, + roachpb.NormalUserPriority, test.readTimestamp, clock.MaxOffset().Nanoseconds(), + 0, // coordinatorNodeID ) ba := &roachpb.BatchRequest{} @@ -155,9 +158,12 @@ func TestCheckExecutionCanProceedAllowsFollowerReadWithInvalidLease(t *testing.T gArgs := getArgs(key) txn := roachpb.MakeTransaction( - "test", key, roachpb.NormalUserPriority, + "test", + key, + roachpb.NormalUserPriority, tsBelowClosedTimestamp, clock.MaxOffset().Nanoseconds(), + 0, // coordinatorNodeID ) ba := &roachpb.BatchRequest{} diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index e9eb70ecc5c1..202ed815b552 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -900,7 +900,7 @@ func TestLearnerAndVoterOutgoingFollowerRead(t *testing.T) { check := func() { ts := tc.Server(0).Clock().Now() - txn := roachpb.MakeTransaction("txn", nil, 0, ts, 0) + txn := roachpb.MakeTransaction("txn", nil, 0, ts, 0, int32(tc.Server(0).SQLInstanceID())) req := roachpb.BatchRequest{Header: roachpb.Header{ RangeID: scratchDesc.RangeID, Timestamp: ts, diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 7040e7238f8b..d75885572d92 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -322,7 +322,7 @@ func newTransaction( offset = clock.MaxOffset().Nanoseconds() now = clock.Now() } - txn := roachpb.MakeTransaction(name, baseKey, userPriority, now, offset) + txn := roachpb.MakeTransaction(name, baseKey, userPriority, now, offset, 0) return &txn } @@ -4549,8 +4549,8 @@ func TestErrorsDontCarryWriteTooOldFlag(t *testing.T) { keyA := roachpb.Key("a") keyB := roachpb.Key("b") // Start a transaction early to get a low timestamp. - txn := roachpb.MakeTransaction( - "test", keyA, roachpb.NormalUserPriority, tc.Clock().Now(), 0 /* offset */) + txn := roachpb.MakeTransaction("test", keyA, roachpb.NormalUserPriority, + tc.Clock().Now(), 0 /* maxOffsetNs */, 0 /* coordinatorNodeID */) // Write a value outside of the txn to cause a WriteTooOldError later. put := putArgs(keyA, []byte("val1")) @@ -10021,7 +10021,7 @@ func TestReplicaServersideRefreshes(t *testing.T) { newTxn := func(key string, ts hlc.Timestamp) *roachpb.Transaction { txn := roachpb.MakeTransaction( - "test", roachpb.Key(key), roachpb.NormalUserPriority, ts, 0, + "test", roachpb.Key(key), roachpb.NormalUserPriority, ts, 0, 0, ) return &txn } @@ -10551,7 +10551,7 @@ func TestReplicaPushed1PC(t *testing.T) { // Start a transaction and assign its ReadTimestamp. ts1 := tc.Clock().Now() - txn := roachpb.MakeTransaction("test", k, roachpb.NormalUserPriority, ts1, 0) + txn := roachpb.MakeTransaction("test", k, roachpb.NormalUserPriority, ts1, 0, 0) // Write a value outside the transaction. tc.manualClock.Increment(10) diff --git a/pkg/kv/kvserver/txnrecovery/manager_test.go b/pkg/kv/kvserver/txnrecovery/manager_test.go index 2a20cedc2530..6823debf4e7a 100644 --- a/pkg/kv/kvserver/txnrecovery/manager_test.go +++ b/pkg/kv/kvserver/txnrecovery/manager_test.go @@ -41,7 +41,7 @@ func makeManager(s *kv.Sender) (Manager, *hlc.Clock, *stop.Stopper) { func makeStagingTransaction(clock *hlc.Clock) roachpb.Transaction { now := clock.Now() offset := clock.MaxOffset().Nanoseconds() - txn := roachpb.MakeTransaction("test", roachpb.Key("a"), 0, now, offset) + txn := roachpb.MakeTransaction("test", roachpb.Key("a"), 0, now, offset, 0) txn.Status = roachpb.STAGING return txn } diff --git a/pkg/kv/kvserver/txnwait/queue_test.go b/pkg/kv/kvserver/txnwait/queue_test.go index aaf82c100493..74f1a7edf4fd 100644 --- a/pkg/kv/kvserver/txnwait/queue_test.go +++ b/pkg/kv/kvserver/txnwait/queue_test.go @@ -198,7 +198,7 @@ func TestMaybeWaitForPushWithContextCancellation(t *testing.T) { q.Enable(1 /* leaseSeq */) // Enqueue pushee transaction in the queue. - txn := roachpb.MakeTransaction("test", nil, 0, cfg.Clock.Now(), 0) + txn := roachpb.MakeTransaction("test", nil, 0, cfg.Clock.Now(), 0, 0) q.EnqueueTxn(&txn) // Mock out responses to any QueryTxn requests. @@ -287,7 +287,7 @@ func TestPushersReleasedAfterAnyQueryTxnFindsAbortedTxn(t *testing.T) { defer TestingOverrideTxnLivenessThreshold(time.Hour)() // Enqueue pushee transaction in the queue. - txn := roachpb.MakeTransaction("test", nil, 0, cfg.Clock.Now(), 0) + txn := roachpb.MakeTransaction("test", nil, 0, cfg.Clock.Now(), 0, 0) q.EnqueueTxn(&txn) const numPushees = 3 diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index f19b6d479e0c..2c01af05fb94 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -130,6 +130,7 @@ func NewTxn(ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID) *Txn { roachpb.NormalUserPriority, now.ToTimestamp(), db.clock.MaxOffset().Nanoseconds(), + int32(db.ctx.NodeID.SQLInstanceID()), ) return NewTxnFromProto(ctx, db, gatewayNodeID, now, RootTxn, &kvTxn) @@ -1467,7 +1468,8 @@ func (txn *Txn) GenerateForcedRetryableError(ctx context.Context, msg string) er txn.mu.userPriority, now.ToTimestamp(), txn.db.clock.MaxOffset().Nanoseconds(), - )) + int32(txn.db.ctx.NodeID.SQLInstanceID())), + ) } // PrepareRetryableError returns a diff --git a/pkg/roachpb/api_test.go b/pkg/roachpb/api_test.go index 2a445bc5dc74..45d0ce748b8f 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/roachpb/api_test.go @@ -258,7 +258,7 @@ func TestMustSetInner(t *testing.T) { func TestContentionEvent_SafeFormat(t *testing.T) { ce := &ContentionEvent{ Key: Key("foo"), - TxnMeta: enginepb.TxnMeta{ID: uuid.FromStringOrNil("51b5ef6a-f18f-4e85-bc3f-c44e33f2bb27")}, + TxnMeta: enginepb.TxnMeta{ID: uuid.FromStringOrNil("51b5ef6a-f18f-4e85-bc3f-c44e33f2bb27"), CoordinatorNodeID: 6}, } const exp = redact.RedactableString(`conflicted with ‹51b5ef6a-f18f-4e85-bc3f-c44e33f2bb27› on ‹"foo"› for 0.000s`) require.Equal(t, exp, redact.Sprint(ce)) diff --git a/pkg/roachpb/batch_test.go b/pkg/roachpb/batch_test.go index bf0ceaa34715..9ee32adaeb7d 100644 --- a/pkg/roachpb/batch_test.go +++ b/pkg/roachpb/batch_test.go @@ -344,7 +344,7 @@ func TestBatchResponseCombine(t *testing.T) { { txn := MakeTransaction( "test", nil /* baseKey */, NormalUserPriority, - hlc.Timestamp{WallTime: 123}, 0, /* maxOffsetNs */ + hlc.Timestamp{WallTime: 123}, 0 /* baseKey */, 99, /* coordinatorNodeID */ ) brTxn := &BatchResponse{ BatchResponse_Header: BatchResponse_Header{ diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index e5eb08f6bfee..bf2d1a1a3b70 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -914,10 +914,20 @@ func (TransactionStatus) SafeValue() {} // write conflicts in a way that avoids starvation of long-running // transactions (see Replica.PushTxn). // +// coordinatorNodeID is provided to track the SQL (or possibly KV) node +// that created this transaction, in order to be used (as +// of this writing) to enable observability on contention events +// between different transactions. +// // baseKey can be nil, in which case it will be set when sending the first // write. func MakeTransaction( - name string, baseKey Key, userPriority UserPriority, now hlc.Timestamp, maxOffsetNs int64, + name string, + baseKey Key, + userPriority UserPriority, + now hlc.Timestamp, + maxOffsetNs int64, + coordinatorNodeID int32, ) Transaction { u := uuid.FastMakeV4() // TODO(nvanbenschoten): technically, gul should be a synthetic timestamp. @@ -927,12 +937,13 @@ func MakeTransaction( return Transaction{ TxnMeta: enginepb.TxnMeta{ - Key: baseKey, - ID: u, - WriteTimestamp: now, - MinTimestamp: now, - Priority: MakePriority(userPriority), - Sequence: 0, // 1-indexed, incremented before each Request + Key: baseKey, + ID: u, + WriteTimestamp: now, + MinTimestamp: now, + Priority: MakePriority(userPriority), + Sequence: 0, // 1-indexed, incremented before each Request + CoordinatorNodeID: coordinatorNodeID, }, Name: name, LastHeartbeat: now, @@ -1446,6 +1457,7 @@ func PrepareTransactionForRetry( NormalUserPriority, now.ToTimestamp(), clock.MaxOffset().Nanoseconds(), + txn.CoordinatorNodeID, ) // Use the priority communicated back by the server. txn.Priority = errTxnPri diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 56ef630faa0c..f38d012cb49c 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -402,7 +402,7 @@ func TestSetGetChecked(t *testing.T) { func TestTransactionBumpEpoch(t *testing.T) { origNow := makeTS(10, 1) - txn := MakeTransaction("test", Key("a"), 1, origNow, 0) + txn := MakeTransaction("test", Key("a"), 1, origNow, 0, 99) // Advance the txn timestamp. txn.WriteTimestamp = txn.WriteTimestamp.Add(10, 2) txn.BumpEpoch() @@ -469,13 +469,14 @@ func TestFastPathObservedTimestamp(t *testing.T) { var nonZeroTxn = Transaction{ TxnMeta: enginepb.TxnMeta{ - Key: Key("foo"), - ID: uuid.MakeV4(), - Epoch: 2, - WriteTimestamp: makeSynTS(20, 21), - MinTimestamp: makeSynTS(10, 11), - Priority: 957356782, - Sequence: 123, + Key: Key("foo"), + ID: uuid.MakeV4(), + Epoch: 2, + WriteTimestamp: makeSynTS(20, 21), + MinTimestamp: makeSynTS(10, 11), + Priority: 957356782, + Sequence: 123, + CoordinatorNodeID: 3, }, Name: "name", Status: COMMITTED, @@ -517,6 +518,7 @@ func TestTransactionUpdate(t *testing.T) { txn3.Status = STAGING txn3.Name = "carl" txn3.Priority = 123 + txn3.CoordinatorNodeID = 3 txn3.Update(&txn) expTxn3 := txn @@ -531,6 +533,7 @@ func TestTransactionUpdate(t *testing.T) { txn4.Sequence = txn.Sequence + 10 txn4.Name = "carl" txn4.Priority = 123 + txn4.CoordinatorNodeID = 3 txn4.Update(&txn) expTxn4 := txn @@ -570,6 +573,7 @@ func TestTransactionUpdate(t *testing.T) { txn5.Sequence = txn.Sequence - 10 txn5.Name = "carl" txn5.Priority = 123 + txn5.CoordinatorNodeID = 3 txn5.Update(&txn) expTxn5 := txn @@ -2035,7 +2039,7 @@ func TestTxnLocksAsLockUpdates(t *testing.T) { defer leaktest.AfterTest(t)() ts := hlc.Timestamp{WallTime: 1} - txn := MakeTransaction("hello", Key("k"), 0, ts, 0) + txn := MakeTransaction("hello", Key("k"), 0, ts, 0, 99) txn.Status = COMMITTED txn.IgnoredSeqNums = []enginepb.IgnoredSeqNumRange{{Start: 0, End: 0}} diff --git a/pkg/roachpb/errors_test.go b/pkg/roachpb/errors_test.go index 697a90bc0ba5..a7a6ed2ccc0e 100644 --- a/pkg/roachpb/errors_test.go +++ b/pkg/roachpb/errors_test.go @@ -49,7 +49,7 @@ func TestNewErrorNil(t *testing.T) { // TestSetTxn verifies that SetTxn updates the error message. func TestSetTxn(t *testing.T) { e := NewError(NewTransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)) - txn := MakeTransaction("test", Key("a"), 1, hlc.Timestamp{}, 0) + txn := MakeTransaction("test", Key("a"), 1, hlc.Timestamp{}, 0, 99) e.SetTxn(&txn) if !strings.HasPrefix( e.String(), "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND): \"test\"") { @@ -143,7 +143,7 @@ func TestErrorRedaction(t *testing.T) { GlobalUncertaintyLimit: hlc.Timestamp{WallTime: 3}, ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: hlc.ClockTimestamp{WallTime: 4}}}, })) - txn := MakeTransaction("foo", Key("bar"), 1, hlc.Timestamp{WallTime: 1}, 1) + txn := MakeTransaction("foo", Key("bar"), 1, hlc.Timestamp{WallTime: 1}, 1, 99) txn.ID = uuid.Nil txn.Priority = 1234 wrappedPErr.UnexposedTxn = &txn @@ -173,7 +173,7 @@ func TestErrorDeprecatedFields(t *testing.T) { require.Equal(t, TransactionRestart_NONE, pErr.deprecatedTransactionRestart) require.Nil(t, pErr.deprecatedDetail.Value) }) - txn := MakeTransaction("foo", Key("k"), 0, hlc.Timestamp{WallTime: 1}, 50000) + txn := MakeTransaction("foo", Key("k"), 0, hlc.Timestamp{WallTime: 1}, 50000, 99) t.Run("structured-wrapped", func(t *testing.T) { // For extra spice, wrap the structured error. This ensures diff --git a/pkg/roachpb/string_test.go b/pkg/roachpb/string_test.go index a08e7a24b9ff..87854fd38a1c 100644 --- a/pkg/roachpb/string_test.go +++ b/pkg/roachpb/string_test.go @@ -60,10 +60,11 @@ func TestBatchRequestString(t *testing.T) { ba := roachpb.BatchRequest{} txn := roachpb.MakeTransaction( "test", - nil, /* baseKey */ + nil, // baseKey roachpb.NormalUserPriority, hlc.Timestamp{}, // now 0, // maxOffsetNs + 99, // coordinatorNodeID ) txn.ID = uuid.NamespaceDNS ba.Txn = &txn diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 31a7ebe00411..aeb820d71111 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -796,13 +796,15 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { distSQLServer.ServerConfig.SessionBoundInternalExecutorFactory = ieFactory jobRegistry.SetSessionBoundInternalExecutorFactory(ieFactory) execCfg.IndexBackfiller = sql.NewIndexBackfiller(execCfg, ieFactory) - execCfg.IndexValidator = scsqldeps.NewIndexValidator(execCfg.DB, + execCfg.IndexValidator = scsqldeps.NewIndexValidator( + execCfg.DB, execCfg.Codec, execCfg.Settings, ieFactory, sql.ValidateForwardIndexes, sql.ValidateInvertedIndexes, - sql.NewFakeSessionData) + sql.NewFakeSessionData, + ) execCfg.InternalExecutorFactory = ieFactory distSQLServer.ServerConfig.ProtectedTimestampProvider = execCfg.ProtectedTimestampProvider diff --git a/pkg/sql/contention/registry_test.go b/pkg/sql/contention/registry_test.go index 3b5ba8292515..4b20b9ca6bb6 100644 --- a/pkg/sql/contention/registry_test.go +++ b/pkg/sql/contention/registry_test.go @@ -146,7 +146,8 @@ func TestRegistry(t *testing.T) { registry.AddContentionEvent(roachpb.ContentionEvent{ Key: keyBytes, TxnMeta: enginepb.TxnMeta{ - ID: contendingTxnID, + ID: contendingTxnID, + CoordinatorNodeID: 6, }, Duration: time.Duration(contentionDuration), }) @@ -233,8 +234,9 @@ func TestSerializedRegistryInvariants(t *testing.T) { r.AddContentionEvent(roachpb.ContentionEvent{ Key: key, TxnMeta: enginepb.TxnMeta{ - ID: uuid.MakeV4(), - Key: getKey(), + ID: uuid.MakeV4(), + Key: getKey(), + CoordinatorNodeID: 6, }, Duration: time.Duration(int64(rng.Uint64())), }) diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index f5fe964bafa0..65f5708c64df 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -169,6 +169,8 @@ func (p *planner) maybeCreatePublicSchemaWithDescriptor( // In CockroachDB, root is our substitute for the postgres user. publicSchemaPrivileges := descpb.NewBasePrivilegeDescriptor(security.AdminRoleName()) // By default, everyone has USAGE and CREATE on the public schema. + // Once https://github.com/cockroachdb/cockroach/issues/70266 is resolved, + // the public role will no longer have CREATE privileges. publicSchemaPrivileges.Grant(security.PublicRoleName(), privilege.List{privilege.CREATE, privilege.USAGE}, false) publicSchemaDesc := schemadesc.NewBuilder(&descpb.SchemaDescriptor{ ParentID: dbID, diff --git a/pkg/sql/event_log.go b/pkg/sql/event_log.go index 8d8d6da6258d..a882369c7b8d 100644 --- a/pkg/sql/event_log.go +++ b/pkg/sql/event_log.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" @@ -286,8 +287,10 @@ func logEventInternalForSQLStatements( } } - return insertEventRecords(ctx, - execCfg.InternalExecutor, txn, + return insertEventRecords( + ctx, + execCfg.InternalExecutor, + txn, int32(execCfg.NodeID.SQLInstanceID()), /* reporter ID */ 1+depth, /* depth */ opts, /* eventLogOptions */ @@ -295,24 +298,35 @@ func logEventInternalForSQLStatements( ) } -// LogEventForSchemaChanger allows then declarative schema changer -// to generate event log entries with context information available -// inside that package. -func LogEventForSchemaChanger( - ctx context.Context, - execCfg interface{}, - txn *kv.Txn, - depth int, - descID descpb.ID, - metadata scpb.ElementMetadata, - event eventpb.EventPayload, +type schemaChangerEventLogger struct { + txn *kv.Txn + execCfg *ExecutorConfig + depth int +} + +var _ scexec.EventLogger = (*schemaChangerEventLogger)(nil) + +// NewSchemaChangerEventLogger returns a scexec.EventLogger implementation. +func NewSchemaChangerEventLogger( + txn *kv.Txn, execCfg *ExecutorConfig, depth int, +) scexec.EventLogger { + return &schemaChangerEventLogger{ + txn: txn, + execCfg: execCfg, + depth: depth, + } +} + +// LogEvent implements the scexec.EventLogger interface. +func (l schemaChangerEventLogger) LogEvent( + ctx context.Context, descID descpb.ID, metadata scpb.ElementMetadata, event eventpb.EventPayload, ) error { entry := eventLogEntry{targetID: int32(descID), event: event} commonPayload := makeCommonSQLEventDetails(metadata.Username, metadata.Statement, metadata.AppName) return logEventInternalForSQLStatements(ctx, - execCfg.(*ExecutorConfig), - txn, - depth, + l.execCfg, + l.txn, + l.depth, eventLogOptions{dst: LogEverywhere}, *commonPayload, entry) diff --git a/pkg/sql/flowinfra/cluster_test.go b/pkg/sql/flowinfra/cluster_test.go index 52d0e44e999e..236ddeafe17c 100644 --- a/pkg/sql/flowinfra/cluster_test.go +++ b/pkg/sql/flowinfra/cluster_test.go @@ -97,7 +97,8 @@ func TestClusterFlow(t *testing.T) { nil, // baseKey roachpb.NormalUserPriority, now.ToTimestamp(), - 0, // maxOffset + 0, // maxOffsetNs + int32(tc.Server(0).SQLInstanceID()), ) txn := kv.NewTxnFromProto(ctx, kvDB, tc.Server(0).NodeID(), now, kv.RootTxn, &txnProto) leafInputState := txn.GetLeafTxnInputState(ctx) @@ -417,7 +418,8 @@ func TestLimitedBufferingDeadlock(t *testing.T) { nil, // baseKey roachpb.NormalUserPriority, now.ToTimestamp(), - 0, // maxOffset + 0, // maxOffsetNs + int32(tc.Server(0).SQLInstanceID()), ) txn := kv.NewTxnFromProto( context.Background(), tc.Server(0).DB(), tc.Server(0).NodeID(), @@ -708,7 +710,8 @@ func BenchmarkInfrastructure(b *testing.B) { nil, // baseKey roachpb.NormalUserPriority, now.ToTimestamp(), - 0, // maxOffset + 0, // maxOffsetNs + int32(tc.Server(0).SQLInstanceID()), ) txn := kv.NewTxnFromProto( context.Background(), tc.Server(0).DB(), tc.Server(0).NodeID(), diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 802040b9d50b..13dad79006ce 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -282,8 +282,11 @@ import ( // - sleep // Introduces a sleep period. Example: sleep 2s // -// - user +// - user [nodeidx=N] // Changes the user for subsequent statements or queries. +// If nodeidx is specified, this user will connect to the node +// in the cluster with index N (note this is 0-indexed, while +// node IDs themselves are 1-indexed). // // - skipif // Skips the following `statement` or `query` if the argument is @@ -1329,7 +1332,7 @@ func (t *logicTest) outf(format string, args ...interface{}) { // setUser sets the DB client to the specified user. // It returns a cleanup function to be run when the credentials // are no longer needed. -func (t *logicTest) setUser(user string) func() { +func (t *logicTest) setUser(user string, nodeIdxOverride int) func() { if t.clients == nil { t.clients = map[string]*gosql.DB{} } @@ -1341,9 +1344,14 @@ func (t *logicTest) setUser(user string) func() { return func() {} } - addr := t.cluster.Server(t.nodeIdx).ServingSQLAddr() + nodeIdx := t.nodeIdx + if nodeIdxOverride > 0 { + nodeIdx = nodeIdxOverride + } + + addr := t.cluster.Server(nodeIdx).ServingSQLAddr() if len(t.tenantAddrs) > 0 { - addr = t.tenantAddrs[t.nodeIdx] + addr = t.tenantAddrs[nodeIdx] } pgURL, cleanupFunc := sqlutils.PGUrl(t.rootT, addr, "TestLogic", url.User(user)) pgURL.Path = "test" @@ -1671,7 +1679,7 @@ func (t *logicTest) newCluster(serverArgs TestServerArgs, opts []clusterOpt) { // db may change over the lifetime of this function, with intermediate // values cached in t.clients and finally closed in t.close(). - t.clusterCleanupFuncs = append(t.clusterCleanupFuncs, t.setUser(security.RootUser)) + t.clusterCleanupFuncs = append(t.clusterCleanupFuncs, t.setUser(security.RootUser, 0 /* nodeIdxOverride */)) } // shutdownCluster performs the necessary cleanup to shutdown the current test @@ -2474,13 +2482,23 @@ func (t *logicTest) processSubtest(subtest subtestDetails, path string) error { case "halt", "hash-threshold": case "user": + var nodeIdx int if len(fields) < 2 { return errors.Errorf("user command requires one argument, found: %v", fields) } if len(fields[1]) == 0 { return errors.Errorf("user command requires a non-blank argument") } - cleanupUserFunc := t.setUser(fields[1]) + if len(fields) >= 3 { + if strings.HasPrefix(fields[2], "nodeidx=") { + idx, err := strconv.ParseInt(strings.SplitN(fields[2], "=", 2)[1], 10, 64) + if err != nil { + return errors.Wrapf(err, "error parsing nodeidx") + } + nodeIdx = int(idx) + } + } + cleanupUserFunc := t.setUser(fields[1], nodeIdx) defer cleanupUserFunc() case "skip": @@ -3191,7 +3209,7 @@ func (t *logicTest) validateAfterTestCompletion() error { t.Fatalf("failed to close connection for user %s: %v", username, err) } } - t.setUser("root") + t.setUser("root", 0 /* nodeIdxOverride */) // Some cleanup to make sure the following validation queries can run // successfully. First we rollback in case the logic test had an uncommitted diff --git a/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_schema b/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_schema index c6bd6ad5ac2d..b4a975acfc74 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_schema +++ b/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_schema @@ -8,6 +8,18 @@ user testuser statement ok USE d; +# The public schema is special and has hard-coded privileges for the public role. +# When https://github.com/cockroachdb/cockroach/issues/70266 is resolved, +# the public role will no longer have CREATE privilege. +query TTTT colnames +SHOW GRANTS ON SCHEMA public +---- +database_name schema_name grantee privilege_type +d public admin ALL +d public public CREATE +d public public USAGE +d public root ALL + statement ok CREATE SCHEMA testuser_s; diff --git a/pkg/sql/logictest/testdata/logic_test/contention_event b/pkg/sql/logictest/testdata/logic_test/contention_event index 2ca7cae12ca3..0903e67e48b8 100644 --- a/pkg/sql/logictest/testdata/logic_test/contention_event +++ b/pkg/sql/logictest/testdata/logic_test/contention_event @@ -16,7 +16,7 @@ query TT SELECT * FROM kv ---- -user testuser +user testuser nodeidx=3 statement ok BEGIN @@ -38,7 +38,7 @@ BEGIN; SET TRANSACTION PRIORITY HIGH; SELECT * FROM kv ORDER BY k ASC -user testuser +user testuser nodeidx=3 statement ok ROLLBACK @@ -77,6 +77,19 @@ WITH payloads AS ( ---- true +# check that the coordinator node id that initiates the transaction is properly stored with the contention event +query B +WITH payloads AS ( + SELECT * + FROM crdb_internal.payloads_for_trace(crdb_internal.trace_id()) +) SELECT count(*) > 0 + FROM payloads + WHERE payload_type = 'roachpb.ContentionEvent' + AND crdb_internal.pretty_key(decode(payload_jsonb->>'key', 'base64'), 1) LIKE '/1/"k"/%' + AND (payload_jsonb->'txnMeta'->>'coordinatorNodeId')::INTEGER = 4 +---- +true + # Check that there is at least 1 contention event for kv table in the contention # virtual tables. query B diff --git a/pkg/sql/schema_change_plan_node.go b/pkg/sql/schema_change_plan_node.go index a19e7e3fcc00..3ffd0f7f75e8 100644 --- a/pkg/sql/schema_change_plan_node.go +++ b/pkg/sql/schema_change_plan_node.go @@ -30,7 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/retry" ) @@ -164,10 +163,8 @@ func newSchemaChangerTxnRunDependencies( execCfg.JobRegistry, execCfg.IndexBackfiller, execCfg.IndexValidator, - scsqldeps.NewCCLCallbacks(execCfg.Settings, evalContext), - func(ctx context.Context, txn *kv.Txn, depth int, descID descpb.ID, metadata scpb.ElementMetadata, event eventpb.EventPayload) error { - return LogEventForSchemaChanger(ctx, execCfg, txn, depth+1, descID, metadata, event) - }, + scsqldeps.NewPartitioner(execCfg.Settings, evalContext), + NewSchemaChangerEventLogger(txn, execCfg, 1), stmts, ) } diff --git a/pkg/sql/schemachanger/scdeps/BUILD.bazel b/pkg/sql/schemachanger/scdeps/BUILD.bazel index efc5026f32fc..ab525bd4c103 100644 --- a/pkg/sql/schemachanger/scdeps/BUILD.bazel +++ b/pkg/sql/schemachanger/scdeps/BUILD.bazel @@ -22,16 +22,14 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/resolver", - "//pkg/sql/catalog/tabledesc", "//pkg/sql/schemachanger/scbuild", "//pkg/sql/schemachanger/scexec", - "//pkg/sql/schemachanger/scpb", + "//pkg/sql/schemachanger/scexec/scmutationexec", "//pkg/sql/schemachanger/scrun", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/sqlutil", "//pkg/sql/types", - "//pkg/util/log/eventpb", "@com_github_cockroachdb_errors//:errors", "@com_github_lib_pq//oid", ], diff --git a/pkg/sql/schemachanger/scdeps/exec_deps.go b/pkg/sql/schemachanger/scdeps/exec_deps.go index 37b49dcfddb9..2fddfc6c56f3 100644 --- a/pkg/sql/schemachanger/scdeps/exec_deps.go +++ b/pkg/sql/schemachanger/scdeps/exec_deps.go @@ -13,7 +13,6 @@ package scdeps import ( "context" "fmt" - "sort" "time" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -26,11 +25,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/errors" ) @@ -54,8 +51,8 @@ func NewExecutorDependencies( jobRegistry JobRegistry, indexBackfiller scexec.IndexBackfiller, indexValidator scexec.IndexValidator, - cclCallbacks scexec.Partitioner, - logEventFn LogEventCallback, + partitioner scmutationexec.Partitioner, + eventLogger scexec.EventLogger, statements []string, ) scexec.Dependencies { return &execDeps{ @@ -65,11 +62,11 @@ func NewExecutorDependencies( descsCollection: descsCollection, jobRegistry: jobRegistry, indexValidator: indexValidator, - partitioner: cclCallbacks, - eventLogWriter: newEventLogWriter(txn, logEventFn), + eventLogger: eventLogger, }, indexBackfiller: indexBackfiller, statements: statements, + partitioner: partitioner, user: user, } } @@ -80,8 +77,7 @@ type txnDeps struct { descsCollection *descs.Collection jobRegistry JobRegistry indexValidator scexec.IndexValidator - partitioner scexec.Partitioner - eventLogWriter *eventLogWriter + eventLogger scexec.EventLogger deletedDescriptors catalog.DescriptorIDSet } @@ -177,28 +173,6 @@ func (d *txnDeps) RemoveSyntheticDescriptor(id descpb.ID) { d.descsCollection.RemoveSyntheticDescriptor(id) } -// AddPartitioning implements the scmutationexec.CatalogReader interface. -func (d *txnDeps) AddPartitioning( - tableDesc *tabledesc.Mutable, - indexDesc *descpb.IndexDescriptor, - partitionFields []string, - listPartition []*scpb.ListPartition, - rangePartition []*scpb.RangePartitions, - allowedNewColumnNames []tree.Name, - allowImplicitPartitioning bool, -) error { - ctx := context.Background() - - return d.partitioner.AddPartitioning(ctx, - tableDesc, - indexDesc, - partitionFields, - listPartition, - rangePartition, - allowedNewColumnNames, - allowImplicitPartitioning) -} - // MustReadMutableDescriptor implements the scexec.Catalog interface. func (d *txnDeps) MustReadMutableDescriptor( ctx context.Context, id descpb.ID, @@ -311,6 +285,7 @@ func (d *txnDeps) SetResumeSpans( type execDeps struct { txnDeps indexBackfiller scexec.IndexBackfiller + partitioner scmutationexec.Partitioner statements []string user security.SQLUsername } @@ -322,6 +297,11 @@ func (d *execDeps) Catalog() scexec.Catalog { return d } +// Partitioner implements the scexec.Dependencies interface. +func (d *execDeps) Partitioner() scmutationexec.Partitioner { + return d.partitioner +} + // IndexBackfiller implements the scexec.Dependencies interface. func (d *execDeps) IndexBackfiller() scexec.IndexBackfiller { return d.indexBackfiller @@ -356,134 +336,7 @@ func (d *execDeps) User() security.SQLUsername { return d.user } -// LogEventCallback call back to allow the new schema changer -// to generate event log entries. -type LogEventCallback func(ctx context.Context, - txn *kv.Txn, - depth int, - descID descpb.ID, - metadata scpb.ElementMetadata, - event eventpb.EventPayload, -) error - -type eventPayload struct { - descID descpb.ID - metadata *scpb.ElementMetadata - event eventpb.EventPayload -} - -type eventLogWriter struct { - txn *kv.Txn - logEvent LogEventCallback - eventStatementMap map[uint32][]eventPayload -} - -// newEventLogWriter makes a new event log writer which will accumulate, -// and emit events. -func newEventLogWriter(txn *kv.Txn, logEvent LogEventCallback) *eventLogWriter { - return &eventLogWriter{ - txn: txn, - logEvent: logEvent, - eventStatementMap: make(map[uint32][]eventPayload), - } -} - -// EnqueueEvent implements scexec.EventLogger -func (m *eventLogWriter) EnqueueEvent( - _ context.Context, descID descpb.ID, metadata *scpb.ElementMetadata, event eventpb.EventPayload, -) error { - eventList := m.eventStatementMap[metadata.StatementID] - m.eventStatementMap[metadata.StatementID] = append(eventList, - eventPayload{descID: descID, - event: event, - metadata: metadata}, - ) - return nil -} - -// ProcessAndSubmitEvents implements scexec.EventLogger -func (m *eventLogWriter) ProcessAndSubmitEvents(ctx context.Context) error { - for _, events := range m.eventStatementMap { - // A dependent event is one which is generated because of a - // dependency getting modified from the source object. An example - // of this is a DROP TABLE will be the source event, which will track - // any dependent views dropped. - var dependentEvents = make(map[uint32][]eventPayload) - var sourceEvents = make(map[uint32]eventPayload) - // First separate out events, where the first event generated will always - // be the source and everything else before will be dependencies if they have - // the same subtask ID. - for _, event := range events { - dependentEvents[event.metadata.SubWorkID] = append(dependentEvents[event.metadata.SubWorkID], event) - } - // Split of the source events. - orderedSubWorkID := make([]uint32, 0, len(dependentEvents)) - for subWorkID := range dependentEvents { - elems := dependentEvents[subWorkID] - sort.SliceStable(elems, func(i, j int) bool { - return elems[i].metadata.SourceElementID < elems[j].metadata.SourceElementID - }) - sourceEvents[subWorkID] = elems[0] - dependentEvents[subWorkID] = elems[1:] - orderedSubWorkID = append(orderedSubWorkID, subWorkID) - } - // Store an ordered list of sub-work IDs for deterministic - // event order. - sort.SliceStable(orderedSubWorkID, func(i, j int) bool { - return orderedSubWorkID[i] < orderedSubWorkID[j] - }) - // Collect the dependent objects for each - // source event, and generate an event log entry. - for _, subWorkID := range orderedSubWorkID { - // Determine which objects we should collect. - collectDependentViewNames := false - collectDependentSchemaNames := false - sourceEvent := sourceEvents[subWorkID] - switch sourceEvent.event.(type) { - case *eventpb.DropDatabase: - // Drop database only reports dependent schemas. - collectDependentSchemaNames = true - case *eventpb.DropView, *eventpb.DropTable: - // Drop view and drop tables only cares about - // dependent views - collectDependentViewNames = true - } - var dependentObjects []string - for _, dependentEvent := range dependentEvents[subWorkID] { - switch ev := dependentEvent.event.(type) { - case *eventpb.DropView: - if collectDependentViewNames { - dependentObjects = append(dependentObjects, ev.ViewName) - } - case *eventpb.DropSchema: - if collectDependentSchemaNames { - dependentObjects = append(dependentObjects, ev.SchemaName) - } - } - } - // Add anything that we determined based - // on the dependencies. - switch ev := sourceEvent.event.(type) { - case *eventpb.DropTable: - ev.CascadeDroppedViews = dependentObjects - case *eventpb.DropView: - ev.CascadeDroppedViews = dependentObjects - case *eventpb.DropDatabase: - ev.DroppedSchemaObjects = dependentObjects - } - // Generate event log entries for the source event only. The dependent - // events will be ignored. - if m.logEvent != nil { - err := m.logEvent(ctx, m.txn, 0, sourceEvent.descID, *sourceEvent.metadata, sourceEvent.event) - if err != nil { - return err - } - } - } - } - return nil -} - +// EventLogger implements scexec.Dependencies func (d *execDeps) EventLogger() scexec.EventLogger { - return d.eventLogWriter + return d.eventLogger } diff --git a/pkg/sql/schemachanger/scdeps/run_deps.go b/pkg/sql/schemachanger/scdeps/run_deps.go index eff880e682fe..980d80aefe1a 100644 --- a/pkg/sql/schemachanger/scdeps/run_deps.go +++ b/pkg/sql/schemachanger/scdeps/run_deps.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" ) @@ -30,44 +31,44 @@ func NewJobRunDependencies( db *kv.DB, internalExecutor sqlutil.InternalExecutor, indexBackfiller scexec.IndexBackfiller, - logEventFn LogEventCallback, + eventLoggerBuilder func(txn *kv.Txn) scexec.EventLogger, + partitioner scmutationexec.Partitioner, jobRegistry *jobs.Registry, job *jobs.Job, codec keys.SQLCodec, settings *cluster.Settings, indexValidator scexec.IndexValidator, - cclCallbacks scexec.Partitioner, testingKnobs *scrun.TestingKnobs, statements []string, ) scrun.JobRunDependencies { return &jobExecutionDeps{ - collectionFactory: collectionFactory, - db: db, - internalExecutor: internalExecutor, - indexBackfiller: indexBackfiller, - logEventFn: logEventFn, - jobRegistry: jobRegistry, - job: job, - codec: codec, - settings: settings, - testingKnobs: testingKnobs, - statements: statements, - indexValidator: indexValidator, - partitioner: cclCallbacks, + collectionFactory: collectionFactory, + db: db, + internalExecutor: internalExecutor, + indexBackfiller: indexBackfiller, + eventLoggerBuilder: eventLoggerBuilder, + partitioner: partitioner, + jobRegistry: jobRegistry, + job: job, + codec: codec, + settings: settings, + testingKnobs: testingKnobs, + statements: statements, + indexValidator: indexValidator, } } type jobExecutionDeps struct { - collectionFactory *descs.CollectionFactory - db *kv.DB - internalExecutor sqlutil.InternalExecutor - indexBackfiller scexec.IndexBackfiller - logEventFn LogEventCallback - jobRegistry *jobs.Registry - job *jobs.Job + collectionFactory *descs.CollectionFactory + db *kv.DB + internalExecutor sqlutil.InternalExecutor + indexBackfiller scexec.IndexBackfiller + eventLoggerBuilder func(txn *kv.Txn) scexec.EventLogger + partitioner scmutationexec.Partitioner + jobRegistry *jobs.Registry + job *jobs.Job indexValidator scexec.IndexValidator - partitioner scexec.Partitioner codec keys.SQLCodec settings *cluster.Settings @@ -94,11 +95,11 @@ func (d *jobExecutionDeps) WithTxnInJob(ctx context.Context, fn scrun.JobTxnFunc descsCollection: descriptors, jobRegistry: d.jobRegistry, indexValidator: d.indexValidator, - partitioner: d.partitioner, - eventLogWriter: newEventLogWriter(txn, d.logEventFn), + eventLogger: d.eventLoggerBuilder(txn), }, indexBackfiller: d.indexBackfiller, statements: d.statements, + partitioner: d.partitioner, user: d.job.Payload().UsernameProto.Decode(), }) }) diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/BUILD.bazel b/pkg/sql/schemachanger/scdeps/sctestdeps/BUILD.bazel index b092744ee6fa..f7e132316ae3 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/BUILD.bazel +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/BUILD.bazel @@ -26,7 +26,6 @@ go_library( "//pkg/sql/catalog/schemadesc", "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", - "//pkg/sql/parser", "//pkg/sql/privilege", "//pkg/sql/schemachanger/scbuild", "//pkg/sql/schemachanger/scdeps/sctestutils", diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index 4427870c0b02..21da0bd25833 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -27,7 +27,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdeps/sctestutils" @@ -354,7 +353,6 @@ var _ catalog.TypeDescriptorResolver = (*TestState)(nil) func (s *TestState) GetTypeDescriptor( ctx context.Context, id descpb.ID, ) (tree.TypeName, catalog.TypeDescriptor, error) { - desc, err := s.mustReadImmutableDescriptor(id) if err != nil { return tree.TypeName{}, nil, err @@ -461,59 +459,6 @@ func (s *TestState) RemoveSyntheticDescriptor(id descpb.ID) { s.syntheticDescriptors.Remove(id) } -// AddPartitioning implements the scmutationexec.CatalogReader interface. -func (s *TestState) AddPartitioning( - tableDesc *tabledesc.Mutable, - _ *descpb.IndexDescriptor, - partitionFields []string, - listPartition []*scpb.ListPartition, - rangePartition []*scpb.RangePartitions, - _ []tree.Name, - _ bool, -) error { - // Deserialize back into tree based types. - partitionBy := &tree.PartitionBy{} - partitionBy.List = make([]tree.ListPartition, 0, len(listPartition)) - partitionBy.Range = make([]tree.RangePartition, 0, len(rangePartition)) - for _, partition := range listPartition { - exprs, err := parser.ParseExprs(partition.Expr) - if err != nil { - return err - } - partitionBy.List = append(partitionBy.List, - tree.ListPartition{ - Name: tree.UnrestrictedName(partition.Name), - Exprs: exprs, - }) - } - for _, partition := range rangePartition { - toExpr, err := parser.ParseExprs(partition.To) - if err != nil { - return err - } - fromExpr, err := parser.ParseExprs(partition.From) - if err != nil { - return err - } - partitionBy.Range = append(partitionBy.Range, - tree.RangePartition{ - Name: tree.UnrestrictedName(partition.Name), - To: toExpr, - From: fromExpr, - }) - } - partitionBy.Fields = make(tree.NameList, 0, len(partitionFields)) - for _, field := range partitionFields { - partitionBy.Fields = append(partitionBy.Fields, tree.Name(field)) - } - // For the purpose of testing we will only track - // these values. - s.partitioningInfo[tableDesc.GetID()] = &testPartitionInfo{ - PartitionBy: *partitionBy, - } - return nil -} - var _ scexec.Catalog = (*TestState)(nil) // MustReadMutableDescriptor implements the scexec.Catalog interface. @@ -654,7 +599,7 @@ func (b *testCatalogChangeBatcher) ValidateAndRun(ctx context.Context) error { b.s.descriptors.Upsert(desc) } for _, deletedID := range b.descriptorsToDelete.Ordered() { - b.s.LogSideEffectf("deleted descriptor #%d", deletedID) + b.s.LogSideEffectf("delete descriptor #%d", deletedID) b.s.descriptors.Remove(deletedID) } return catalog.Validate(ctx, b.s, catalog.NoValidationTelemetry, catalog.ValidationLevelAllPreTxnCommit, b.descs...).CombinedError() @@ -684,6 +629,28 @@ func (s *TestState) GetNamespaceEntry( return s.namespace[nameInfo], nil } +// Partitioner implements the scexec.Dependencies interface. +func (s *TestState) Partitioner() scmutationexec.Partitioner { + return s +} + +// AddPartitioning implements the scmutationexec.Partitioner interface. +func (s *TestState) AddPartitioning( + _ context.Context, + tbl *tabledesc.Mutable, + index catalog.Index, + _ []string, + _ []*scpb.ListPartition, + _ []*scpb.RangePartitions, + _ []tree.Name, + _ bool, +) error { + s.LogSideEffectf("skip partitioning index #%d in table #%d", index.GetID(), tbl.GetID()) + return nil +} + +var _ scmutationexec.Partitioner = (*TestState)(nil) + // IndexBackfiller implements the scexec.Dependencies interface. func (s *TestState) IndexBackfiller() scexec.IndexBackfiller { return s @@ -695,10 +662,11 @@ var _ scexec.IndexBackfiller = (*TestState)(nil) func (s *TestState) BackfillIndex( _ context.Context, _ scexec.JobProgressTracker, - _ catalog.TableDescriptor, - _ descpb.IndexID, - _ ...descpb.IndexID, + tbl catalog.TableDescriptor, + source descpb.IndexID, + destinations ...descpb.IndexID, ) error { + s.LogSideEffectf("backfill indexes %v from index #%d in table #%d", destinations, source, tbl.GetID()) return nil } @@ -847,23 +815,33 @@ func (s *TestState) WithTxnInJob(ctx context.Context, fn scrun.JobTxnFunc) (err // ValidateForwardIndexes implements the index validator interface. func (s *TestState) ValidateForwardIndexes( ctx context.Context, - tableDesc catalog.TableDescriptor, + tbl catalog.TableDescriptor, indexes []catalog.Index, withFirstMutationPublic bool, gatherAllInvalid bool, override sessiondata.InternalExecutorOverride, ) error { + ids := make([]descpb.IndexID, len(indexes)) + for i, idx := range indexes { + ids[i] = idx.GetID() + } + s.LogSideEffectf("validate forward indexes %v in table #%d", ids, tbl.GetID()) return nil } // ValidateInvertedIndexes implements the index validator interface. func (s *TestState) ValidateInvertedIndexes( ctx context.Context, - tableDesc catalog.TableDescriptor, + tbl catalog.TableDescriptor, indexes []catalog.Index, gatherAllInvalid bool, override sessiondata.InternalExecutorOverride, ) error { + ids := make([]descpb.IndexID, len(indexes)) + for i, idx := range indexes { + ids[i] = idx.GetID() + } + s.LogSideEffectf("validate inverted indexes %v in table #%d", ids, tbl.GetID()) return nil } @@ -872,15 +850,12 @@ func (s *TestState) IndexValidator() scexec.IndexValidator { return s } -// EnqueueEvent implements scexec.EventLogger -func (s *TestState) EnqueueEvent( - _ context.Context, descID descpb.ID, metadata *scpb.ElementMetadata, event eventpb.EventPayload, +// LogEvent implements scexec.EventLogger +func (s *TestState) LogEvent( + _ context.Context, descID descpb.ID, metadata scpb.ElementMetadata, event eventpb.EventPayload, ) error { - return nil -} - -// ProcessAndSubmitEvents implements scexec.EventLogger -func (s *TestState) ProcessAndSubmitEvents(ctx context.Context) error { + s.LogSideEffectf("write %T to event log for descriptor #%d: %s", + event, descID, metadata.Statement) return nil } diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go index d87de2434bcd..31e8c72eb40f 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go @@ -20,22 +20,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" ) -// testPartitionInfo tracks partitioning information -// for testing -type testPartitionInfo struct { - tree.PartitionBy -} - // TestState is a backing struct used to implement all schema changer // dependencies, like scbuild.Dependencies or scexec.Dependencies, for the // purpose of facilitating end-to-end testing of the declarative schema changer. type TestState struct { descriptors, syntheticDescriptors nstree.Map - partitioningInfo map[descpb.ID]*testPartitionInfo namespace map[descpb.NameInfo]descpb.ID currentDatabase string phase scop.Phase diff --git a/pkg/sql/schemachanger/scexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/BUILD.bazel index cbeca0e7bf93..04bee2f878ec 100644 --- a/pkg/sql/schemachanger/scexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/BUILD.bazel @@ -19,14 +19,13 @@ go_library( "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/nstree", - "//pkg/sql/catalog/tabledesc", "//pkg/sql/schemachanger/scexec/descriptorutils", "//pkg/sql/schemachanger/scexec/scmutationexec", "//pkg/sql/schemachanger/scop", "//pkg/sql/schemachanger/scpb", - "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/util/log", + "//pkg/util/log/eventpb", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", ], @@ -59,6 +58,7 @@ go_test( "//pkg/sql/schemachanger/scbuild", "//pkg/sql/schemachanger/scdeps", "//pkg/sql/schemachanger/scdeps/sctestutils", + "//pkg/sql/schemachanger/scexec/scmutationexec", "//pkg/sql/schemachanger/scgraphviz", "//pkg/sql/schemachanger/scop", "//pkg/sql/schemachanger/scpb", diff --git a/pkg/sql/schemachanger/scexec/dependencies.go b/pkg/sql/schemachanger/scexec/dependencies.go index ee95026a7231..0fa4bbb6b9fa 100644 --- a/pkg/sql/schemachanger/scexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/dependencies.go @@ -19,16 +19,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" ) // Dependencies contains all the dependencies required by the executor. type Dependencies interface { Catalog() Catalog + Partitioner() scmutationexec.Partitioner TransactionalJobCreator() TransactionalJobCreator IndexBackfiller() IndexBackfiller IndexValidator() IndexValidator @@ -57,11 +57,10 @@ type Catalog interface { NewCatalogChangeBatcher() CatalogChangeBatcher } -// EventLogger encapsulates the operations for collecting -// and emitting event log entries. +// EventLogger encapsulates the operations for emitting event log entries. type EventLogger interface { - scmutationexec.EventLogWriter - ProcessAndSubmitEvents(ctx context.Context) error + // LogEvent writes to the eventlog. + LogEvent(ctx context.Context, descID descpb.ID, metadata scpb.ElementMetadata, event eventpb.EventPayload) error } // CatalogChangeBatcher encapsulates batched updates to the catalog: descriptor @@ -113,26 +112,11 @@ type IndexBackfiller interface { ) error } -// Partitioner provides an interface that implements CCL exclusive -// callbacks. -type Partitioner interface { - AddPartitioning( - ctx context.Context, - tableDesc *tabledesc.Mutable, - indexDesc *descpb.IndexDescriptor, - partitionFields []string, - listPartition []*scpb.ListPartition, - rangePartition []*scpb.RangePartitions, - allowedNewColumnNames []tree.Name, - allowImplicitPartitioning bool, - ) (err error) -} - // IndexValidator provides interfaces that allow indexes to be validated. type IndexValidator interface { ValidateForwardIndexes( ctx context.Context, - tableDesc catalog.TableDescriptor, + tbl catalog.TableDescriptor, indexes []catalog.Index, withFirstMutationPublic bool, gatherAllInvalid bool, @@ -141,7 +125,7 @@ type IndexValidator interface { ValidateInvertedIndexes( ctx context.Context, - tableDesc catalog.TableDescriptor, + tbl catalog.TableDescriptor, indexes []catalog.Index, gatherAllInvalid bool, override sessiondata.InternalExecutorOverride, diff --git a/pkg/sql/schemachanger/scexec/exec_mutation.go b/pkg/sql/schemachanger/scexec/exec_mutation.go index 5da1154a6089..78e9df31583d 100644 --- a/pkg/sql/schemachanger/scexec/exec_mutation.go +++ b/pkg/sql/schemachanger/scexec/exec_mutation.go @@ -13,6 +13,7 @@ package scexec import ( "context" "fmt" + "sort" "strings" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -24,13 +25,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) func executeDescriptorMutationOps(ctx context.Context, deps Dependencies, ops []scop.Op) error { mvs := newMutationVisitorState(deps.Catalog()) - v := scmutationexec.NewMutationVisitor(deps.Catalog(), mvs, deps.EventLogger()) + v := scmutationexec.NewMutationVisitor(deps.Catalog(), mvs, deps.Partitioner()) for _, op := range ops { if err := op.(scop.MutationOp).Visit(ctx, v); err != nil { return err @@ -118,8 +120,21 @@ func executeDescriptorMutationOps(ctx context.Context, deps Dependencies, ops [] } } - if err := deps.EventLogger().ProcessAndSubmitEvents(ctx); err != nil { - return err + statementIDs := make([]uint32, 0, len(mvs.eventsByStatement)) + for statementID := range mvs.eventsByStatement { + statementIDs = append(statementIDs, statementID) + } + sort.Slice(statementIDs, func(i, j int) bool { + return statementIDs[i] < statementIDs[j] + }) + for _, statementID := range statementIDs { + entries := eventLogEntriesForStatement(mvs.eventsByStatement[statementID]) + for _, e := range entries { + // TODO(postamar): batch these + if err := deps.EventLogger().LogEvent(ctx, e.id, *e.metadata, e.event); err != nil { + return err + } + } } for _, id := range mvs.descriptorsToDelete.Ordered() { @@ -142,6 +157,81 @@ func executeDescriptorMutationOps(ctx context.Context, deps Dependencies, ops [] return b.ValidateAndRun(ctx) } +func eventLogEntriesForStatement(statementEvents []eventPayload) (logEntries []eventPayload) { + // A dependent event is one which is generated because of a + // dependency getting modified from the source object. An example + // of this is a DROP TABLE will be the source event, which will track + // any dependent views dropped. + var dependentEvents = make(map[uint32][]eventPayload) + var sourceEvents = make(map[uint32]eventPayload) + // First separate out events, where the first event generated will always + // be the source and everything else before will be dependencies if they have + // the same subtask ID. + for _, event := range statementEvents { + dependentEvents[event.metadata.SubWorkID] = append(dependentEvents[event.metadata.SubWorkID], event) + } + // Split of the source events. + orderedSubWorkID := make([]uint32, 0, len(dependentEvents)) + for subWorkID := range dependentEvents { + elems := dependentEvents[subWorkID] + sort.SliceStable(elems, func(i, j int) bool { + return elems[i].metadata.SourceElementID < elems[j].metadata.SourceElementID + }) + sourceEvents[subWorkID] = elems[0] + dependentEvents[subWorkID] = elems[1:] + orderedSubWorkID = append(orderedSubWorkID, subWorkID) + } + // Store an ordered list of sub-work IDs for deterministic + // event order. + sort.SliceStable(orderedSubWorkID, func(i, j int) bool { + return orderedSubWorkID[i] < orderedSubWorkID[j] + }) + // Collect the dependent objects for each + // source event, and generate an event log entry. + for _, subWorkID := range orderedSubWorkID { + // Determine which objects we should collect. + collectDependentViewNames := false + collectDependentSchemaNames := false + sourceEvent := sourceEvents[subWorkID] + switch sourceEvent.event.(type) { + case *eventpb.DropDatabase: + // Drop database only reports dependent schemas. + collectDependentSchemaNames = true + case *eventpb.DropView, *eventpb.DropTable: + // Drop view and drop tables only cares about + // dependent views + collectDependentViewNames = true + } + var dependentObjects []string + for _, dependentEvent := range dependentEvents[subWorkID] { + switch ev := dependentEvent.event.(type) { + case *eventpb.DropView: + if collectDependentViewNames { + dependentObjects = append(dependentObjects, ev.ViewName) + } + case *eventpb.DropSchema: + if collectDependentSchemaNames { + dependentObjects = append(dependentObjects, ev.SchemaName) + } + } + } + // Add anything that we determined based + // on the dependencies. + switch ev := sourceEvent.event.(type) { + case *eventpb.DropTable: + ev.CascadeDroppedViews = dependentObjects + case *eventpb.DropView: + ev.CascadeDroppedViews = dependentObjects + case *eventpb.DropDatabase: + ev.DroppedSchemaObjects = dependentObjects + } + // Generate event log entries for the source event only. The dependent + // events will be ignored. + logEntries = append(logEntries, sourceEvent) + } + return logEntries +} + type mutationVisitorState struct { c Catalog checkedOutDescriptors nstree.Map @@ -152,6 +242,13 @@ type mutationVisitorState struct { indexGCJobs map[descpb.ID][]jobspb.SchemaChangeGCDetails_DroppedIndex schemaChangerJob *jobs.Record schemaChangerStatusUpdates map[jobspb.JobID][]scpb.Status + eventsByStatement map[uint32][]eventPayload +} + +type eventPayload struct { + id descpb.ID + metadata *scpb.ElementMetadata + event eventpb.EventPayload } func (mvs *mutationVisitorState) UpdateSchemaChangerJobProgress( @@ -168,10 +265,11 @@ func (mvs *mutationVisitorState) UpdateSchemaChangerJobProgress( func newMutationVisitorState(c Catalog) *mutationVisitorState { return &mutationVisitorState{ - c: c, - drainedNames: make(map[descpb.ID][]descpb.NameInfo), - indexGCJobs: make(map[descpb.ID][]jobspb.SchemaChangeGCDetails_DroppedIndex), - descriptorGCJobs: make(map[descpb.ID][]jobspb.SchemaChangeGCDetails_DroppedID), + c: c, + drainedNames: make(map[descpb.ID][]descpb.NameInfo), + indexGCJobs: make(map[descpb.ID][]jobspb.SchemaChangeGCDetails_DroppedIndex), + descriptorGCJobs: make(map[descpb.ID][]jobspb.SchemaChangeGCDetails_DroppedID), + eventsByStatement: make(map[uint32][]eventPayload), } } @@ -261,3 +359,19 @@ func createGCJobRecord( NonCancelable: true, } } + +// EnqueueEvent implements the scmutationexec.MutationVisitorStateUpdater +// interface. +func (mvs *mutationVisitorState) EnqueueEvent( + id descpb.ID, metadata *scpb.ElementMetadata, event eventpb.EventPayload, +) error { + mvs.eventsByStatement[metadata.StatementID] = append( + mvs.eventsByStatement[metadata.StatementID], + eventPayload{ + id: id, + event: event, + metadata: metadata, + }, + ) + return nil +} diff --git a/pkg/sql/schemachanger/scexec/executor_external_test.go b/pkg/sql/schemachanger/scexec/executor_external_test.go index fa4dbe2be038..1142c71d7b85 100644 --- a/pkg/sql/schemachanger/scexec/executor_external_test.go +++ b/pkg/sql/schemachanger/scexec/executor_external_test.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdeps" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdeps/sctestutils" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraphviz" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" @@ -66,11 +67,9 @@ func (ti testInfra) newExecDeps( noopJobRegistry{}, /* jobRegistry */ noopBackfiller{}, /* indexBackfiller */ noopIndexValidator{}, /* indexValidator */ - noopCCLCallbacks{}, /* noopCCLCallbacks */ - func(ctx context.Context, txn *kv.Txn, depth int, descID descpb.ID, metadata scpb.ElementMetadata, event eventpb.EventPayload) error { - return nil - }, - nil, /* statements */ + noopPartitioner{}, /* partitioner */ + noopEventLogger{}, /* eventLogger */ + nil, /* statements */ ) } @@ -507,13 +506,12 @@ func (noopIndexValidator) ValidateInvertedIndexes( return nil } -type noopCCLCallbacks struct { -} +type noopPartitioner struct{} -func (noopCCLCallbacks) AddPartitioning( +func (noopPartitioner) AddPartitioning( ctx context.Context, - tableDesc *tabledesc.Mutable, - indexDesc *descpb.IndexDescriptor, + tbl *tabledesc.Mutable, + index catalog.Index, partitionFields []string, listPartition []*scpb.ListPartition, rangePartition []*scpb.RangePartitions, @@ -523,6 +521,15 @@ func (noopCCLCallbacks) AddPartitioning( return nil } +type noopEventLogger struct{} + +func (noopEventLogger) LogEvent( + ctx context.Context, descID descpb.ID, metadata scpb.ElementMetadata, event eventpb.EventPayload, +) error { + return nil +} + var _ scexec.IndexBackfiller = noopBackfiller{} var _ scexec.IndexValidator = noopIndexValidator{} -var _ scexec.Partitioner = noopCCLCallbacks{} +var _ scmutationexec.Partitioner = noopPartitioner{} +var _ scexec.EventLogger = noopEventLogger{} diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go b/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go index 2737d43f1698..1227c59f3815 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go @@ -50,11 +50,14 @@ type CatalogReader interface { // RemoveSyntheticDescriptor undoes the effects of AddSyntheticDescriptor. RemoveSyntheticDescriptor(id descpb.ID) +} - // AddPartitioning adds partitioning information on an index descriptor. +// Partitioner is the interface for adding partitioning to a table descriptor. +type Partitioner interface { AddPartitioning( - tableDesc *tabledesc.Mutable, - indexDesc *descpb.IndexDescriptor, + ctx context.Context, + tbl *tabledesc.Mutable, + index catalog.Index, partitionFields []string, listPartition []*scpb.ListPartition, rangePartition []*scpb.RangePartitions, @@ -91,34 +94,26 @@ type MutationVisitorStateUpdater interface { // UpdateSchemaChangerJobProgress will write the status of the job to the // specified job progress. UpdateSchemaChangerJobProgress(jobID jobspb.JobID, statuses []scpb.Status) error -} -// EventLogWriter encapsulates operations for generating -// event log entries. -type EventLogWriter interface { - EnqueueEvent( - ctx context.Context, - descID descpb.ID, - metadata *scpb.ElementMetadata, - event eventpb.EventPayload, - ) error + // EnqueueEvent will enqueue an event to be written to the event log. + EnqueueEvent(id descpb.ID, metadata *scpb.ElementMetadata, event eventpb.EventPayload) error } // NewMutationVisitor creates a new scop.MutationVisitor. func NewMutationVisitor( - cr CatalogReader, s MutationVisitorStateUpdater, ev EventLogWriter, + cr CatalogReader, s MutationVisitorStateUpdater, p Partitioner, ) scop.MutationVisitor { return &visitor{ cr: cr, s: s, - ev: ev, + p: p, } } type visitor struct { cr CatalogReader s MutationVisitorStateUpdater - ev EventLogWriter + p Partitioner } func (m *visitor) RemoveJobReference(ctx context.Context, reference scop.RemoveJobReference) error { @@ -939,7 +934,7 @@ func (m *visitor) LogEvent(ctx context.Context, op scop.LogEvent) error { if err != nil { return err } - return m.ev.EnqueueEvent(ctx, op.DescID, &op.Metadata, event) + return m.s.EnqueueEvent(op.DescID, &op.Metadata, event) } func asEventPayload( @@ -1017,7 +1012,16 @@ func (m *visitor) AddIndexPartitionInfo(ctx context.Context, op scop.AddIndexPar if err != nil { return err } - return m.cr.AddPartitioning(tbl, index.IndexDesc(), op.PartitionFields, op.ListPartitions, op.RangePartitions, nil, true) + return m.p.AddPartitioning( + ctx, + tbl, + index, + op.PartitionFields, + op.ListPartitions, + op.RangePartitions, + nil, /* allowedNewColumnNames */ + true, /* allowImplicitPartitioning */ + ) } func (m *visitor) SetIndexName(ctx context.Context, op scop.SetIndexName) error { diff --git a/pkg/sql/schemachanger/scjob/BUILD.bazel b/pkg/sql/schemachanger/scjob/BUILD.bazel index 9f73cbe91bda..ef15e6d141bf 100644 --- a/pkg/sql/schemachanger/scjob/BUILD.bazel +++ b/pkg/sql/schemachanger/scjob/BUILD.bazel @@ -11,11 +11,9 @@ go_library( "//pkg/kv", "//pkg/settings/cluster", "//pkg/sql", - "//pkg/sql/catalog/descpb", "//pkg/sql/schemachanger/scdeps", - "//pkg/sql/schemachanger/scpb", + "//pkg/sql/schemachanger/scexec", "//pkg/sql/schemachanger/scrun", "//pkg/sql/schemachanger/scsqldeps", - "//pkg/util/log/eventpb", ], ) diff --git a/pkg/sql/schemachanger/scjob/job.go b/pkg/sql/schemachanger/scjob/job.go index 709b0a3c2292..273041dd77d8 100644 --- a/pkg/sql/schemachanger/scjob/job.go +++ b/pkg/sql/schemachanger/scjob/job.go @@ -18,12 +18,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdeps" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scsqldeps" - "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" ) func init() { @@ -72,15 +70,15 @@ func (n *newSchemaChangeResumer) run(ctx context.Context, execCtxI interface{}) execCfg.DB, execCfg.InternalExecutor, execCfg.IndexBackfiller, - func(ctx context.Context, txn *kv.Txn, depth int, descID descpb.ID, metadata scpb.ElementMetadata, event eventpb.EventPayload) error { - return sql.LogEventForSchemaChanger(ctx, execCtx.ExecCfg(), txn, depth, descID, metadata, event) + func(txn *kv.Txn) scexec.EventLogger { + return sql.NewSchemaChangerEventLogger(txn, execCfg, 0) }, + scsqldeps.NewPartitioner(execCfg.Settings, &execCtx.ExtendedEvalContext().EvalContext), execCfg.JobRegistry, n.job, execCfg.Codec, execCfg.Settings, execCfg.IndexValidator, - scsqldeps.NewCCLCallbacks(execCfg.Settings, nil), execCfg.DeclarativeSchemaChangerTestingKnobs, payload.Statement, ) diff --git a/pkg/sql/schemachanger/scsqldeps/BUILD.bazel b/pkg/sql/schemachanger/scsqldeps/BUILD.bazel index 8e5000abadba..f3f6618e86a4 100644 --- a/pkg/sql/schemachanger/scsqldeps/BUILD.bazel +++ b/pkg/sql/schemachanger/scsqldeps/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/sql/catalog/tabledesc", "//pkg/sql/parser", "//pkg/sql/schemachanger/scexec", + "//pkg/sql/schemachanger/scexec/scmutationexec", "//pkg/sql/schemachanger/scpb", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", diff --git a/pkg/sql/schemachanger/scsqldeps/ccl_deps.go b/pkg/sql/schemachanger/scsqldeps/ccl_deps.go index 840b8576f5d9..a5c7fa3dd652 100644 --- a/pkg/sql/schemachanger/scsqldeps/ccl_deps.go +++ b/pkg/sql/schemachanger/scsqldeps/ccl_deps.go @@ -18,32 +18,29 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/parser" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/errors" ) -type schemaChangerCCLCallbacks struct { +type partitioner struct { settings *cluster.Settings evalContext *tree.EvalContext } -func (s *schemaChangerCCLCallbacks) AddPartitioning( +// AddPartitioning implements the scmutationexec.Partitioner interface. +func (s *partitioner) AddPartitioning( ctx context.Context, - tableDesc *tabledesc.Mutable, - indexDesc *descpb.IndexDescriptor, + tbl *tabledesc.Mutable, + index catalog.Index, partitionFields []string, listPartition []*scpb.ListPartition, rangePartition []*scpb.RangePartitions, allowedNewColumnNames []tree.Name, allowImplicitPartitioning bool, ) (err error) { - if s.settings == nil || - s.evalContext == nil { - panic("unimplemented when settings or evalContext are omitted") - } // Deserialize back into tree based types partitionBy := &tree.PartitionBy{} partitionBy.List = make([]tree.ListPartition, 0, len(listPartition)) @@ -79,19 +76,28 @@ func (s *schemaChangerCCLCallbacks) AddPartitioning( for _, field := range partitionFields { partitionBy.Fields = append(partitionBy.Fields, tree.Name(field)) } - // Create the paritioning - newImplicitCols, newPartitioning, err := CreatePartitioningCCL(ctx, s.settings, s.evalContext, tableDesc, *indexDesc, partitionBy, allowedNewColumnNames, allowImplicitPartitioning) + newImplicitCols, newPartitioning, err := CreatePartitioningCCL( + ctx, + s.settings, + s.evalContext, + tbl, + index.IndexDescDeepCopy(), + partitionBy, + allowedNewColumnNames, + allowImplicitPartitioning, + ) if err != nil { return err } - tabledesc.UpdateIndexPartitioning(indexDesc, false, newImplicitCols, newPartitioning) + tabledesc.UpdateIndexPartitioning(index.IndexDesc(), false, newImplicitCols, newPartitioning) return nil } -// NewCCLCallbacks makes callbacks needed for the new schema -// changer. -func NewCCLCallbacks(settings *cluster.Settings, evalContext *tree.EvalContext) scexec.Partitioner { - return &schemaChangerCCLCallbacks{ +// NewPartitioner returns an implementation of scmutationexec.Partitioner. +func NewPartitioner( + settings *cluster.Settings, evalContext *tree.EvalContext, +) scmutationexec.Partitioner { + return &partitioner{ settings: settings, evalContext: evalContext, } diff --git a/pkg/sql/schemachanger/scsqldeps/index_validator.go b/pkg/sql/schemachanger/scsqldeps/index_validator.go index 2a1a414c946f..ba84e6541c68 100644 --- a/pkg/sql/schemachanger/scsqldeps/index_validator.go +++ b/pkg/sql/schemachanger/scsqldeps/index_validator.go @@ -26,7 +26,7 @@ import ( // ValidateForwardIndexesFn callback function for validating forward indexes. type ValidateForwardIndexesFn func( ctx context.Context, - tableDesc catalog.TableDescriptor, + tbl catalog.TableDescriptor, indexes []catalog.Index, runHistoricalTxn sqlutil.HistoricalInternalExecTxnRunner, withFirstMutationPublic bool, @@ -38,7 +38,7 @@ type ValidateForwardIndexesFn func( type ValidateInvertedIndexesFn func( ctx context.Context, codec keys.SQLCodec, - tableDesc catalog.TableDescriptor, + tbl catalog.TableDescriptor, indexes []catalog.Index, runHistoricalTxn sqlutil.HistoricalInternalExecTxnRunner, gatherAllInvalid bool, @@ -62,7 +62,7 @@ type indexValidator struct { // ValidateForwardIndexes checks that the indexes have entries for all the rows. func (iv indexValidator) ValidateForwardIndexes( ctx context.Context, - tableDesc catalog.TableDescriptor, + tbl catalog.TableDescriptor, indexes []catalog.Index, withFirstMutationPublic bool, gatherAllInvalid bool, @@ -77,13 +77,13 @@ func (iv indexValidator) ValidateForwardIndexes( } return fn(ctx, validationTxn, iv.ieFactory(ctx, iv.newFakeSessionData(&iv.settings.SV))) } - return iv.validateForwardIndexes(ctx, tableDesc, indexes, txnRunner, withFirstMutationPublic, gatherAllInvalid, override) + return iv.validateForwardIndexes(ctx, tbl, indexes, txnRunner, withFirstMutationPublic, gatherAllInvalid, override) } // ValidateInvertedIndexes checks that the indexes have entries for all the rows. func (iv indexValidator) ValidateInvertedIndexes( ctx context.Context, - tableDesc catalog.TableDescriptor, + tbl catalog.TableDescriptor, indexes []catalog.Index, gatherAllInvalid bool, override sessiondata.InternalExecutorOverride, @@ -97,7 +97,7 @@ func (iv indexValidator) ValidateInvertedIndexes( } return fn(ctx, validationTxn, iv.ieFactory(ctx, iv.newFakeSessionData(&iv.settings.SV))) } - return iv.validateInvertedIndexes(ctx, iv.codec, tableDesc, indexes, txnRunner, gatherAllInvalid, override) + return iv.validateInvertedIndexes(ctx, iv.codec, tbl, indexes, txnRunner, gatherAllInvalid, override) } // NewIndexValidator creates a IndexValidator interface diff --git a/pkg/sql/schemachanger/testdata/alter_table_add_column b/pkg/sql/schemachanger/testdata/alter_table_add_column index 9ed6200d1585..9df9c1e76aee 100644 --- a/pkg/sql/schemachanger/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/testdata/alter_table_add_column @@ -17,6 +17,7 @@ begin transaction #1 ## PreCommitPhase stage 1 of 1 with 7 MutationType ops create job #1: "Schema change job" descriptor IDs: [56] +write *eventpb.AlterTable to event log for descriptor #56: ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL DEFAULT 42 upsert descriptor #56 ... - columnIds: @@ -112,6 +113,7 @@ upsert descriptor #56 commit transaction #2 begin transaction #3 ## PostCommitPhase stage 2 of 8 with 1 BackfillType ops +backfill indexes [2] from index #1 in table #56 commit transaction #3 begin transaction #4 ## PostCommitPhase stage 3 of 8 with 1 MutationType ops @@ -119,6 +121,7 @@ update progress of schema change job #1 commit transaction #4 begin transaction #5 ## PostCommitPhase stage 4 of 8 with 1 ValidationType ops +validate forward indexes [2] in table #56 commit transaction #5 begin transaction #6 ## PostCommitPhase stage 5 of 8 with 1 MutationType ops diff --git a/pkg/sql/schemachanger/testdata/drop b/pkg/sql/schemachanger/testdata/drop index eb86bc051e74..0a6d445c6b8f 100644 --- a/pkg/sql/schemachanger/testdata/drop +++ b/pkg/sql/schemachanger/testdata/drop @@ -32,8 +32,9 @@ upsert descriptor #56 + state: DROP + version: "2" ## PreCommitPhase non-revertible stage 2 of 2 with 3 MutationType ops +write *eventpb.DropSchema to event log for descriptor #56: DROP SCHEMA db.sc delete schema namespace entry {54 0 sc} -> 56 -deleted descriptor #56 +delete descriptor #56 # end PreCommitPhase commit transaction #1 @@ -81,6 +82,7 @@ begin transaction #2 ## PostCommitPhase non-revertible stage 1 of 1 with 3 MutationType ops create job #2: "GC for dropping descriptor 58" descriptor IDs: [58] +write *eventpb.DropTable to event log for descriptor #58: DROP TABLE db.sc.t upsert descriptor #58 ... modificationTime: {} @@ -143,10 +145,11 @@ commit transaction #1 # begin PostCommitPhase begin transaction #2 ## PostCommitPhase non-revertible stage 1 of 1 with 11 MutationType ops +write *eventpb.DropSchema to event log for descriptor #57: DROP SCHEMA db.sc CASCADE delete schema namespace entry {54 0 sc} -> 57 -deleted descriptor #57 -deleted descriptor #59 -deleted descriptor #60 +delete descriptor #57 +delete descriptor #59 +delete descriptor #60 commit transaction #2 # end PostCommitPhase @@ -180,6 +183,7 @@ upsert descriptor #55 ## PreCommitPhase non-revertible stage 2 of 2 with 6 MutationType ops create job #1: "GC for dropping descriptors and parent database 54" descriptor IDs: [] +write *eventpb.DropDatabase to event log for descriptor #54: DROP DATABASE db CASCADE delete database namespace entry {0 0 db} -> 54 delete schema namespace entry {54 0 public} -> 55 upsert descriptor #54 @@ -188,7 +192,7 @@ upsert descriptor #54 state: DROP - version: "6" + version: "7" -deleted descriptor #55 +delete descriptor #55 # end PreCommitPhase commit transaction #1 @@ -542,6 +546,7 @@ begin transaction #2 ## PostCommitPhase non-revertible stage 1 of 1 with 45 MutationType ops create job #2: "GC for dropping descriptors 64 67 65 66 68 69 70 71 74 and parent database 61" descriptor IDs: [64 67 65 66 68 69 70 71 74] +write *eventpb.DropDatabase to event log for descriptor #61: DROP DATABASE db1 CASCADE delete database namespace entry {0 0 db1} -> 61 delete schema namespace entry {61 0 public} -> 62 delete schema namespace entry {61 0 sc1} -> 63 @@ -664,9 +669,9 @@ upsert descriptor #74 - version: "2" + version: "3" viewQuery: (SELECT 'a':::sc1.typ::STRING AS k, n2, n1 FROM db1.sc1.v4) -deleted descriptor #62 -deleted descriptor #63 -deleted descriptor #72 -deleted descriptor #73 +delete descriptor #62 +delete descriptor #63 +delete descriptor #72 +delete descriptor #73 commit transaction #2 # end PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/index b/pkg/sql/schemachanger/testdata/index index b725ace94c7d..7db54ff11ce3 100644 --- a/pkg/sql/schemachanger/testdata/index +++ b/pkg/sql/schemachanger/testdata/index @@ -79,6 +79,7 @@ upsert descriptor #54 commit transaction #2 begin transaction #3 ## PostCommitPhase stage 2 of 6 with 1 BackfillType ops +backfill indexes [2] from index #1 in table #54 commit transaction #3 begin transaction #4 ## PostCommitPhase stage 3 of 6 with 1 MutationType ops @@ -86,6 +87,7 @@ update progress of schema change job #1 commit transaction #4 begin transaction #5 ## PostCommitPhase stage 4 of 6 with 1 ValidationType ops +validate forward indexes [2] in table #54 commit transaction #5 begin transaction #6 ## PostCommitPhase stage 5 of 6 with 1 MutationType ops diff --git a/pkg/sql/sem/tree/timeconv_test.go b/pkg/sql/sem/tree/timeconv_test.go index 62bffa7fe429..44cf02d9506a 100644 --- a/pkg/sql/sem/tree/timeconv_test.go +++ b/pkg/sql/sem/tree/timeconv_test.go @@ -62,7 +62,8 @@ func TestClusterTimestampConversion(t *testing.T) { nil, // baseKey roachpb.NormalUserPriority, ts.ToTimestamp(), - 0, /* maxOffsetNs */ + 0, // maxOffsetNs + 1, // coordinatorNodeID ) ctx := tree.EvalContext{ diff --git a/pkg/storage/enginepb/mvcc3.pb.go b/pkg/storage/enginepb/mvcc3.pb.go index 9b83f26c8c06..3cd610d994a1 100644 --- a/pkg/storage/enginepb/mvcc3.pb.go +++ b/pkg/storage/enginepb/mvcc3.pb.go @@ -125,6 +125,12 @@ type TxnMeta struct { // last request. Used to provide idempotency and to protect against // out-of-order application (by means of a transaction retry). Sequence TxnSeq `protobuf:"varint,7,opt,name=sequence,proto3,casttype=TxnSeq" json:"sequence,omitempty"` + // The ID of the node where this transaction originated. + // This field represents either a SQLInstanceID of a SQL pod, a SQL + // gateway NodeID, or a KV node ID (in the case of KV-initiated + // transactions) and was introduced for the purposes of SQL Observability. + // TODO(sarkesian): Refactor to use gogoproto.casttype GenericNodeID when #73309 completes. + CoordinatorNodeID int32 `protobuf:"varint,10,opt,name=coordinator_node_id,json=coordinatorNodeId,proto3" json:"coordinator_node_id,omitempty"` } func (m *TxnMeta) Reset() { *m = TxnMeta{} } @@ -636,86 +642,88 @@ func init() { func init() { proto.RegisterFile("storage/enginepb/mvcc3.proto", fileDescriptor_6599d13285d4d9cc) } var fileDescriptor_6599d13285d4d9cc = []byte{ - // 1250 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x57, 0xcf, 0x6e, 0xdb, 0xc6, - 0x13, 0x36, 0x45, 0xca, 0xa6, 0x46, 0xb2, 0x2d, 0x6f, 0xf2, 0xfb, 0x55, 0xc8, 0x1f, 0xc9, 0xd5, - 0xa1, 0x30, 0xd2, 0x84, 0x2a, 0x92, 0x9e, 0x7c, 0x93, 0xe4, 0x20, 0x55, 0x1a, 0xc7, 0x09, 0xad, - 0xa4, 0x40, 0x0b, 0x94, 0x58, 0x91, 0x5b, 0x9a, 0x30, 0xb5, 0x64, 0xc8, 0x95, 0x22, 0xbd, 0x45, - 0x2f, 0x05, 0x7a, 0x68, 0x01, 0x9f, 0xfa, 0x04, 0x3d, 0xf4, 0x11, 0x7c, 0xcc, 0xa1, 0x87, 0xa0, - 0x40, 0x85, 0x56, 0xb9, 0xf4, 0x19, 0x92, 0x4b, 0xb1, 0xbb, 0x14, 0x25, 0x25, 0x8d, 0xac, 0x36, - 0xa8, 0x81, 0xde, 0x76, 0xe7, 0x9b, 0xf9, 0x66, 0x38, 0xfa, 0x76, 0x67, 0x05, 0x57, 0x62, 0x16, - 0x44, 0xd8, 0x25, 0x35, 0x42, 0x5d, 0x8f, 0x92, 0xb0, 0x53, 0xeb, 0xf6, 0x6d, 0xfb, 0x96, 0x11, - 0x46, 0x01, 0x0b, 0xd0, 0x25, 0x3b, 0xb0, 0x8f, 0xa3, 0x00, 0xdb, 0x47, 0x46, 0xe2, 0x67, 0x4c, - 0xfc, 0x2e, 0x95, 0x7a, 0xcc, 0xf3, 0x6b, 0x47, 0xbe, 0x5d, 0x63, 0x5e, 0x97, 0xc4, 0x0c, 0x77, - 0x43, 0x19, 0x75, 0xe9, 0xa2, 0x1b, 0xb8, 0x81, 0x58, 0xd6, 0xf8, 0x4a, 0x5a, 0xab, 0xdf, 0xa8, - 0xb0, 0xd6, 0x1e, 0xd0, 0x7d, 0xc2, 0x30, 0x7a, 0x08, 0x19, 0xcf, 0x29, 0x29, 0xdb, 0xca, 0x4e, - 0xa1, 0x51, 0x3f, 0x1d, 0x55, 0x56, 0x7e, 0x19, 0x55, 0x6e, 0xb9, 0x1e, 0x3b, 0xea, 0x75, 0x0c, - 0x3b, 0xe8, 0xd6, 0xd2, 0xb4, 0x4e, 0x67, 0xba, 0xae, 0x85, 0xc7, 0x6e, 0x4d, 0x24, 0xed, 0xf5, - 0x3c, 0xc7, 0x78, 0xf4, 0xa8, 0xb5, 0x37, 0x1e, 0x55, 0x32, 0xad, 0x3d, 0x33, 0xe3, 0x39, 0xa8, - 0x08, 0xea, 0x31, 0x19, 0x96, 0x54, 0xce, 0x69, 0xf2, 0x25, 0xaa, 0x42, 0x96, 0x84, 0x81, 0x7d, - 0x54, 0xd2, 0xb6, 0x95, 0x9d, 0x6c, 0xa3, 0xf0, 0x72, 0x54, 0xd1, 0xdb, 0x03, 0x7a, 0x9b, 0xdb, - 0x4c, 0x09, 0xa1, 0x7b, 0xb0, 0xf9, 0x34, 0xf2, 0x18, 0xb1, 0xd2, 0x6f, 0x28, 0x65, 0xb7, 0x95, - 0x9d, 0xfc, 0xcd, 0xab, 0xc6, 0xf4, 0xd3, 0x79, 0x4e, 0xe3, 0xc8, 0xb7, 0x8d, 0xf6, 0xc4, 0xa9, - 0xa1, 0xf1, 0xa2, 0xcd, 0x0d, 0x11, 0x9b, 0x5a, 0xd1, 0x27, 0xb0, 0xde, 0xf5, 0xe8, 0x0c, 0x57, - 0x6e, 0x79, 0xae, 0x42, 0xd7, 0xa3, 0x53, 0xa6, 0x0f, 0x41, 0x0f, 0x23, 0x2f, 0x88, 0x3c, 0x36, - 0x2c, 0xad, 0x8a, 0xf2, 0x37, 0x5f, 0x8e, 0x2a, 0xf9, 0xf6, 0x80, 0x3e, 0x48, 0xcc, 0x66, 0xea, - 0x80, 0x3e, 0x00, 0x3d, 0x26, 0x4f, 0x7a, 0x84, 0xda, 0xa4, 0xb4, 0x26, 0x9c, 0xe1, 0xe5, 0xa8, - 0xb2, 0xda, 0x1e, 0xd0, 0x43, 0xf2, 0xc4, 0x4c, 0xb1, 0x5d, 0xfd, 0xdb, 0x93, 0xca, 0xca, 0x4f, - 0x27, 0x15, 0xe5, 0xae, 0xa6, 0x67, 0x8a, 0xea, 0x5d, 0x4d, 0xd7, 0x8b, 0xb9, 0xea, 0x97, 0x80, - 0x5a, 0x2e, 0x0d, 0x22, 0xe2, 0x1c, 0x92, 0x27, 0xf7, 0x7b, 0x5d, 0x13, 0x53, 0x97, 0xa0, 0x6d, - 0xc8, 0xc6, 0x0c, 0x47, 0x4c, 0xfc, 0x48, 0xf3, 0x84, 0x12, 0x40, 0x57, 0x40, 0x25, 0xd4, 0x29, - 0x65, 0xde, 0xc0, 0xb9, 0x79, 0x57, 0xe7, 0x79, 0xfe, 0x38, 0xa9, 0x28, 0xd5, 0x9f, 0x35, 0xd8, - 0xd8, 0x7f, 0xdc, 0x6c, 0x1e, 0x32, 0xcc, 0xe2, 0x3d, 0xe2, 0x33, 0x8c, 0x6e, 0x00, 0xb2, 0x03, - 0xca, 0xb0, 0x47, 0x63, 0x8b, 0xc4, 0xcc, 0xeb, 0x62, 0x46, 0xe2, 0xd2, 0xc6, 0xb6, 0xb2, 0xa3, - 0x9a, 0x5b, 0x13, 0xe4, 0xf6, 0x04, 0x40, 0xd7, 0x60, 0xcb, 0xc7, 0x31, 0xb3, 0x7a, 0xa1, 0x83, - 0x19, 0xb1, 0x28, 0xa6, 0x41, 0x2c, 0xea, 0x2a, 0x9a, 0x9b, 0x1c, 0x78, 0x24, 0xec, 0xf7, 0xb9, - 0x19, 0x5d, 0x05, 0xf0, 0x28, 0x23, 0x94, 0x59, 0xd8, 0x25, 0xa2, 0xb8, 0xa2, 0x99, 0x93, 0x96, - 0xba, 0x4b, 0xd0, 0x47, 0x50, 0x70, 0x6d, 0xab, 0x33, 0x64, 0x24, 0x16, 0x0e, 0x5c, 0x2e, 0xc5, - 0xc6, 0xc6, 0x78, 0x54, 0x81, 0x3b, 0xcd, 0x06, 0x37, 0xd7, 0x5d, 0x62, 0x82, 0x6b, 0x4f, 0xd6, - 0x9c, 0xd0, 0xf7, 0xfa, 0x44, 0xc6, 0x08, 0x29, 0x21, 0x33, 0xc7, 0x2d, 0xc2, 0x23, 0x85, 0xed, - 0xa0, 0x47, 0x99, 0xd0, 0x4e, 0x02, 0x37, 0xb9, 0x01, 0x5d, 0x86, 0xdc, 0x31, 0x19, 0x26, 0xc1, - 0xab, 0x02, 0xd5, 0x8f, 0xc9, 0x50, 0xc6, 0x26, 0xa0, 0x0c, 0x5d, 0x4b, 0xc1, 0x34, 0xb2, 0x8f, - 0xfd, 0x24, 0x52, 0x97, 0x60, 0x1f, 0xfb, 0x69, 0x24, 0x07, 0x65, 0x64, 0x2e, 0x05, 0x65, 0xe4, - 0xfb, 0x50, 0x48, 0x5a, 0x20, 0x83, 0x41, 0xe0, 0x79, 0x69, 0x93, 0xf1, 0x53, 0x17, 0x49, 0x91, - 0x9f, 0x75, 0x91, 0x2c, 0x1f, 0xc3, 0xff, 0x63, 0x12, 0xe2, 0x08, 0x33, 0xe2, 0x58, 0x73, 0xce, - 0x45, 0xe1, 0x7c, 0x31, 0x45, 0x5b, 0x33, 0x51, 0x97, 0x21, 0x17, 0x0f, 0xe3, 0x24, 0x71, 0x41, - 0x16, 0x16, 0x0f, 0xe3, 0xb4, 0x6a, 0x0e, 0x4a, 0x96, 0xf5, 0x14, 0x94, 0x91, 0x3b, 0x50, 0xc4, - 0x9d, 0x20, 0x62, 0x56, 0x1c, 0x62, 0x9a, 0x10, 0x6c, 0x0a, 0x9f, 0x0d, 0x61, 0x3f, 0x0c, 0x31, - 0x15, 0x34, 0xbb, 0x9a, 0x90, 0xd5, 0xaf, 0x1a, 0x5c, 0xe0, 0xb2, 0x7a, 0x40, 0xa2, 0xd8, 0x8b, - 0x79, 0x05, 0x42, 0x60, 0xff, 0x71, 0x6d, 0xa9, 0x8b, 0xb5, 0xa5, 0x2e, 0xd4, 0x96, 0xba, 0x48, - 0x5b, 0xea, 0x22, 0x6d, 0xa9, 0x8b, 0xb4, 0xa5, 0x9e, 0xa1, 0x2d, 0xf5, 0x6c, 0x6d, 0xa9, 0x7f, - 0x47, 0x5b, 0xea, 0xb2, 0xda, 0x52, 0x17, 0x69, 0x4b, 0x5d, 0x42, 0x5b, 0xea, 0x1b, 0xda, 0x9a, - 0x5e, 0x5b, 0x3f, 0x64, 0x60, 0x4b, 0x5c, 0x85, 0xf5, 0x30, 0xf4, 0x3d, 0xe2, 0x70, 0x75, 0x11, - 0x74, 0x1d, 0x50, 0x84, 0xbf, 0x62, 0x16, 0x96, 0x46, 0xcb, 0xa3, 0x0e, 0x19, 0x08, 0xbd, 0x68, - 0x66, 0x91, 0x23, 0x89, 0x77, 0x8b, 0xdb, 0x91, 0x01, 0x17, 0x7c, 0x82, 0x63, 0xf2, 0x9a, 0x7b, - 0x46, 0xb8, 0x6f, 0x09, 0x68, 0xce, 0xff, 0x31, 0xe4, 0x23, 0x9e, 0xd2, 0x8a, 0xb9, 0x94, 0x85, - 0x80, 0xf2, 0x37, 0x6b, 0xc6, 0xdb, 0x87, 0xb0, 0xf1, 0x17, 0x27, 0x20, 0x99, 0x27, 0x20, 0x98, - 0xe4, 0x99, 0x78, 0x08, 0xff, 0x13, 0x55, 0xdb, 0x7e, 0x10, 0x13, 0x67, 0x66, 0x3e, 0x69, 0x4b, - 0xcc, 0x27, 0xf3, 0x02, 0x8f, 0x6d, 0x8a, 0xd0, 0xd4, 0x38, 0xd3, 0xa8, 0xef, 0x14, 0x28, 0xf2, - 0x32, 0x3e, 0xe3, 0xb3, 0xf0, 0x31, 0xf6, 0x7b, 0xe4, 0x20, 0x9c, 0x4c, 0x63, 0x65, 0x3a, 0x8d, - 0xeb, 0x90, 0x9b, 0xe6, 0xcd, 0x2c, 0x3f, 0x17, 0xa7, 0x51, 0xe8, 0x22, 0x64, 0xfb, 0x9c, 0x3f, - 0x19, 0xf2, 0x72, 0xc3, 0x4f, 0x49, 0x18, 0x91, 0xbe, 0x25, 0x21, 0x4d, 0x40, 0x39, 0x6e, 0x11, - 0xb5, 0x54, 0xbf, 0xcf, 0xc0, 0x56, 0x5a, 0x9e, 0x94, 0xd3, 0x41, 0x88, 0xbe, 0x80, 0x55, 0x36, - 0xa0, 0x56, 0xfa, 0x08, 0xd9, 0x7b, 0xb7, 0x47, 0x48, 0xb6, 0x3d, 0xa0, 0xad, 0x3d, 0x33, 0xcb, - 0x06, 0xb4, 0xe5, 0xa0, 0xf7, 0x60, 0x8d, 0x93, 0xf3, 0x06, 0x64, 0x44, 0x39, 0x3c, 0xd7, 0xa7, - 0x64, 0x88, 0x0e, 0x60, 0x8b, 0x03, 0xf3, 0x6f, 0x04, 0x6d, 0xf9, 0x5e, 0x6c, 0xb2, 0x01, 0xdd, - 0x9f, 0x7d, 0x26, 0xcc, 0x35, 0x55, 0xfd, 0x27, 0x4d, 0xad, 0xfe, 0xa8, 0x00, 0xe2, 0xfd, 0x91, - 0x17, 0xdd, 0xf9, 0x34, 0xe8, 0xdd, 0xb5, 0x50, 0x7d, 0x95, 0x94, 0xdd, 0x0c, 0xba, 0x5d, 0x8f, - 0x9d, 0x4f, 0xd9, 0x89, 0xa8, 0x33, 0x6f, 0x11, 0xb5, 0xfa, 0x6e, 0xa2, 0xd6, 0xde, 0x2e, 0xea, - 0xec, 0xeb, 0xa2, 0x0e, 0xa5, 0xa6, 0xeb, 0xfc, 0xf2, 0x3a, 0x97, 0x6f, 0xaf, 0x76, 0xe5, 0x23, - 0x4e, 0x64, 0x6c, 0x0f, 0xe8, 0xbf, 0x9d, 0xee, 0x95, 0x0a, 0xeb, 0x3c, 0xdf, 0xbd, 0xc0, 0xf5, - 0x6c, 0xec, 0x1f, 0x84, 0x68, 0x1f, 0xf2, 0xf2, 0xa5, 0x2e, 0x5b, 0xa2, 0x88, 0x66, 0x5f, 0x3f, - 0xeb, 0x6e, 0x9c, 0xbd, 0x94, 0x4c, 0x78, 0x9a, 0xee, 0xd0, 0x03, 0x28, 0x48, 0x3a, 0x39, 0x7e, - 0x12, 0x15, 0xde, 0x58, 0x8a, 0x6f, 0xd2, 0x71, 0x53, 0x56, 0x24, 0xb7, 0xe8, 0x10, 0xd6, 0x93, - 0x47, 0x44, 0x42, 0x29, 0xf5, 0x60, 0x9c, 0x45, 0x39, 0x7f, 0xf0, 0xcc, 0x42, 0x6f, 0x66, 0xcf, - 0x49, 0x6d, 0xa1, 0xf0, 0x09, 0xa9, 0xb6, 0x1c, 0xe9, 0xfc, 0xb1, 0x30, 0x0b, 0xf6, 0xcc, 0x9e, - 0x7f, 0xbb, 0x1c, 0x87, 0x09, 0x67, 0x76, 0xb9, 0x6f, 0x9f, 0x53, 0x9b, 0x99, 0xc7, 0xd3, 0x2d, - 0xba, 0x03, 0x39, 0xc9, 0xc8, 0x06, 0x54, 0x3c, 0x45, 0xf2, 0x37, 0xaf, 0x2d, 0x45, 0x27, 0xa4, - 0x64, 0xea, 0x38, 0x59, 0xef, 0x6a, 0xa7, 0x27, 0x15, 0xa5, 0x71, 0xed, 0xf4, 0xf7, 0xf2, 0xca, - 0xe9, 0xb8, 0xac, 0x3c, 0x1b, 0x97, 0x95, 0xe7, 0xe3, 0xb2, 0xf2, 0xdb, 0xb8, 0xac, 0x7c, 0xfd, - 0xa2, 0xbc, 0xf2, 0xec, 0x45, 0x79, 0xe5, 0xf9, 0x8b, 0xf2, 0xca, 0xe7, 0xfa, 0x84, 0xaa, 0xb3, - 0x2a, 0xfe, 0x5d, 0xde, 0xfa, 0x33, 0x00, 0x00, 0xff, 0xff, 0x28, 0x30, 0x17, 0xed, 0xc9, 0x0e, - 0x00, 0x00, + // 1286 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x57, 0xcd, 0x6e, 0xdb, 0x46, + 0x10, 0x36, 0x45, 0xca, 0xa6, 0x46, 0xb2, 0x2d, 0xad, 0x93, 0x56, 0xc8, 0x8f, 0xe8, 0xea, 0x50, + 0x18, 0x69, 0x42, 0x15, 0x49, 0x4f, 0xbe, 0xd9, 0x72, 0x90, 0x2a, 0x8d, 0xe3, 0x84, 0x56, 0x52, + 0xa0, 0x05, 0x4a, 0xac, 0xc9, 0x2d, 0x4d, 0x98, 0x5a, 0x32, 0xe4, 0xca, 0x91, 0xde, 0xa2, 0xc7, + 0x1e, 0x5a, 0xc0, 0xa7, 0x3e, 0x41, 0x0f, 0x7d, 0x04, 0x1f, 0x73, 0xe8, 0x21, 0x28, 0x5a, 0xa1, + 0x55, 0x2e, 0x7d, 0x86, 0xe4, 0x52, 0xec, 0x2e, 0x45, 0x49, 0x49, 0x23, 0xab, 0x0d, 0x6a, 0xa0, + 0xb7, 0xdd, 0xf9, 0x66, 0xbe, 0x19, 0x8e, 0xbe, 0xdd, 0x59, 0xc1, 0x95, 0x84, 0x85, 0x31, 0xf6, + 0x48, 0x83, 0x50, 0xcf, 0xa7, 0x24, 0x3a, 0x68, 0x74, 0x8e, 0x1d, 0xe7, 0x96, 0x19, 0xc5, 0x21, + 0x0b, 0xd1, 0x25, 0x27, 0x74, 0x8e, 0xe2, 0x10, 0x3b, 0x87, 0x66, 0xea, 0x67, 0x8e, 0xfc, 0x2e, + 0x55, 0xbb, 0xcc, 0x0f, 0x1a, 0x87, 0x81, 0xd3, 0x60, 0x7e, 0x87, 0x24, 0x0c, 0x77, 0x22, 0x19, + 0x75, 0xe9, 0x82, 0x17, 0x7a, 0xa1, 0x58, 0x36, 0xf8, 0x4a, 0x5a, 0xeb, 0xbf, 0xaa, 0xb0, 0xd4, + 0xee, 0xd1, 0x5d, 0xc2, 0x30, 0x7a, 0x08, 0x39, 0xdf, 0xad, 0x2a, 0xeb, 0xca, 0x46, 0x69, 0x7b, + 0xeb, 0x74, 0x60, 0x2c, 0xfc, 0x32, 0x30, 0x6e, 0x79, 0x3e, 0x3b, 0xec, 0x1e, 0x98, 0x4e, 0xd8, + 0x69, 0x64, 0x69, 0xdd, 0x83, 0xf1, 0xba, 0x11, 0x1d, 0x79, 0x0d, 0x91, 0xb4, 0xdb, 0xf5, 0x5d, + 0xf3, 0xd1, 0xa3, 0xd6, 0xce, 0x70, 0x60, 0xe4, 0x5a, 0x3b, 0x56, 0xce, 0x77, 0x51, 0x19, 0xd4, + 0x23, 0xd2, 0xaf, 0xaa, 0x9c, 0xd3, 0xe2, 0x4b, 0x54, 0x87, 0x3c, 0x89, 0x42, 0xe7, 0xb0, 0xaa, + 0xad, 0x2b, 0x1b, 0xf9, 0xed, 0xd2, 0xcb, 0x81, 0xa1, 0xb7, 0x7b, 0xf4, 0x36, 0xb7, 0x59, 0x12, + 0x42, 0xf7, 0x60, 0xf5, 0x69, 0xec, 0x33, 0x62, 0x67, 0xdf, 0x50, 0xcd, 0xaf, 0x2b, 0x1b, 0xc5, + 0x9b, 0x57, 0xcd, 0xf1, 0xa7, 0xf3, 0x9c, 0xe6, 0x61, 0xe0, 0x98, 0xed, 0x91, 0xd3, 0xb6, 0xc6, + 0x8b, 0xb6, 0x56, 0x44, 0x6c, 0x66, 0x45, 0x9f, 0xc2, 0x72, 0xc7, 0xa7, 0x13, 0x5c, 0x85, 0xf9, + 0xb9, 0x4a, 0x1d, 0x9f, 0x8e, 0x99, 0x3e, 0x02, 0x3d, 0x8a, 0xfd, 0x30, 0xf6, 0x59, 0xbf, 0xba, + 0x28, 0xca, 0x5f, 0x7d, 0x39, 0x30, 0x8a, 0xed, 0x1e, 0x7d, 0x90, 0x9a, 0xad, 0xcc, 0x01, 0x7d, + 0x08, 0x7a, 0x42, 0x9e, 0x74, 0x09, 0x75, 0x48, 0x75, 0x49, 0x38, 0xc3, 0xcb, 0x81, 0xb1, 0xd8, + 0xee, 0xd1, 0x7d, 0xf2, 0xc4, 0xca, 0x30, 0x74, 0x1b, 0xd6, 0x9c, 0x30, 0x8c, 0x5d, 0x9f, 0x62, + 0x16, 0xc6, 0x36, 0x0d, 0x5d, 0x62, 0xfb, 0x6e, 0x15, 0x44, 0xc8, 0xc5, 0xe1, 0xc0, 0xa8, 0x34, + 0xc7, 0xf0, 0xfd, 0xd0, 0x25, 0xad, 0x1d, 0xab, 0xe2, 0xbc, 0x66, 0x72, 0x37, 0xf5, 0x6f, 0x4f, + 0x8c, 0x85, 0x9f, 0x4e, 0x0c, 0xe5, 0xae, 0xa6, 0xe7, 0xca, 0xea, 0x5d, 0x4d, 0xd7, 0xcb, 0x85, + 0xfa, 0x57, 0x80, 0x5a, 0x1e, 0x0d, 0x63, 0xe2, 0xee, 0x93, 0x27, 0xf7, 0xbb, 0x1d, 0x0b, 0x53, + 0x8f, 0xa0, 0x75, 0xc8, 0x27, 0x0c, 0xc7, 0x4c, 0xfc, 0xd6, 0xd3, 0x75, 0x49, 0x00, 0x5d, 0x01, + 0x95, 0x50, 0xb7, 0x9a, 0x7b, 0x03, 0xe7, 0xe6, 0x4d, 0x9d, 0xe7, 0xf9, 0xf3, 0xc4, 0x50, 0xea, + 0x3f, 0x6b, 0xb0, 0xb2, 0xfb, 0xb8, 0xd9, 0xdc, 0x67, 0x98, 0x25, 0x3b, 0x24, 0x60, 0x18, 0xdd, + 0x00, 0xe4, 0x84, 0x94, 0x61, 0x9f, 0x26, 0x36, 0x49, 0x98, 0xdf, 0xc1, 0x8c, 0x24, 0xd5, 0x95, + 0x75, 0x65, 0x43, 0xe5, 0x75, 0x4b, 0xe4, 0xf6, 0x08, 0x40, 0xd7, 0xa0, 0x12, 0xe0, 0x84, 0xd9, + 0xdd, 0xc8, 0xc5, 0x8c, 0xd8, 0x14, 0xd3, 0x30, 0x11, 0x75, 0x95, 0xad, 0x55, 0x0e, 0x3c, 0x12, + 0xf6, 0xfb, 0xdc, 0x8c, 0xae, 0x02, 0xf8, 0x94, 0x11, 0xca, 0x6c, 0xec, 0x11, 0x51, 0x5c, 0xd9, + 0x2a, 0x48, 0xcb, 0x96, 0x47, 0xd0, 0xc7, 0x50, 0xf2, 0x1c, 0xfb, 0xa0, 0xcf, 0x48, 0x22, 0x1c, + 0xb8, 0xea, 0xca, 0xdb, 0x2b, 0xc3, 0x81, 0x01, 0x77, 0x9a, 0xdb, 0xdc, 0xbc, 0xe5, 0x11, 0x0b, + 0x3c, 0x67, 0xb4, 0xe6, 0x84, 0x81, 0x7f, 0x4c, 0x64, 0x8c, 0x50, 0x24, 0xb2, 0x0a, 0xdc, 0x22, + 0x3c, 0x32, 0xd8, 0x09, 0xbb, 0x94, 0x09, 0x09, 0xa6, 0x70, 0x93, 0x1b, 0xd0, 0x65, 0x28, 0x1c, + 0x91, 0x7e, 0x1a, 0xbc, 0x28, 0x50, 0xfd, 0x88, 0xf4, 0x65, 0x6c, 0x0a, 0xca, 0xd0, 0xa5, 0x0c, + 0xcc, 0x22, 0x8f, 0x71, 0x90, 0x46, 0xea, 0x12, 0x3c, 0xc6, 0x41, 0x16, 0xc9, 0x41, 0x19, 0x59, + 0xc8, 0x40, 0x19, 0xf9, 0x01, 0x94, 0xd2, 0x16, 0xc8, 0x60, 0x10, 0x78, 0x51, 0xda, 0x64, 0xfc, + 0xd8, 0x45, 0x52, 0x14, 0x27, 0x5d, 0x24, 0xcb, 0x27, 0xf0, 0x5e, 0x42, 0x22, 0x1c, 0x63, 0x46, + 0x5c, 0x7b, 0xca, 0xb9, 0x2c, 0x9c, 0x2f, 0x64, 0x68, 0x6b, 0x22, 0xea, 0x32, 0x14, 0x92, 0x7e, + 0x92, 0x26, 0x2e, 0xc9, 0xc2, 0x92, 0x7e, 0x92, 0x55, 0xcd, 0x41, 0xc9, 0xb2, 0x9c, 0x81, 0x32, + 0x72, 0x03, 0xca, 0xf8, 0x20, 0x8c, 0x99, 0x9d, 0x44, 0x98, 0xa6, 0x04, 0xab, 0xc2, 0x67, 0x45, + 0xd8, 0xf7, 0x23, 0x4c, 0x05, 0xcd, 0xa6, 0x26, 0x64, 0xf5, 0x9b, 0x06, 0x6b, 0x5c, 0x56, 0x0f, + 0x48, 0x9c, 0xf8, 0x09, 0xaf, 0x40, 0x08, 0xec, 0x7f, 0xae, 0x2d, 0x75, 0xb6, 0xb6, 0xd4, 0x99, + 0xda, 0x52, 0x67, 0x69, 0x4b, 0x9d, 0xa5, 0x2d, 0x75, 0x96, 0xb6, 0xd4, 0x33, 0xb4, 0xa5, 0x9e, + 0xad, 0x2d, 0xf5, 0x9f, 0x68, 0x4b, 0x9d, 0x57, 0x5b, 0xea, 0x2c, 0x6d, 0xa9, 0x73, 0x68, 0x4b, + 0x7d, 0x43, 0x5b, 0xe3, 0x6b, 0xeb, 0x87, 0x1c, 0x54, 0xc4, 0x55, 0xb8, 0x15, 0x45, 0x81, 0x4f, + 0x5c, 0xae, 0x2e, 0x82, 0xae, 0x03, 0x8a, 0xf1, 0xd7, 0xcc, 0xc6, 0xd2, 0x68, 0xfb, 0xd4, 0x25, + 0x3d, 0xa1, 0x17, 0xcd, 0x2a, 0x73, 0x24, 0xf5, 0x6e, 0x71, 0x3b, 0x32, 0x61, 0x2d, 0x20, 0x38, + 0x21, 0xaf, 0xb9, 0xe7, 0x84, 0x7b, 0x45, 0x40, 0x53, 0xfe, 0x8f, 0xa1, 0x18, 0xf3, 0x94, 0x76, + 0xc2, 0xa5, 0x2c, 0x04, 0x54, 0xbc, 0xd9, 0x30, 0xdf, 0x3e, 0xcb, 0xcd, 0xbf, 0x39, 0x01, 0xe9, + 0x58, 0x02, 0xc1, 0x24, 0xcf, 0xc4, 0x43, 0xb8, 0x28, 0xaa, 0x76, 0x82, 0x30, 0x21, 0xee, 0xc4, + 0x98, 0xd3, 0xe6, 0x18, 0x73, 0xd6, 0x1a, 0x8f, 0x6d, 0x8a, 0xd0, 0xcc, 0x38, 0xd1, 0xa8, 0xef, + 0x14, 0x28, 0xf3, 0x32, 0x3e, 0xe7, 0x23, 0xf5, 0x31, 0x0e, 0xba, 0x64, 0x2f, 0x1a, 0x0d, 0x75, + 0x65, 0x3c, 0xd4, 0xb7, 0xa0, 0x30, 0xce, 0x9b, 0x9b, 0x7f, 0xbc, 0x8e, 0xa3, 0xd0, 0x05, 0xc8, + 0x1f, 0x73, 0xfe, 0xf4, 0xad, 0x20, 0x37, 0xfc, 0x94, 0x44, 0x31, 0x39, 0xb6, 0x25, 0xa4, 0x09, + 0xa8, 0xc0, 0x2d, 0xa2, 0x96, 0xfa, 0xf7, 0x39, 0xa8, 0x64, 0xe5, 0x49, 0x39, 0xed, 0x45, 0xe8, + 0x4b, 0x58, 0x64, 0x3d, 0x6a, 0x67, 0x6f, 0x99, 0x9d, 0x77, 0x7b, 0xcb, 0xe4, 0xdb, 0x3d, 0xda, + 0xda, 0xb1, 0xf2, 0xac, 0x47, 0x5b, 0x2e, 0x7a, 0x1f, 0x96, 0x38, 0x39, 0x6f, 0x40, 0x4e, 0x94, + 0xc3, 0x73, 0x7d, 0x46, 0xfa, 0x68, 0x0f, 0x2a, 0x1c, 0x98, 0x7e, 0x6a, 0x68, 0xf3, 0xf7, 0x62, + 0x95, 0xf5, 0xe8, 0xee, 0xe4, 0x6b, 0x63, 0xaa, 0xa9, 0xea, 0xbf, 0x69, 0x6a, 0xfd, 0x47, 0x05, + 0x10, 0xef, 0x8f, 0xbc, 0xe8, 0xce, 0xa7, 0x41, 0xef, 0xae, 0x85, 0xfa, 0xab, 0xb4, 0xec, 0x66, + 0xd8, 0xe9, 0xf8, 0xec, 0x7c, 0xca, 0x4e, 0x45, 0x9d, 0x7b, 0x8b, 0xa8, 0xd5, 0x77, 0x13, 0xb5, + 0xf6, 0x76, 0x51, 0xe7, 0x5f, 0x17, 0x75, 0x24, 0x35, 0xbd, 0xc5, 0x2f, 0xaf, 0x73, 0xf9, 0xf6, + 0x7a, 0x47, 0x3e, 0xe2, 0x44, 0xc6, 0x76, 0x8f, 0xfe, 0xd7, 0xe9, 0x5e, 0xa9, 0xb0, 0xcc, 0xf3, + 0xdd, 0x0b, 0x3d, 0xdf, 0xc1, 0xc1, 0x5e, 0x84, 0x76, 0xa1, 0x28, 0x1f, 0xfc, 0xb2, 0x25, 0x8a, + 0x68, 0xf6, 0xf5, 0xb3, 0xee, 0xc6, 0xc9, 0x4b, 0xc9, 0x82, 0xa7, 0xd9, 0x0e, 0x3d, 0x80, 0x92, + 0xa4, 0x93, 0xe3, 0x27, 0x55, 0xe1, 0x8d, 0xb9, 0xf8, 0x46, 0x1d, 0xb7, 0x64, 0x45, 0x72, 0x8b, + 0xf6, 0x61, 0x39, 0x7d, 0x44, 0xa4, 0x94, 0x52, 0x0f, 0xe6, 0x59, 0x94, 0xd3, 0x07, 0xcf, 0x2a, + 0x75, 0x27, 0xf6, 0x9c, 0xd4, 0x11, 0x0a, 0x1f, 0x91, 0x6a, 0xf3, 0x91, 0x4e, 0x1f, 0x0b, 0xab, + 0xe4, 0x4c, 0xec, 0xf9, 0xb7, 0xcb, 0x71, 0x98, 0x72, 0xe6, 0xe7, 0xfb, 0xf6, 0x29, 0xb5, 0x59, + 0x45, 0x3c, 0xde, 0xa2, 0x3b, 0x50, 0x90, 0x8c, 0xac, 0x47, 0xc5, 0x53, 0xa4, 0x78, 0xf3, 0xda, + 0x5c, 0x74, 0x42, 0x4a, 0x96, 0x8e, 0xd3, 0xf5, 0xa6, 0x76, 0x7a, 0x62, 0x28, 0xdb, 0xd7, 0x4e, + 0xff, 0xa8, 0x2d, 0x9c, 0x0e, 0x6b, 0xca, 0xb3, 0x61, 0x4d, 0x79, 0x3e, 0xac, 0x29, 0xbf, 0x0f, + 0x6b, 0xca, 0x37, 0x2f, 0x6a, 0x0b, 0xcf, 0x5e, 0xd4, 0x16, 0x9e, 0xbf, 0xa8, 0x2d, 0x7c, 0xa1, + 0x8f, 0xa8, 0x0e, 0x16, 0xc5, 0x9f, 0xd4, 0x5b, 0x7f, 0x05, 0x00, 0x00, 0xff, 0xff, 0x08, 0x04, + 0x24, 0x4f, 0x10, 0x0f, 0x00, 0x00, } func (this *IgnoredSeqNumRange) Equal(that interface{}) bool { @@ -936,6 +944,11 @@ func (m *TxnMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.CoordinatorNodeID != 0 { + i = encodeVarintMvcc3(dAtA, i, uint64(m.CoordinatorNodeID)) + i-- + dAtA[i] = 0x50 + } { size, err := m.MinTimestamp.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -1713,6 +1726,10 @@ func NewPopulatedTxnMeta(r randyMvcc3, easy bool) *TxnMeta { } v4 := hlc.NewPopulatedTimestamp(r, easy) this.MinTimestamp = *v4 + this.CoordinatorNodeID = int32(r.Int31()) + if r.Intn(2) == 0 { + this.CoordinatorNodeID *= -1 + } if !easy && r.Intn(10) != 0 { } return this @@ -1915,6 +1932,9 @@ func (m *TxnMeta) Size() (n int) { } l = m.MinTimestamp.Size() n += 1 + l + sovMvcc3(uint64(l)) + if m.CoordinatorNodeID != 0 { + n += 1 + sovMvcc3(uint64(m.CoordinatorNodeID)) + } return n } @@ -2469,6 +2489,25 @@ func (m *TxnMeta) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CoordinatorNodeID", wireType) + } + m.CoordinatorNodeID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMvcc3 + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CoordinatorNodeID |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipMvcc3(dAtA[iNdEx:]) diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index 22ed590a9229..8cddd3dff4a6 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -122,6 +122,13 @@ message TxnMeta { int32 sequence = 7 [(gogoproto.casttype) = "TxnSeq"]; reserved 8; + + // The ID of the node where this transaction originated. + // This field represents either a SQLInstanceID of a SQL pod, a SQL + // gateway NodeID, or a KV node ID (in the case of KV-initiated + // transactions) and was introduced for the purposes of SQL Observability. + // TODO(sarkesian): Refactor to use gogoproto.casttype GenericNodeID when #73309 completes. + int32 coordinator_node_id = 10 [(gogoproto.customname) = "CoordinatorNodeID"]; } // IgnoredSeqNumRange describes a range of ignored seqnums. diff --git a/pkg/storage/enginepb/mvcc_test.go b/pkg/storage/enginepb/mvcc_test.go index 0350c9f337a0..7acf323287ff 100644 --- a/pkg/storage/enginepb/mvcc_test.go +++ b/pkg/storage/enginepb/mvcc_test.go @@ -29,11 +29,12 @@ func TestFormatMVCCMetadata(t *testing.T) { ts := hlc.Timestamp{Logical: 1} txnDidNotUpdateMeta := true tmeta := &enginepb.TxnMeta{ - Key: roachpb.Key("a"), - ID: txnID, - Epoch: 1, - WriteTimestamp: ts, - MinTimestamp: ts, + Key: roachpb.Key("a"), + ID: txnID, + Epoch: 1, + WriteTimestamp: ts, + MinTimestamp: ts, + CoordinatorNodeID: 6, } val1 := roachpb.Value{} val1.SetString("foo") diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 278618c260e8..5cc09743c86d 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -1532,7 +1532,7 @@ func TestMVCCStatsRandomized(t *testing.T) { } actions["EnsureTxn"] = func(s *state) string { if s.Txn == nil { - txn := roachpb.MakeTransaction("test", nil, 0, s.TS, 0) + txn := roachpb.MakeTransaction("test", nil, 0, s.TS, 0, 1) s.Txn = &txn } return "" diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 6ea8118ca294..af67ce017e6a 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -57,13 +57,13 @@ var ( txn2ID = uuid.MakeV4() txn1TS = hlc.Timestamp{Logical: 1} txn2TS = hlc.Timestamp{Logical: 2} - txn1 = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 1, WriteTimestamp: txn1TS, MinTimestamp: txn1TS}, ReadTimestamp: txn1TS} - txn1Commit = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 1, WriteTimestamp: txn1TS, MinTimestamp: txn1TS}, ReadTimestamp: txn1TS, Status: roachpb.COMMITTED} - txn1Abort = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 1, WriteTimestamp: txn1TS, MinTimestamp: txn1TS}, Status: roachpb.ABORTED} - txn1e2 = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 2, WriteTimestamp: txn1TS, MinTimestamp: txn1TS}, ReadTimestamp: txn1TS} - txn1e2Commit = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 2, WriteTimestamp: txn1TS, MinTimestamp: txn1TS}, ReadTimestamp: txn1TS, Status: roachpb.COMMITTED} - txn2 = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn2ID, WriteTimestamp: txn2TS, MinTimestamp: txn2TS}, ReadTimestamp: txn2TS} - txn2Commit = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn2ID, WriteTimestamp: txn2TS, MinTimestamp: txn2TS}, ReadTimestamp: txn2TS, Status: roachpb.COMMITTED} + txn1 = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 1, WriteTimestamp: txn1TS, MinTimestamp: txn1TS, CoordinatorNodeID: 1}, ReadTimestamp: txn1TS} + txn1Commit = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 1, WriteTimestamp: txn1TS, MinTimestamp: txn1TS, CoordinatorNodeID: 1}, ReadTimestamp: txn1TS, Status: roachpb.COMMITTED} + txn1Abort = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 1, WriteTimestamp: txn1TS, MinTimestamp: txn1TS, CoordinatorNodeID: 1}, Status: roachpb.ABORTED} + txn1e2 = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 2, WriteTimestamp: txn1TS, MinTimestamp: txn1TS, CoordinatorNodeID: 1}, ReadTimestamp: txn1TS} + txn1e2Commit = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 2, WriteTimestamp: txn1TS, MinTimestamp: txn1TS, CoordinatorNodeID: 1}, ReadTimestamp: txn1TS, Status: roachpb.COMMITTED} + txn2 = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn2ID, WriteTimestamp: txn2TS, MinTimestamp: txn2TS, CoordinatorNodeID: 2}, ReadTimestamp: txn2TS} + txn2Commit = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn2ID, WriteTimestamp: txn2TS, MinTimestamp: txn2TS, CoordinatorNodeID: 2}, ReadTimestamp: txn2TS, Status: roachpb.COMMITTED} value1 = roachpb.MakeValueFromString("testValue1") value2 = roachpb.MakeValueFromString("testValue2") value3 = roachpb.MakeValueFromString("testValue3") @@ -2274,7 +2274,7 @@ func TestMVCCClearTimeRange(t *testing.T) { }) // Add an intent at k3@ts3. - txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, ts3, 1) + txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, ts3, 1, 1) setupKVsWithIntent := func(t *testing.T) Engine { e := setupKVs(t) require.NoError(t, MVCCPut(ctx, e, &enginepb.MVCCStats{}, testKey3, ts3, value3, &txn)) diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 930e6996c5e3..e96f3c75cf7e 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -598,7 +598,7 @@ type testValue struct { func intent(key roachpb.Key, val string, ts hlc.Timestamp) testValue { var value = roachpb.MakeValueFromString(val) value.InitChecksum(key) - tx := roachpb.MakeTransaction(fmt.Sprintf("txn-%v", key), key, roachpb.NormalUserPriority, ts, 1000) + tx := roachpb.MakeTransaction(fmt.Sprintf("txn-%v", key), key, roachpb.NormalUserPriority, ts, 1000, 99) var txn = &tx return testValue{key, value, ts, txn} } diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx index 983202723a67..ecb01d305275 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx @@ -366,7 +366,7 @@ export class DatabaseDetailsPage extends React.Component< ), cell: table => ( diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx index 47a7038028f4..d244bf4f8747 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx @@ -20,6 +20,8 @@ import { } from "src/storybook/fixtures"; import { DatabaseTablePage, DatabaseTablePageProps } from "./databaseTablePage"; import moment from "moment"; +import * as H from "history"; +const history = H.createHashHistory(); const withLoadingIndicator: DatabaseTablePageProps = { databaseName: randomName(), @@ -44,6 +46,14 @@ const withLoadingIndicator: DatabaseTablePageProps = { stats: [], lastReset: moment("2021-09-04T13:55:00Z"), }, + location: history.location, + history, + match: { + url: "", + path: history.location.pathname, + isExact: false, + params: {}, + }, refreshTableDetails: () => {}, refreshTableStats: () => {}, refreshIndexStats: () => {}, @@ -114,6 +124,14 @@ const withData: DatabaseTablePageProps = { ], lastReset: moment("2021-09-04T13:55:00Z"), }, + location: history.location, + history, + match: { + url: "", + path: history.location.pathname, + isExact: false, + params: {}, + }, refreshTableDetails: () => {}, refreshTableStats: () => {}, refreshIndexStats: () => {}, diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx index 3529da0ca03a..f1f797d3bfb1 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx @@ -10,6 +10,7 @@ import React from "react"; import { Col, Row, Tabs } from "antd"; +import { RouteComponentProps } from "react-router-dom"; import classNames from "classnames/bind"; import _ from "lodash"; import { Tooltip } from "antd"; @@ -22,6 +23,7 @@ import { SqlBox } from "src/sql"; import { ColumnDescriptor, SortSetting, SortedTable } from "src/sortedtable"; import { SummaryCard, SummaryCardItem } from "src/summaryCard"; import * as format from "src/util/format"; +import { syncHistory } from "src/util"; import styles from "./databaseTablePage.module.scss"; import { commonStyles } from "src/common"; @@ -129,10 +131,12 @@ export interface DatabaseTablePageActions { } export type DatabaseTablePageProps = DatabaseTablePageData & - DatabaseTablePageActions; + DatabaseTablePageActions & + RouteComponentProps; interface DatabaseTablePageState { sortSetting: SortSetting; + tab: string; } class DatabaseTableGrantsTable extends SortedTable {} @@ -145,18 +149,33 @@ export class DatabaseTablePage extends React.Component< constructor(props: DatabaseTablePageProps) { super(props); + const { history } = this.props; + const searchParams = new URLSearchParams(history.location.search); + const defaultTab = searchParams.get("tab") || "overview"; + this.state = { sortSetting: { ascending: true, }, + tab: defaultTab, }; } - componentDidMount() { + onTabChange = (tab: string): void => { + this.setState({ tab }); + syncHistory( + { + tab: tab, + }, + this.props.history, + ); + }; + + componentDidMount(): void { this.refresh(); } - componentDidUpdate() { + componentDidUpdate(): void { this.refresh(); } @@ -284,7 +303,7 @@ export class DatabaseTablePage extends React.Component< }, ]; - render() { + render(): React.ReactElement { return (
@@ -313,7 +332,11 @@ export class DatabaseTablePage extends React.Component<
- +