Skip to content

Commit

Permalink
kv: fix a buglet in SetSystemConfigTrigger and cleanup
Browse files Browse the repository at this point in the history
We were asking for the system config trigger to be executed on
EndTransaction even when we had failed to anchor the transaction on the
system config range.
I think this was benign we weren't actually committing the transactions
for which SetSystemConfigTrigger() returned errors.

I've also cleaned up the code a bit. Before this patch, there was a
systemConfigTrigger field on both the Txn and the TxnCoordSender.
However, the TxnCoordSender didn't actually know anything about any
trigger; it is just concerned with the anchoring of the transaction on
the system config range. I've clarified this point.

Release note: None
  • Loading branch information
andreimatei committed Aug 6, 2019
1 parent e3e370a commit 3d26bc5
Show file tree
Hide file tree
Showing 4 changed files with 50 additions and 26 deletions.
25 changes: 14 additions & 11 deletions pkg/internal/client/sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package client

import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
Expand Down Expand Up @@ -104,16 +105,16 @@ type TxnSender interface {
// is the only one which will be invoked.
OnFinish(func(error))

// SetSystemConfigTrigger sets the system db trigger to true on this transaction.
// This will impact the EndTransactionRequest.
// AnchorOnSystemConfigRange ensures that the transaction record, if/when it
// will be created, will be created on the system config range. This is useful
// because some commit triggers only work when the EndTransaction is evaluated
// on that range.
//
// NOTE: The system db trigger will only execute correctly if the transaction
// record is located on the range that contains the system span. If a
// transaction is created which modifies both system *and* non-system data, it
// should be ensured that the transaction record itself is on the system span.
// This can be done by making sure a system key is the first key touched in the
// transaction.
SetSystemConfigTrigger() error
// An error is returned if the transaction's key has already been set (i.e. if
// the transaction already performed any writes).
// The note above notwithstanding, it is allowed to call this method multiple
// times (even if there's been writes in between the calls).
AnchorOnSystemConfigRange() error

// GetMeta retrieves a copy of the TxnCoordMeta, which can be sent from root
// to leaf transactions or the other way around. Can be combined via
Expand Down Expand Up @@ -294,8 +295,10 @@ func (m *MockTransactionalSender) OnFinish(f func(error)) {
}
}

// SetSystemConfigTrigger is part of the TxnSender interface.
func (m *MockTransactionalSender) SetSystemConfigTrigger() error { panic("unimplemented") }
// AnchorOnSystemConfigRange is part of the TxnSender interface.
func (m *MockTransactionalSender) AnchorOnSystemConfigRange() error {
return fmt.Errorf("unimplemented")
}

// TxnStatus is part of the TxnSender interface.
func (m *MockTransactionalSender) TxnStatus() roachpb.TransactionStatus {
Expand Down
5 changes: 4 additions & 1 deletion pkg/internal/client/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,8 +274,11 @@ func (txn *Txn) CommitTimestampFixed() bool {
func (txn *Txn) SetSystemConfigTrigger() error {
txn.mu.Lock()
defer txn.mu.Unlock()
if err := txn.mu.sender.AnchorOnSystemConfigRange(); err != nil {
return err
}
txn.systemConfigTrigger = true
return txn.mu.sender.SetSystemConfigTrigger()
return nil
}

// DisablePipelining instructs the transaction not to pipeline requests. It
Expand Down
22 changes: 22 additions & 0 deletions pkg/internal/client/txn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/pkg/errors"
"github.com/stretchr/testify/require"
)

var (
Expand Down Expand Up @@ -487,3 +488,24 @@ func TestUpdateDeadlineMaybe(t *testing.T) {
t.Errorf("unexpected deadline: %s", d)
}
}

// Test that, if SetSystemConfigTrigger() fails, the systemConfigTrigger has not
// been set.
func TestAnchoringErrorNoTrigger(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()

mc := hlc.NewManualClock(1)
clock := hlc.NewClock(mc.UnixNano, time.Nanosecond)
db := NewDB(
testutils.MakeAmbientCtx(),
MakeMockTxnSenderFactory(
func(context.Context, *roachpb.Transaction, roachpb.BatchRequest,
) (*roachpb.BatchResponse, *roachpb.Error) {
return nil, nil
}),
clock)
txn := NewTxn(ctx, db, 0 /* gatewayNodeID */, RootTxn)
require.Error(t, txn.SetSystemConfigTrigger(), "unimplemented")
require.False(t, txn.systemConfigTrigger)
}
24 changes: 10 additions & 14 deletions pkg/kv/txn_coord_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package kv

import (
"bytes"
"context"
"fmt"
"runtime/debug"
Expand Down Expand Up @@ -118,11 +119,6 @@ type TxnCoordSender struct {
// (a retryable TransactionAbortedError in case of the async abort).
closed bool

// systemConfigTrigger is set to true when modifying keys from the
// SystemConfig span. This sets the SystemConfigTrigger on
// EndTransactionRequest.
systemConfigTrigger bool

// txn is the Transaction proto attached to all the requests and updated on
// all the responses.
txn roachpb.Transaction
Expand Down Expand Up @@ -1110,18 +1106,18 @@ func (tc *TxnCoordSender) setTxnAnchorKeyLocked(key roachpb.Key) error {
return nil
}

// SetSystemConfigTrigger is part of the client.TxnSender interface.
func (tc *TxnCoordSender) SetSystemConfigTrigger() error {
// AnchorOnSystemConfigRange is part of the client.TxnSender interface.
func (tc *TxnCoordSender) AnchorOnSystemConfigRange() error {
tc.mu.Lock()
defer tc.mu.Unlock()
if !tc.mu.systemConfigTrigger {
tc.mu.systemConfigTrigger = true
// The system-config trigger must be run on the system-config range which
// means any transaction with the trigger set needs to be anchored to the
// system-config range.
return tc.setTxnAnchorKeyLocked(keys.SystemConfigSpan.Key)
// Allow this to be called more than once.
if bytes.Equal(tc.mu.txn.Key, keys.SystemConfigSpan.Key) {
return nil
}
return nil
// The system-config trigger must be run on the system-config range which
// means any transaction with the trigger set needs to be anchored to the
// system-config range.
return tc.setTxnAnchorKeyLocked(keys.SystemConfigSpan.Key)
}

// TxnStatus is part of the client.TxnSender interface.
Expand Down

0 comments on commit 3d26bc5

Please sign in to comment.