You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
142 lines
5.5 KiB
142 lines
5.5 KiB
// Copyright 2017 PingCAP, Inc.
|
|
//
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
// you may not use this file except in compliance with the License.
|
|
// You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package tikv
|
|
|
|
import (
|
|
"context"
|
|
|
|
. "github.com/pingcap/check"
|
|
"github.com/pingcap/errors"
|
|
"github.com/pingcap/failpoint"
|
|
"github.com/pingcap/parser/terror"
|
|
"github.com/pingcap/tidb/kv"
|
|
)
|
|
|
|
// TestFailCommitPrimaryRpcErrors tests rpc errors are handled properly when
|
|
// committing primary region task.
|
|
func (s *testCommitterSuite) TestFailCommitPrimaryRpcErrors(c *C) {
|
|
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("timeout")`), IsNil)
|
|
defer func() {
|
|
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil)
|
|
}()
|
|
// The rpc error will be wrapped to ErrResultUndetermined.
|
|
t1 := s.begin(c)
|
|
err := t1.Set([]byte("a"), []byte("a1"))
|
|
c.Assert(err, IsNil)
|
|
err = t1.Commit(context.Background())
|
|
c.Assert(err, NotNil)
|
|
c.Assert(terror.ErrorEqual(err, terror.ErrResultUndetermined), IsTrue, Commentf("%s", errors.ErrorStack(err)))
|
|
|
|
// We don't need to call "Rollback" after "Commit" fails.
|
|
err = t1.Rollback()
|
|
c.Assert(err, Equals, kv.ErrInvalidTxn)
|
|
}
|
|
|
|
// TestFailCommitPrimaryRegionError tests RegionError is handled properly when
|
|
// committing primary region task.
|
|
func (s *testCommitterSuite) TestFailCommitPrimaryRegionError(c *C) {
|
|
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("notLeader")`), IsNil)
|
|
defer func() {
|
|
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil)
|
|
}()
|
|
// Ensure it returns the original error without wrapped to ErrResultUndetermined
|
|
// if it exceeds max retry timeout on RegionError.
|
|
t2 := s.begin(c)
|
|
err := t2.Set([]byte("b"), []byte("b1"))
|
|
c.Assert(err, IsNil)
|
|
err = t2.Commit(context.Background())
|
|
c.Assert(err, NotNil)
|
|
c.Assert(terror.ErrorNotEqual(err, terror.ErrResultUndetermined), IsTrue)
|
|
}
|
|
|
|
// TestFailCommitPrimaryRPCErrorThenRegionError tests the case when commit first
|
|
// receive a rpc timeout, then region errors afterwrards.
|
|
func (s *testCommitterSuite) TestFailCommitPrimaryRPCErrorThenRegionError(c *C) {
|
|
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `1*return("timeout")->return("notLeader")`), IsNil)
|
|
defer func() {
|
|
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil)
|
|
}()
|
|
// The region error will be wrapped to ErrResultUndetermined.
|
|
t1 := s.begin(c)
|
|
err := t1.Set([]byte("a"), []byte("a1"))
|
|
c.Assert(err, IsNil)
|
|
err = t1.Commit(context.Background())
|
|
c.Assert(err, NotNil)
|
|
c.Assert(terror.ErrorEqual(err, terror.ErrResultUndetermined), IsTrue, Commentf("%s", errors.ErrorStack(err)))
|
|
}
|
|
|
|
// TestFailCommitPrimaryKeyError tests KeyError is handled properly when
|
|
// committing primary region task.
|
|
func (s *testCommitterSuite) TestFailCommitPrimaryKeyError(c *C) {
|
|
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("keyError")`), IsNil)
|
|
defer func() {
|
|
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil)
|
|
}()
|
|
// Ensure it returns the original error without wrapped to ErrResultUndetermined
|
|
// if it meets KeyError.
|
|
t3 := s.begin(c)
|
|
err := t3.Set([]byte("c"), []byte("c1"))
|
|
c.Assert(err, IsNil)
|
|
err = t3.Commit(context.Background())
|
|
c.Assert(err, NotNil)
|
|
c.Assert(terror.ErrorNotEqual(err, terror.ErrResultUndetermined), IsTrue)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestFailCommitTimeout(c *C) {
|
|
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitTimeout", `return(true)`), IsNil)
|
|
defer func() {
|
|
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitTimeout"), IsNil)
|
|
}()
|
|
txn := s.begin(c)
|
|
err := txn.Set([]byte("a"), []byte("a1"))
|
|
c.Assert(err, IsNil)
|
|
err = txn.Set([]byte("b"), []byte("b1"))
|
|
c.Assert(err, IsNil)
|
|
err = txn.Set([]byte("c"), []byte("c1"))
|
|
c.Assert(err, IsNil)
|
|
err = txn.Commit(context.Background())
|
|
c.Assert(err, NotNil)
|
|
|
|
txn2 := s.begin(c)
|
|
value, err := txn2.Get(context.TODO(), []byte("a"))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(len(value), Greater, 0)
|
|
_, err = txn2.Get(context.TODO(), []byte("b"))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(len(value), Greater, 0)
|
|
}
|
|
|
|
// TestFailPrewriteRegionError tests data race does not happen on retries
|
|
func (s *testCommitterSuite) TestFailPrewriteRegionError(c *C) {
|
|
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcPrewriteResult", `return("notLeader")`), IsNil)
|
|
defer func() {
|
|
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcPrewriteResult"), IsNil)
|
|
}()
|
|
|
|
txn := s.begin(c)
|
|
|
|
// Set the value big enough to create many batches. This increases the chance of data races.
|
|
var bigVal [18000]byte
|
|
for i := 0; i < 1000; i++ {
|
|
err := txn.Set([]byte{byte(i)}, bigVal[:])
|
|
c.Assert(err, IsNil)
|
|
}
|
|
|
|
committer, err := newTwoPhaseCommitterWithInit(txn, 1)
|
|
c.Assert(err, IsNil)
|
|
|
|
ctx := context.Background()
|
|
err = committer.prewriteMutations(NewBackofferWithVars(ctx, 1000, nil), committer.mutations)
|
|
c.Assert(err, NotNil)
|
|
}
|
|
|