Skip to content

Commit

Permalink
Revert "storage: call EnsureSafeSplitKey during load-based splits"
Browse files Browse the repository at this point in the history
This reverts commit fad2024.

Release note (<category, see below>): <what> <show> <why>
  • Loading branch information
danhhz committed Dec 10, 2019
1 parent ad7bd06 commit 62640b2
Show file tree
Hide file tree
Showing 3 changed files with 1 addition and 72 deletions.
5 changes: 0 additions & 5 deletions pkg/sql/row/fetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -587,11 +587,6 @@ func (rf *Fetcher) NextKey(ctx context.Context) (rowDone bool, err error) {
// No more keys in the scan. We need to transition
// rf.rowReadyTable to rf.currentTable for the last
// row.
//
// NB: this assumes that the KV layer will never split a range
// between column families, which is a brittle assumption.
// See:
// https://github.com/cockroachdb/cockroach/pull/42056
rf.rowReadyTable = rf.currentTable
return true, nil
}
Expand Down
32 changes: 1 addition & 31 deletions pkg/storage/split/decider.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ package split
import (
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
)
Expand Down Expand Up @@ -140,36 +139,7 @@ func (d *Decider) MaybeSplitKey(now time.Time) roachpb.Key {
d.mu.Lock()
d.recordLocked(now, 0, nil)
if d.mu.splitFinder != nil && d.mu.splitFinder.Ready(now) {
// We've found a key to split at. This key might be in the middle of a
// SQL row. If we fail to rectify that, we'll cause SQL crashes:
//
// https://github.com/cockroachdb/cockroach/pull/42056
//
// While the behavior at the SQL level is arguably bad and should be
// fixed, splitting between column families is also never a good idea
// for performance in general. So, if the split key is, say
//
// /Table/51/52/53/54/55/9/1
//
// then we want to split instead at
//
// /Table/51/52/53/54/55
//
// (see TestDeciderCallsEnsureSafeSplitKey).
//
// The key found here isn't guaranteed to be a valid SQL column family
// key. This is because the keys are sampled from StartKey of requests
// hitting this replica. Ranged operations may well wish to exclude the
// start point by calling .Next() or may span multiple ranges, and so
// such a key may end up being passed to EnsureSafeSplitKey here.
//
// We take the risk that the result may sometimes not be a good split
// point (or even in this range).
var err error
key, err = keys.EnsureSafeSplitKey(d.mu.splitFinder.Key())
if err != nil {
key = nil
}
key = d.mu.splitFinder.Key()
}
d.mu.Unlock()

Expand Down
36 changes: 0 additions & 36 deletions pkg/storage/split/decider_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,12 +15,9 @@ import (
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)

func TestDecider(t *testing.T) {
Expand Down Expand Up @@ -173,36 +170,3 @@ func TestDecider(t *testing.T) {
assert.Nil(t, d.MaybeSplitKey(ms(tick)))
assert.Nil(t, d.mu.splitFinder)
}

func TestDeciderCallsEnsureSafeSplitKey(t *testing.T) {
defer leaktest.AfterTest(t)()
intn := rand.New(rand.NewSource(11)).Intn

var d Decider
Init(&d, intn, func() float64 { return 1.0 })

baseKey := keys.MakeTablePrefix(51)
for i := 0; i < 4; i++ {
baseKey = encoding.EncodeUvarintAscending(baseKey, uint64(52+i))
}
c0 := func() roachpb.Span { return roachpb.Span{Key: append([]byte(nil), keys.MakeFamilyKey(baseKey, 1)...)} }
c1 := func() roachpb.Span { return roachpb.Span{Key: append([]byte(nil), keys.MakeFamilyKey(baseKey, 9)...)} }

expK, err := keys.EnsureSafeSplitKey(c1().Key)
require.NoError(t, err)

var k roachpb.Key
var now time.Time
for i := 0; i < 2*int(minSplitSuggestionInterval/time.Second); i++ {
now = now.Add(500 * time.Millisecond)
d.Record(now, 1, c0)
now = now.Add(500 * time.Millisecond)
d.Record(now, 1, c1)
k = d.MaybeSplitKey(now)
if len(k) != 0 {
break
}
}

require.Equal(t, expK, k)
}

0 comments on commit 62640b2

Please sign in to comment.