From 06f0fccdc0780964b8f5b81793048d6d7550ac4c Mon Sep 17 00:00:00 2001 From: Tamir Duberstein Date: Mon, 24 Aug 2015 14:35:18 -0400 Subject: [PATCH 1/2] `gorename -from \"github.com/cockroachdb/cockroach/client\".Txn.txn -to Proto` --- client/db_test.go | 1 - client/txn.go | 41 ++++++++++++++--------------------------- client/txn_test.go | 2 +- sql/server.go | 9 +++++---- 4 files changed, 20 insertions(+), 33 deletions(-) diff --git a/client/db_test.go b/client/db_test.go index f4409a07c7a3..145a85906ae0 100644 --- a/client/db_test.go +++ b/client/db_test.go @@ -349,7 +349,6 @@ func TestCommonMethods(t *testing.T) { key{dbType, "Run"}: {}, key{dbType, "Txn"}: {}, key{txnType, "CommitInBatch"}: {}, - key{txnType, "ToProto"}: {}, key{txnType, "Commit"}: {}, key{txnType, "Rollback"}: {}, key{txnType, "DebugName"}: {}, diff --git a/client/txn.go b/client/txn.go index 14e4e979b1fd..c7f6cdde6938 100644 --- a/client/txn.go +++ b/client/txn.go @@ -48,15 +48,15 @@ type txnSender Txn func (ts *txnSender) Send(ctx context.Context, call proto.Call) { // Send call through wrapped sender. - call.Args.Header().Txn = &ts.txn + call.Args.Header().Txn = &ts.Proto ts.wrapped.Send(ctx, call) - ts.txn.Update(call.Reply.Header().Txn) + ts.Proto.Update(call.Reply.Header().Txn) if err, ok := call.Reply.Header().GoError().(*proto.TransactionAbortedError); ok { // On Abort, reset the transaction so we start anew on restart. - ts.txn = proto.Transaction{ - Name: ts.txn.Name, - Isolation: ts.txn.Isolation, + ts.Proto = proto.Transaction{ + Name: ts.Proto.Name, + Isolation: ts.Proto.Isolation, Priority: err.Txn.Priority, // acts as a minimum priority on restart } } @@ -67,7 +67,7 @@ func (ts *txnSender) Send(ctx context.Context, call proto.Call) { type Txn struct { db DB wrapped Sender - txn proto.Transaction + Proto proto.Transaction // haveTxnWrite is true as soon as the current attempt contains a write // (prior to sending). This is in contrast to txn.Writing, which is set // by the coordinator when the first intent has been created, and which @@ -87,7 +87,7 @@ func newTxn(db DB) *Txn { // Caller's caller. file, line, fun := caller.Lookup(2) - txn.txn.Name = fmt.Sprintf("%s:%d %s", file, line, fun) + txn.Proto.Name = fmt.Sprintf("%s:%d %s", file, line, fun) return txn } @@ -96,31 +96,18 @@ func NewTxn(db DB) *Txn { return newTxn(db) } -// NewTxnFromProto returns a transaction created from the preserved -// state. -func NewTxnFromProto(db DB, state proto.Transaction) *Txn { - txn := newTxn(db) - txn.txn = state - return txn -} - -// ToProto returns the transaction state in a protobuf. -func (txn *Txn) ToProto() proto.Transaction { - return txn.txn -} - // SetDebugName sets the debug name associated with the transaction which will // appear in log files and the web UI. Each transaction starts out with an // automatically assigned debug name composed of the file and line number where // the transaction was created. func (txn *Txn) SetDebugName(name string) { file, line, _ := caller.Lookup(1) - txn.txn.Name = fmt.Sprintf("%s:%d %s", file, line, name) + txn.Proto.Name = fmt.Sprintf("%s:%d %s", file, line, name) } // DebugName returns the debug name associated with the transaction. func (txn *Txn) DebugName() string { - return txn.txn.Name + return txn.Proto.Name } // SetSnapshotIsolation sets the transaction's isolation type to @@ -134,7 +121,7 @@ func (txn *Txn) SetSnapshotIsolation() { // TODO(pmattis): Panic if the transaction has already had // operations run on it. Needs to tie into the Txn reset in case of // retries. - txn.txn.Isolation = proto.SNAPSHOT + txn.Proto.Isolation = proto.SNAPSHOT } // InternalSetPriority sets the transaction priority. It is intended for @@ -307,7 +294,7 @@ func (txn *Txn) CommitInBatch(b *Batch) error { // Commit sends an EndTransactionRequest with Commit=true. func (txn *Txn) Commit() error { - if txn.txn.Writing { + if txn.Proto.Writing { return txn.sendEndTxnCall(true /* commit */) } return nil @@ -316,12 +303,12 @@ func (txn *Txn) Commit() error { // Rollback sends an EndTransactionRequest with Commit=false. func (txn *Txn) Rollback() error { var err error - if txn.txn.Writing { + if txn.Proto.Writing { err = txn.sendEndTxnCall(false /* commit */) } // Explicitly set the status as ABORTED so that higher layers // know that this transaction has ended. - txn.txn.Status = proto.ABORTED + txn.Proto.Status = proto.ABORTED return err } @@ -391,7 +378,7 @@ func (txn *Txn) send(calls ...proto.Call) error { // If the transaction record indicates that the coordinator never wrote // an intent (and the client doesn't have one lined up), then there's no // need to send EndTransaction. If there is one anyways, cut it off. - if txn.haveEndTxn && !(txn.txn.Writing || txn.haveTxnWrite) { + if txn.haveEndTxn && !(txn.Proto.Writing || txn.haveTxnWrite) { // There's always a call if we get here. lastIndex := len(calls) - 1 if calls[lastIndex].Method() != proto.EndTransaction { diff --git a/client/txn_test.go b/client/txn_test.go index 04b435a614de..7bf6d22c6242 100644 --- a/client/txn_test.go +++ b/client/txn_test.go @@ -123,7 +123,7 @@ func TestTxnResetTxnOnAbort(t *testing.T) { txn.db.Sender.Send(context.Background(), proto.Call{Args: testPutReq, Reply: &proto.PutResponse{}}) - if len(txn.txn.ID) != 0 { + if len(txn.Proto.ID) != 0 { t.Errorf("expected txn to be cleared") } } diff --git a/sql/server.go b/sql/server.go index c213a0f6c784..e65dde423985 100644 --- a/sql/server.go +++ b/sql/server.go @@ -115,7 +115,9 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { } // Open a pending transaction if needed. if planMaker.session.Txn != nil { - planMaker.txn = client.NewTxnFromProto(*s.db, *planMaker.session.Txn) + txn := client.NewTxn(*s.db) + txn.Proto = *planMaker.session.Txn + planMaker.txn = txn } // Send the Request for SQL execution and set the application-level error @@ -125,8 +127,7 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { // Send back the session state even if there were application-level errors. // Add transaction to session state. if planMaker.txn != nil { - t := planMaker.txn.ToProto() - planMaker.session.Txn = &t + planMaker.session.Txn = &planMaker.txn.Proto } else { planMaker.session.Txn = nil } @@ -219,7 +220,7 @@ func (s *Server) execStmt(stmt parser.Statement, req driver.Request, planMaker * // transaction from being called within an auto-transaction below. planMaker.txn = client.NewTxn(*s.db) } - } else if planMaker.txn.ToProto().Status == proto.ABORTED { + } else if planMaker.txn.Proto.Status == proto.ABORTED { switch stmt := stmt.(type) { case *parser.CommitTransaction, *parser.RollbackTransaction: // Reset to allow starting a new transaction. From 06f7a5719eb83d1cbbe9c67cb2a6262c10d5270f Mon Sep 17 00:00:00 2001 From: Tamir Duberstein Date: Mon, 24 Aug 2015 14:50:30 -0400 Subject: [PATCH 2/2] Give SQL transactions a debug name --- client/client_test.go | 2 +- client/db.go | 4 +++- client/txn.go | 19 +++++++------------ kv/txn_coord_sender.go | 2 +- kv/txn_correctness_test.go | 2 +- sql/server.go | 1 + 6 files changed, 14 insertions(+), 16 deletions(-) diff --git a/client/client_test.go b/client/client_test.go index 94c2d9eac6d6..c665a9828b92 100644 --- a/client/client_test.go +++ b/client/client_test.go @@ -485,7 +485,7 @@ func concurrentIncrements(db *client.DB, t *testing.T) { wgStart.Wait() if err := db.Txn(func(txn *client.Txn) error { - txn.SetDebugName(fmt.Sprintf("test-%d", i)) + txn.SetDebugName(fmt.Sprintf("test-%d", i), 0) // Retrieve the other key. gr, err := txn.Get(readKey) diff --git a/client/db.go b/client/db.go index 4b94868ef613..94b6a3b0a07b 100644 --- a/client/db.go +++ b/client/db.go @@ -416,7 +416,9 @@ func (db *DB) Run(b *Batch) error { // // TODO(pmattis): Allow transaction options to be specified. func (db *DB) Txn(retryable func(txn *Txn) error) error { - return newTxn(*db).exec(retryable) + txn := NewTxn(*db) + txn.SetDebugName("", 1) + return txn.exec(retryable) } // send runs the specified calls synchronously in a single batch and diff --git a/client/txn.go b/client/txn.go index c7f6cdde6938..1c58a81f5f4d 100644 --- a/client/txn.go +++ b/client/txn.go @@ -78,30 +78,25 @@ type Txn struct { haveEndTxn bool // True if there was an explicit EndTransaction } -func newTxn(db DB) *Txn { +// NewTxn returns a new txn. +func NewTxn(db DB) *Txn { txn := &Txn{ db: db, wrapped: db.Sender, } txn.db.Sender = (*txnSender)(txn) - - // Caller's caller. - file, line, fun := caller.Lookup(2) - txn.Proto.Name = fmt.Sprintf("%s:%d %s", file, line, fun) return txn } -// NewTxn returns a new txn. -func NewTxn(db DB) *Txn { - return newTxn(db) -} - // SetDebugName sets the debug name associated with the transaction which will // appear in log files and the web UI. Each transaction starts out with an // automatically assigned debug name composed of the file and line number where // the transaction was created. -func (txn *Txn) SetDebugName(name string) { - file, line, _ := caller.Lookup(1) +func (txn *Txn) SetDebugName(name string, depth int) { + file, line, fun := caller.Lookup(depth + 1) + if name == "" { + name = fun + } txn.Proto.Name = fmt.Sprintf("%s:%d %s", file, line, name) } diff --git a/kv/txn_coord_sender.go b/kv/txn_coord_sender.go index 553fc09075af..8031b3c81e6f 100644 --- a/kv/txn_coord_sender.go +++ b/kv/txn_coord_sender.go @@ -513,7 +513,7 @@ func (tc *TxnCoordSender) sendOne(ctx context.Context, call proto.Call) { } call.Reply.Reset() if err := tmpDB.Txn(func(txn *client.Txn) error { - txn.SetDebugName("auto-wrap") + txn.SetDebugName("auto-wrap", 0) b := &client.Batch{} b.InternalAddCall(call) return txn.CommitInBatch(b) diff --git a/kv/txn_correctness_test.go b/kv/txn_correctness_test.go index 597ee64dee6c..fbb9e1eba514 100644 --- a/kv/txn_correctness_test.go +++ b/kv/txn_correctness_test.go @@ -563,7 +563,7 @@ func (hv *historyVerifier) runTxn(txnIdx int, priority int32, var retry int txnName := fmt.Sprintf("txn%d", txnIdx) err := db.Txn(func(txn *client.Txn) error { - txn.SetDebugName(txnName) + txn.SetDebugName(txnName, 0) if isolation == proto.SNAPSHOT { txn.SetSnapshotIsolation() } diff --git a/sql/server.go b/sql/server.go index e65dde423985..8527eacf2465 100644 --- a/sql/server.go +++ b/sql/server.go @@ -219,6 +219,7 @@ func (s *Server) execStmt(stmt parser.Statement, req driver.Request, planMaker * // Start a transaction here and not in planMaker to prevent begin // transaction from being called within an auto-transaction below. planMaker.txn = client.NewTxn(*s.db) + planMaker.txn.SetDebugName("sql", 0) } } else if planMaker.txn.Proto.Status == proto.ABORTED { switch stmt := stmt.(type) {