[cockroach] add -linearizable flag (#244)

157 views
Skip to first unread message

Tobias Schottdorf

unread,
Dec 22, 2014, 5:14:57 AM12/22/14
to cockroachdb/cockroach

when the -linearizable flag is set, the coordinator receiving the
EndTransaction request will make a note of the local wall time
before dispatching the request further, reexamining the local
clock and txn commit timestamp when the call completes successfully.

if either MaxOffset ns were spent between those two events, or
the transaction's commit timestamp is MaxOffset behind the
local clock after the EndTransaction call has been carried
out successfully, control is returned to the client immediately.
Otherwise, the smaller of the two durations described above
is waited before returning control.

This ensures that at the point of returning control, all nodes
in the system are ahead of the commit timestamp in the transaction.


You can merge this Pull Request by running

  git pull https://github.com/tschottdorf/cockroach linflag

Or view, comment on, or merge it at:

  https://github.com/cockroachdb/cockroach/pull/244

Commit Summary

  • mention rest explorer; update outdated comment
  • add -linearizable flag

File Changes

Patch Links:


Reply to this email directly or view it on GitHub.

Tobias Schottdorf

unread,
Dec 22, 2014, 5:28:13 AM12/22/14
to cockroachdb/cockroach

@spencerkimball I've added taking into account the commit timestamp which is something we didn't discuss. Please scrutinize :)

Spencer Kimball

unread,
Dec 22, 2014, 5:54:59 PM12/22/14
to cockroachdb/cockroach

In kv/txn_coord_sender.go:

> @@ -32,6 +33,10 @@ import (
>  	"github.com/cockroachdb/cockroach/util/log"
>  )
>  
> +var linearizable = flag.Bool("linearizable", false, "enables linearizable behaviour "+
> +	"of operations on this node by making sure that no commit timestamp is "+
> +	"reported back to the client until all other node's clocks have passed it.")

s/node's clocks have/node clocks have necessarily/

Spencer Kimball

unread,
Dec 22, 2014, 5:56:07 PM12/22/14
to cockroachdb/cockroach

In kv/txn_coord_sender.go:

> @@ -313,6 +322,25 @@ func (tc *TxnCoordSender) sendOne(call *client.Call) {
>  		var txn *proto.Transaction
>  		if call.Method == proto.EndTransaction {
>  			txn = call.Reply.Header().Txn
> +			// If the -linearizable flag is set, we want to make sure that
> +			// all the clocks in the system are past the commit timestamp
> +			// of the transaction. This is guaranteed if either
> +			// - the commit timestamp is MaxOffset behind startNS
> +			// - MaxOffset ns were spent in this function
> +			// when returning to the client. Below we chose the option

s/chose/choose/

Spencer Kimball

unread,
Dec 22, 2014, 6:03:37 PM12/22/14
to cockroachdb/cockroach

In kv/txn_coord_sender.go:

> +			// If the -linearizable flag is set, we want to make sure that
> +			// all the clocks in the system are past the commit timestamp
> +			// of the transaction. This is guaranteed if either
> +			// - the commit timestamp is MaxOffset behind startNS
> +			// - MaxOffset ns were spent in this function
> +			// when returning to the client. Below we chose the option
> +			// that involves less waiting, which is likely the first one
> +			// unless a transaction commits with an odd timestamp.
> +			if tsNS := txn.Timestamp.WallTime; startNS > tsNS {
> +				startNS = tsNS
> +			}
> +			sleepNS := tc.clock.MaxOffset() -
> +				time.Duration(tc.clock.PhysicalNow()-startNS)
> +			if *linearizable && sleepNS > 0 {
> +				defer func() {
> +					log.Infof("%v: waiting %dms on EndTransaction for linearizability", txn.ID, sleepNS/1000000)

log.V(1).Infof would be better.

Spencer Kimball

unread,
Dec 22, 2014, 6:03:50 PM12/22/14
to cockroachdb/cockroach

LGTM. Nice

Tobias Schottdorf

unread,
Dec 23, 2014, 1:14:18 AM12/23/14
to cockroachdb/cockroach

In kv/txn_coord_sender.go:

> @@ -32,6 +33,10 @@ import (
>  	"github.com/cockroachdb/cockroach/util/log"
>  )
>  
> +var linearizable = flag.Bool("linearizable", false, "enables linearizable behaviour "+
> +	"of operations on this node by making sure that no commit timestamp is "+
> +	"reported back to the client until all other node's clocks have passed it.")

done.

Tobias Schottdorf

unread,
Dec 23, 2014, 1:14:43 AM12/23/14
to cockroachdb/cockroach

In kv/txn_coord_sender.go:

> @@ -313,6 +322,25 @@ func (tc *TxnCoordSender) sendOne(call *client.Call) {
>  		var txn *proto.Transaction
>  		if call.Method == proto.EndTransaction {
>  			txn = call.Reply.Header().Txn
> +			// If the -linearizable flag is set, we want to make sure that
> +			// all the clocks in the system are past the commit timestamp
> +			// of the transaction. This is guaranteed if either
> +			// - the commit timestamp is MaxOffset behind startNS
> +			// - MaxOffset ns were spent in this function
> +			// when returning to the client. Below we chose the option

done.

Tobias Schottdorf

unread,
Dec 23, 2014, 1:15:03 AM12/23/14
to cockroachdb/cockroach

In kv/txn_coord_sender.go:

> +			// If the -linearizable flag is set, we want to make sure that
> +			// all the clocks in the system are past the commit timestamp
> +			// of the transaction. This is guaranteed if either
> +			// - the commit timestamp is MaxOffset behind startNS
> +			// - MaxOffset ns were spent in this function
> +			// when returning to the client. Below we chose the option
> +			// that involves less waiting, which is likely the first one
> +			// unless a transaction commits with an odd timestamp.
> +			if tsNS := txn.Timestamp.WallTime; startNS > tsNS {
> +				startNS = tsNS
> +			}
> +			sleepNS := tc.clock.MaxOffset() -
> +				time.Duration(tc.clock.PhysicalNow()-startNS)
> +			if *linearizable && sleepNS > 0 {
> +				defer func() {
> +					log.Infof("%v: waiting %dms on EndTransaction for linearizability", txn.ID, sleepNS/1000000)

done.

Tobias Schottdorf

unread,
Dec 23, 2014, 2:07:31 AM12/23/14
to cockroachdb/cockroach

Merged #244.

Reply all
Reply to author
Forward
0 new messages