Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
73500: kv,storage: persist gateway node id in transaction intents r=AlexTalks a=AlexTalks

This change augments the `TxnMeta` protobuf structure to include the
gateway node ID (responsible for initiating the transaction) when
serializing the intent.  By doing so, this commit enables the Contention
Event Store proposed in #71965, utilizing option 2.

Release note: None

73862: sql: add test asserting CREATE/USAGE on public schema r=otan a=rafiss

refs #70266

The public schema currently always has CREATE/USAGE privileges
for the public role. Add a test that confirms this.

Release note: None

73873: scdeps: tighten dependencies, log more side effects r=postamar a=postamar

This commit reworks the dependency injection for the event logger, among
other declarative schema changer dependencies. It also makes the test
dependencies more chatty in the side effects log.

Release note: None

73932: ui: select grants tab on table details page r=maryliag a=maryliag

Previosuly, when the grants view was selected on the Database
Details page, it was going to the Table Details with the Overview
tab selected.
With this commit, if the view mode selected is Grant, the grant
tab is selected on the Table Details page.

Fixes #68829

Release note: None

73943: cli: support --locality and --max-offset flags with sql tenant pods r=nvanbenschoten a=nvanbenschoten

This commit adds support for the `--locality` and `--max-offset` flags to the `cockroach mt start-sql` command.

The first of these is important because tenant SQL pods should know where they reside. This will be important in the future for multi-region serverless and also for projects like #72593.

The second of these is important because the SQL pod's max-offset setting needs to be the same as the host cluster's. If we want to be able to configure the host cluster's maximum clock offset to some non-default value, we'll need SQL pods to be configured identically.

Validation of plumbing:
```sh
./cockroach start-single-node --insecure --max-offset=250ms
./cockroach sql --insecure -e 'select crdb_internal.create_tenant(2)'

 # verify --max-offset

./cockroach mt start-sql --insecure --tenant-id=2 --sql-addr=:26258 --http-addr=:0
 # CRDB crashes with error "locally configured maximum clock offset (250ms) does not match that of node [::]:62744 (500ms)"

./cockroach mt start-sql --insecure --tenant-id=2 --sql-addr=:26258 --http-addr=:0 --max-offset=250ms
 # successful

 # verify --locality

./cockroach sql --insecure --port=26258 -e 'select gateway_region()'

ERROR: gateway_region(): no region set on the locality flag on this node

./cockroach mt start-sql --insecure --tenant-id=2 --sql-addr=:26258 --http-addr=:0 --max-offset=250ms --locality=region=us-east1

./cockroach sql --insecure --port=26258 -e 'select gateway_region()'

  gateway_region
------------------
  us-east1
```

73946: ccl/sqlproxyccl: fix TestWatchPods under stressrace r=jaylim-crl a=jaylim-crl

Fixes #69220.
Regression from #67452.

In #67452, we omitted DRAINING pods from the tenant directory. Whenever a pod
goes into the DRAINING state, the pod watcher needs time to update the
directory. Not waiting for that while calling EnsureTenantAddr produces a
stale result. This commit updates TestWatchPods by polling on EnsureTenantAddr
until the pod watcher updates the directory.

Release note: None

73954: sqlsmith: don't compare voids for joins r=rafiss a=otan

No comparison expr is defined on voids, so don't generate comparisons
for them.

Resolves #73901
Resolves #73898
Resolves #73777

Release note: None

Co-authored-by: Alex Sarkesian <sarkesian@cockroachlabs.com>
Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
Co-authored-by: Marius Posta <marius@cockroachlabs.com>
Co-authored-by: Marylia Gutierrez <marylia@cockroachlabs.com>
Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Co-authored-by: Jay <jay@cockroachlabs.com>
Co-authored-by: Oliver Tan <otan@cockroachlabs.com>
  • Loading branch information
8 people committed Dec 17, 2021
8 parents 63d35df + 1420276 + 99eeb09 + b986c48 + 83862d5 + 1190d41 + 1120d60 + 568e480 commit 089affc
Show file tree
Hide file tree
Showing 83 changed files with 736 additions and 595 deletions.
2 changes: 2 additions & 0 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -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(),
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
17 changes: 10 additions & 7 deletions pkg/ccl/sqlproxyccl/tenant/directory_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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()})
Expand Down
2 changes: 2 additions & 0 deletions pkg/cli/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
5 changes: 5 additions & 0 deletions pkg/internal/sqlsmith/relational.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/internal/sqlsmith/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvcoord/dist_sender_server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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)
Expand Down
15 changes: 11 additions & 4 deletions pkg/kv/kvclient/kvcoord/dist_sender_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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.
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvnemesis/validator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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]
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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}}

Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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}}
Expand Down Expand Up @@ -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 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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),
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_revert_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/intent_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/transaction_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/below_raft_protos_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/client_merge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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()
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/client_metrics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/client_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down Expand Up @@ -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)

Expand Down Expand Up @@ -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())

Expand Down
7 changes: 4 additions & 3 deletions pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}

Expand Down Expand Up @@ -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++
Expand Down Expand Up @@ -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
Expand Down
Loading

0 comments on commit 089affc

Please sign in to comment.