Skip to content

Commit

Permalink
Add tracepoints
Browse files Browse the repository at this point in the history
```
go test -benchtime 1s -run - -bench TrackChoices1280_Cockroach -timeout 5m ./sql -benchmem
10000 127583 ns/op 27432 B/op 145 allocs/op
go test -benchtime 10s -run - -bench TrackChoices1280_Cockroach -timeout 5m ./sql -benchmem
100000 181085 ns/op 27421 B/op 144 allocs/op
go test -benchtime 20s -run - -bench TrackChoices1280_Cockroach -timeout 5m ./sql -benchmem
300000 166564 ns/op 27673 B/op 142 allocs/op
```

```

```
2016/05/10 15:31:53.900328	0.025948	node 1
15:31:53.900331	 .     3	... node 1
15:31:53.900338	 .     7	... read has no clock uncertainty
15:31:53.900426	 .    87	... executing 1282 requests
15:31:53.900706	 .   280	... read-write path
15:31:53.900707	 .     1	... command queue
15:31:53.901148	 .   441	... left command queue
15:31:53.901151	 .     3	... request leader lease (attempt #1)
15:31:53.901392	 .   240	... prep for ts cache
15:31:53.904028	 .  2637	... applied ts cache
15:31:53.904613	 .   584	... proposed to Raft
15:31:53.905698	 .  1085	... applying batch
15:31:53.905769	 .    72	... checked aborted txn
15:31:53.905959	 .   189	... checked leadership
15:31:53.906425	 .   466	... 1280 blind puts
15:31:53.914285	 .  7860	... executed batch
15:31:53.914340	 .    55	... prep for commit
15:31:53.915122	 .   782	... committed
15:31:53.915177	 .    55	... processed async intents
15:31:53.915178	 .     1	... applied batch
15:31:53.915186	 .     9	... response obtained
```

out.base
  • Loading branch information
tbg committed May 10, 2016
1 parent e5ff7a4 commit bf27fa7
Show file tree
Hide file tree
Showing 5 changed files with 30 additions and 6 deletions.
3 changes: 3 additions & 0 deletions out.10s
Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
PASS
BenchmarkTrackChoices1280_Cockroach-8 200000 171532 ns/op 27241 B/op 142 allocs/op
ok github.com/cockroachdb/cockroach/sql 35.986s
3 changes: 3 additions & 0 deletions out.1s
Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
PASS
BenchmarkTrackChoices1280_Cockroach-8 10000 116387 ns/op 27429 B/op 145 allocs/op
ok github.com/cockroachdb/cockroach/sql 1.515s
3 changes: 3 additions & 0 deletions out.20s
Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
PASS
BenchmarkTrackChoices1280_Cockroach-8 200000 174189 ns/op 27485 B/op 143 allocs/op
ok github.com/cockroachdb/cockroach/sql 36.634s
25 changes: 20 additions & 5 deletions storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"math"
"math/rand"
"reflect"
"strconv"
"sync"
"time"

Expand Down Expand Up @@ -1062,7 +1063,9 @@ func (r *Replica) endCmds(cmd *cmd, ba *roachpb.BatchRequest, br *roachpb.BatchR
//
// TODO(tschottdorf): find a way not to update the batch txn
// which should be immutable.
func (r *Replica) applyTimestampCache(ba *roachpb.BatchRequest) *roachpb.Error {
func (r *Replica) applyTimestampCache(
ctx context.Context, ba *roachpb.BatchRequest,
) *roachpb.Error {
r.mu.Lock()
defer r.mu.Unlock()
for _, union := range ba.Requests {
Expand Down Expand Up @@ -1251,6 +1254,7 @@ func (r *Replica) addWriteCmd(
// been run to successful completion.
log.Trace(ctx, "command queue")
endCmdsFunc := r.beginCmds(&ba)
log.Trace(ctx, "left command queue")

// Guarantee we remove the commands from the command queue. This is
// wrapped to delay pErr evaluation to its value when returning.
Expand All @@ -1273,17 +1277,19 @@ func (r *Replica) addWriteCmd(
pErr = nil
}

log.Trace(ctx, "prep for ts cache")
// Examine the read and write timestamp caches for preceding
// commands which require this command to move its timestamp
// forward. Or, in the case of a transactional write, the txn
// timestamp and possible write-too-old bool.
if pErr := r.applyTimestampCache(&ba); pErr != nil {
if pErr := r.applyTimestampCache(ctx, &ba); pErr != nil {
return nil, pErr
}

log.Trace(ctx, "raft")
log.Trace(ctx, "applied ts cache")

pendingCmd, err := r.proposeRaftCommand(ctx, ba)
log.Trace(ctx, "proposed to Raft")

signal()

Expand All @@ -1293,6 +1299,7 @@ func (r *Replica) addWriteCmd(
for br == nil && pErr == nil {
select {
case respWithErr := <-pendingCmd.done:
log.Trace(ctx, "response obtained")
br, pErr = respWithErr.Reply, respWithErr.Err
case <-ctxDone:
// Cancellation is somewhat tricky since we can't prevent the
Expand Down Expand Up @@ -1644,6 +1651,7 @@ func (r *Replica) processRaftCommand(idKey storagebase.CmdIDKey, index uint64, r
// replica corruption (as of now, signaled by a replicaCorruptionError).
// We feed its return through maybeSetCorrupt to act when that happens.
br, err := r.applyRaftCommand(idKey, ctx, index, raftCmd.OriginReplica, raftCmd.Cmd)
log.Trace(ctx, "applied batch")
err = r.maybeSetCorrupt(err)

if cmd != nil {
Expand Down Expand Up @@ -1698,6 +1706,7 @@ func (r *Replica) applyRaftCommand(idKey storagebase.CmdIDKey, ctx context.Conte
br, rErr = nil, roachpb.NewError(roachpb.NewRangeFrozenError(*r.Desc()))
}
defer batch.Close()
log.Trace(ctx, "prep for commit")

// Advance the last applied index and commit the batch.
if err := setAppliedIndex(batch, &ms, r.RangeID, index); err != nil {
Expand Down Expand Up @@ -1726,6 +1735,7 @@ func (r *Replica) applyRaftCommand(idKey storagebase.CmdIDKey, ctx context.Conte
}
r.mu.Unlock()
}
log.Trace(ctx, "committed")

// On successful write commands handle write-related triggers including
// splitting.
Expand All @@ -1738,6 +1748,7 @@ func (r *Replica) applyRaftCommand(idKey storagebase.CmdIDKey, ctx context.Conte
// asynchronously - even on failure.
if originReplica.StoreID == r.store.StoreID() {
r.store.intentResolver.processIntentsAsync(r, intents)
log.Trace(ctx, "processed async intents")
}

return br, rErr
Expand All @@ -1760,6 +1771,7 @@ func (r *Replica) applyRaftCommandInBatch(
if pErr := r.checkIfTxnAborted(ctx, r.store.Engine(), *ba.Txn); pErr != nil {
return r.store.Engine().NewBatch(), engine.MVCCStats{}, nil, nil, pErr
}
log.Trace(ctx, "checked aborted txn")
}

for _, union := range ba.Requests {
Expand All @@ -1786,6 +1798,7 @@ func (r *Replica) applyRaftCommandInBatch(
roachpb.NewError(r.newNotLeaderError(lease, originReplica.StoreID))
}
}
log.Trace(ctx, "checked leadership")

// Keep track of original txn Writing state to santitize txn
// reported with any error except TransactionRetryError.
Expand All @@ -1795,6 +1808,7 @@ func (r *Replica) applyRaftCommandInBatch(
// be committed (EndTransaction with a CommitTrigger may unlock
// readOnlyCmdMu via a batch.Defer).
btch, ms, br, intents, pErr := r.executeWriteBatch(ctx, idKey, ba)
log.Trace(ctx, "executed batch")

if ba.IsWrite() {
if pErr != nil {
Expand Down Expand Up @@ -1944,7 +1958,7 @@ func isOnePhaseCommit(ba roachpb.BatchRequest) bool {
// range of keys being written is empty. If so, then the run can be
// set to put "blindly", meaning no iterator need be used to read
// existing values during the MVCC write.
func optimizePuts(batch engine.Engine, reqs []roachpb.RequestUnion) {
func optimizePuts(ctx context.Context, batch engine.Engine, reqs []roachpb.RequestUnion) {
var minKey, maxKey roachpb.Key
unique := make(map[string]struct{}, len(reqs))
// Returns false on occurrence of a duplicate key.
Expand Down Expand Up @@ -1980,6 +1994,7 @@ func optimizePuts(batch engine.Engine, reqs []roachpb.RequestUnion) {
if len(unique) < optimizePutThreshold { // don't bother if below this threshold
return
}
log.Trace(ctx, strconv.Itoa(len(unique))+" blind puts")
iter := batch.NewIterator(false /* total order iterator */)
defer iter.Close()

Expand Down Expand Up @@ -2024,7 +2039,7 @@ func (r *Replica) executeBatch(

// Optimize any contiguous sequences of put and conditional put ops.
if len(ba.Requests) >= optimizePutThreshold {
optimizePuts(batch, ba.Requests)
optimizePuts(ctx, batch, ba.Requests)
}

for index, union := range ba.Requests {
Expand Down
2 changes: 1 addition & 1 deletion storage/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1499,7 +1499,7 @@ func TestOptimizePuts(t *testing.T) {
for _, r := range c.reqs {
batch.Add(r)
}
optimizePuts(tc.engine, batch.Requests)
optimizePuts(context.Background(), tc.engine, batch.Requests)
blind := []bool{}
for _, r := range batch.Requests {
switch t := r.GetInner().(type) {
Expand Down

0 comments on commit bf27fa7

Please sign in to comment.