Skip to content

Commit

Permalink
Support setting assertions and receiving assertion errors for Prewrit…
Browse files Browse the repository at this point in the history
…e requests (tikv#311)

* support set assertion in 2pc mutation

Signed-off-by: lysu <[email protected]>
Signed-off-by: MyonKeminta <[email protected]>

* Receive assertion fail errors from TiKV

Signed-off-by: MyonKeminta <[email protected]>

* Add test log

Signed-off-by: MyonKeminta <[email protected]>

* Remove verbose log

Signed-off-by: MyonKeminta <[email protected]>

* update kvproto

Signed-off-by: MyonKeminta <[email protected]>

* Add metrics counter for assertions

Signed-off-by: MyonKeminta <[email protected]>

* Address some comments

Signed-off-by: MyonKeminta <[email protected]>

* Try to optimize assertion for pessimistic transactions

Signed-off-by: MyonKeminta <[email protected]>

* Fix panic on optimistic transactions

Signed-off-by: MyonKeminta <[email protected]>

* Add InitCheckExistence method for LockCtx

Signed-off-by: MyonKeminta <[email protected]>

* Support assertion level

Signed-off-by: MyonKeminta <[email protected]>

* Check assertion level before doing assertion on client side

Signed-off-by: MyonKeminta <[email protected]>

* Test bitoperations of menBUfferMutations.Push

Signed-off-by: MyonKeminta <[email protected]>

* Add test for assertion in tikv

Signed-off-by: MyonKeminta <[email protected]>

* Support run assertion test with unistore

Signed-off-by: MyonKeminta <[email protected]>

* Fix test

Signed-off-by: MyonKeminta <[email protected]>

* Fix license

Signed-off-by: MyonKeminta <[email protected]>

* Fix test

Signed-off-by: MyonKeminta <[email protected]>

* export DeleteKey

Signed-off-by: ekexium <[email protected]>

* Renaming DeleteKey

Signed-off-by: MyonKeminta <[email protected]>

* fix build

Signed-off-by: MyonKeminta <[email protected]>

* Address comments

Signed-off-by: MyonKeminta <[email protected]>

* Avoid panic when running with old version of TiKV; Add schema check on fast assertion

Signed-off-by: MyonKeminta <[email protected]>

* Add test for fast assertion

Signed-off-by: MyonKeminta <[email protected]>

* Add test for pessimistic lock check existence

Signed-off-by: MyonKeminta <[email protected]>

* Test assertion takes no effect if amending is enabled

Signed-off-by: MyonKeminta <[email protected]>

* Add HasAssertUnknown function

Signed-off-by: MyonKeminta <[email protected]>

* Add comments

Signed-off-by: MyonKeminta <[email protected]>

* Cleanup locks after assertion fail

Signed-off-by: MyonKeminta <[email protected]>

* update tidb dependency

Signed-off-by: MyonKeminta <[email protected]>

* Fix panic in TestIllegalTSO

Signed-off-by: MyonKeminta <[email protected]>

* Address comments

Signed-off-by: MyonKeminta <[email protected]>

* Add comments

Signed-off-by: MyonKeminta <[email protected]>

* Update dependency to tidb

Signed-off-by: MyonKeminta <[email protected]>

* Fix test

Signed-off-by: MyonKeminta <[email protected]>

Co-authored-by: lysu <[email protected]>
Co-authored-by: MyonKeminta <[email protected]>
Co-authored-by: ekexium <[email protected]>
  • Loading branch information
4 people authored Feb 10, 2022
1 parent cea0c4c commit b5eb031
Show file tree
Hide file tree
Showing 15 changed files with 723 additions and 86 deletions.
13 changes: 13 additions & 0 deletions error/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -231,6 +231,15 @@ func (e *ErrTokenLimit) Error() string {
return fmt.Sprintf("Store token is up to the limit, store id = %d.", e.StoreID)
}

// ErrAssertionFailed is the error that assertion on data failed.
type ErrAssertionFailed struct {
*kvrpcpb.AssertionFailed
}

func (e *ErrAssertionFailed) Error() string {
return fmt.Sprintf("assertion failed { %s }", e.AssertionFailed.String())
}

// ExtractKeyErr extracts a KeyError.
func ExtractKeyErr(keyErr *kvrpcpb.KeyError) error {
if val, err := util.EvalFailpoint("mockRetryableErrorResp"); err == nil {
Expand All @@ -248,6 +257,10 @@ func ExtractKeyErr(keyErr *kvrpcpb.KeyError) error {
return errors.WithStack(&ErrRetryable{Retryable: keyErr.Retryable})
}

if keyErr.AssertionFailed != nil {
return &ErrAssertionFailed{AssertionFailed: keyErr.AssertionFailed}
}

if keyErr.Abort != "" {
err := errors.Errorf("tikv aborts txn: %s", keyErr.GetAbort())
logutil.BgLogger().Warn("2PC failed", zap.Error(err))
Expand Down
88 changes: 87 additions & 1 deletion integration_tests/2pc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -748,6 +748,39 @@ func (s *testCommitterSuite) TestPessimisticLockReturnValues() {
s.Equal(lockCtx.Values[string(key2)].Value, key2)
}

func (s *testCommitterSuite) TestPessimisticLockCheckExistence() {
key := []byte("key")
key2 := []byte("key2")
txn := s.begin()
s.Nil(txn.Set(key, key))
s.Nil(txn.Commit(context.Background()))

txn = s.begin()
txn.SetPessimistic(true)
lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()}
lockCtx.InitCheckExistence(2)
s.Nil(txn.LockKeys(context.Background(), lockCtx, key, key2))
s.Len(lockCtx.Values, 2)
s.Empty(lockCtx.Values[string(key)].Value)
s.True(lockCtx.Values[string(key)].Exists)
s.Empty(lockCtx.Values[string(key2)].Value)
s.False(lockCtx.Values[string(key2)].Exists)
s.Nil(txn.Rollback())

txn = s.begin()
txn.SetPessimistic(true)
lockCtx = &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()}
lockCtx.InitCheckExistence(2)
lockCtx.InitReturnValues(2)
s.Nil(txn.LockKeys(context.Background(), lockCtx, key, key2))
s.Len(lockCtx.Values, 2)
s.Equal(lockCtx.Values[string(key)].Value, key)
s.True(lockCtx.Values[string(key)].Exists)
s.Empty(lockCtx.Values[string(key2)].Value)
s.False(lockCtx.Values[string(key2)].Exists)
s.Nil(txn.Rollback())
}

// TestElapsedTTL tests that elapsed time is correct even if ts physical time is greater than local time.
func (s *testCommitterSuite) TestElapsedTTL() {
key := []byte("key")
Expand Down Expand Up @@ -1217,7 +1250,7 @@ func (s *testCommitterSuite) TestResolveMixed() {
// stop txn ttl manager and remove primary key, make the other keys left behind
committer.CloseTTLManager()
muts := transaction.NewPlainMutations(1)
muts.Push(kvrpcpb.Op_Lock, pk, nil, true)
muts.Push(kvrpcpb.Op_Lock, pk, nil, true, false, false)
err = committer.PessimisticRollbackMutations(context.Background(), &muts)
s.Nil(err)

Expand Down Expand Up @@ -1600,3 +1633,56 @@ func (s *testCommitterSuite) TestNewlyInsertedMemDBFlag() {
err = txn.Commit(ctx)
s.Nil(err)
}

func (s *testCommitterSuite) TestFlagsInMemBufferMutations() {
// Get a MemDB object from a transaction object.
db := s.begin().GetMemBuffer()

// A helper for iterating all cases.
forEachCase := func(f func(op kvrpcpb.Op, key []byte, value []byte, index int, isPessimisticLock, assertExist, assertNotExist bool)) {
keyIndex := 0
for _, op := range []kvrpcpb.Op{kvrpcpb.Op_Put, kvrpcpb.Op_Del, kvrpcpb.Op_CheckNotExists} {
for flags := 0; flags < (1 << 3); flags++ {
key := []byte(fmt.Sprintf("k%05d", keyIndex))
value := []byte(fmt.Sprintf("v%05d", keyIndex))

// `flag` Iterates all combinations of flags in binary.
isPessimisticLock := (flags & 0x4) != 0
assertExist := (flags & 0x2) != 0
assertNotExist := (flags & 0x1) != 0

f(op, key, value, keyIndex, isPessimisticLock, assertExist, assertNotExist)
keyIndex++
}
}
}

// Put some keys to the MemDB
forEachCase(func(op kvrpcpb.Op, key []byte, value []byte, i int, isPessimisticLock, assertExist, assertNotExist bool) {
if op == kvrpcpb.Op_Put {
err := db.Set(key, value)
s.Nil(err)
} else if op == kvrpcpb.Op_Del {
err := db.Delete(key)
s.Nil(err)
} else {
db.UpdateFlags(key, kv.SetPresumeKeyNotExists)
}
})

// Create memBufferMutations object and add keys with flags to it.
mutations := transaction.NewMemBufferMutationsProbe(db.Len(), db)

forEachCase(func(op kvrpcpb.Op, key []byte, value []byte, i int, isPessimisticLock, assertExist, assertNotExist bool) {
handle := db.IterWithFlags(key, nil).Handle()
mutations.Push(op, isPessimisticLock, assertExist, assertNotExist, handle)
})

forEachCase(func(op kvrpcpb.Op, key []byte, value []byte, i int, isPessimisticLock, assertExist, assertNotExist bool) {
s.Equal(key, mutations.GetKey(i))
s.Equal(op, mutations.GetOp(i))
s.Equal(isPessimisticLock, mutations.IsPessimisticLock(i))
s.Equal(assertExist, mutations.IsAssertExists(i))
s.Equal(assertNotExist, mutations.IsAssertNotExist(i))
})
}
223 changes: 223 additions & 0 deletions integration_tests/assertion_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,223 @@
// Copyright 2021 TiKV Authors
//
// 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,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package tikv_test

import (
"context"
"fmt"
"testing"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/stretchr/testify/suite"
tikverr "github.com/tikv/client-go/v2/error"
"github.com/tikv/client-go/v2/kv"
"github.com/tikv/client-go/v2/oracle"
"github.com/tikv/client-go/v2/testutils"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/txnkv/transaction"
)

func TestAssertion(t *testing.T) {
suite.Run(t, new(testAssertionSuite))
}

type testAssertionSuite struct {
suite.Suite
cluster testutils.Cluster
store tikv.StoreProbe
}

func (s *testAssertionSuite) SetupTest() {
s.store = tikv.StoreProbe{KVStore: NewTestStore(s.T())}
}

func (s *testAssertionSuite) TearDownTest() {
s.store.Close()
}

type mockAmender struct{}

func (*mockAmender) AmendTxn(ctx context.Context, startInfoSchema transaction.SchemaVer, change *transaction.RelatedSchemaChange, mutations transaction.CommitterMutations) (transaction.CommitterMutations, error) {
return nil, nil
}

func (s *testAssertionSuite) testAssertionImpl(keyPrefix string, pessimistic bool, lockKeys bool, assertionLevel kvrpcpb.AssertionLevel, enableAmend bool) {
if assertionLevel != kvrpcpb.AssertionLevel_Strict {
s.Nil(failpoint.Enable("tikvclient/assertionSkipCheckFromPrewrite", "return"))
defer func() {
s.Nil(failpoint.Disable("tikvclient/assertionSkipCheckFromPrewrite"))
}()
}
if assertionLevel != kvrpcpb.AssertionLevel_Fast {
s.Nil(failpoint.Enable("tikvclient/assertionSkipCheckFromLock", "return"))
defer func() {
s.Nil(failpoint.Disable("tikvclient/assertionSkipCheckFromLock"))
}()
}

// Compose the key
k := func(i byte) []byte {
return append([]byte(keyPrefix), 'k', i)
}

// Prepare some data. Make k1, k3, k7 exist.
prepareTxn, err := s.store.Begin()
s.Nil(err)
err = prepareTxn.Set(k(1), []byte("v1"))
s.Nil(err)
err = prepareTxn.Set(k(3), []byte("v3"))
s.Nil(err)
err = prepareTxn.Set(k(7), []byte("v7"))
s.Nil(err)
err = prepareTxn.Commit(context.Background())
s.Nil(err)
prepareStartTS := prepareTxn.GetCommitter().GetStartTS()
prepareCommitTS := prepareTxn.GetCommitTS()

// A helper to perform a complete transaction. When multiple keys are passed in, assertion will be set on only
// the last key.
doTxn := func(lastAssertion kv.FlagsOp, keys ...[]byte) (uint64, error) {
txn, err := s.store.Begin()
s.Nil(err)
txn.SetAssertionLevel(assertionLevel)
txn.SetPessimistic(pessimistic)
if enableAmend {
txn.SetSchemaAmender(&mockAmender{})
}
if lockKeys {
lockCtx := kv.NewLockCtx(txn.StartTS(), 1000, time.Now())
lockCtx.InitCheckExistence(1)
err = txn.LockKeys(context.Background(), lockCtx, keys...)
s.Nil(err)
} else if pessimistic {
// Since we don't want to lock the keys to be tested, set another key as the primary.
err = txn.LockKeysWithWaitTime(context.Background(), 10000, []byte("primary"))
s.Nil(err)
}
for _, key := range keys {
err = txn.Set(key, append([]byte{'v'}, key...))
s.Nil(err)
}
txn.GetMemBuffer().UpdateFlags(keys[len(keys)-1], lastAssertion)
err = txn.Commit(context.Background())
startTS := txn.GetCommitter().GetStartTS()
return startTS, err
}

checkAssertionFailError := func(err error, startTS uint64, key []byte, assertion kvrpcpb.Assertion, existingStartTS uint64, existingCommitTS uint64) {
assertionFailed, ok := errors.Cause(err).(*tikverr.ErrAssertionFailed)
s.True(ok)
s.Equal(startTS, assertionFailed.StartTs)
s.Equal(key, assertionFailed.Key)
s.Equal(assertion, assertionFailed.Assertion)
s.Equal(existingStartTS, assertionFailed.ExistingStartTs)
s.Equal(existingCommitTS, assertionFailed.ExistingCommitTs)
}

if assertionLevel == kvrpcpb.AssertionLevel_Strict && !enableAmend {
// Single key.
_, err = doTxn(kv.SetAssertExist, k(1))
s.Nil(err)
_, err = doTxn(kv.SetAssertNotExist, k(2))
s.Nil(err)
startTS, err := doTxn(kv.SetAssertNotExist, k(3))
s.NotNil(err)
checkAssertionFailError(err, startTS, k(3), kvrpcpb.Assertion_NotExist, prepareStartTS, prepareCommitTS)
startTS, err = doTxn(kv.SetAssertExist, k(4))
s.NotNil(err)
checkAssertionFailError(err, startTS, k(4), kvrpcpb.Assertion_Exist, 0, 0)

// Multiple keys
startTS, err = doTxn(kv.SetAssertNotExist, k(5), k(6), k(7))
s.NotNil(err)
checkAssertionFailError(err, startTS, k(7), kvrpcpb.Assertion_NotExist, prepareStartTS, prepareCommitTS)
startTS, err = doTxn(kv.SetAssertExist, k(8), k(9), k(10))
s.NotNil(err)
checkAssertionFailError(err, startTS, k(10), kvrpcpb.Assertion_Exist, 0, 0)
} else if assertionLevel == kvrpcpb.AssertionLevel_Fast && pessimistic && lockKeys && !enableAmend {
// Different from STRICT level, the already-existing version's startTS and commitTS cannot be fetched.

// Single key.
_, err = doTxn(kv.SetAssertExist, k(1))
s.Nil(err)
_, err = doTxn(kv.SetAssertNotExist, k(2))
s.Nil(err)
startTS, err := doTxn(kv.SetAssertNotExist, k(3))
s.NotNil(err)
checkAssertionFailError(err, startTS, k(3), kvrpcpb.Assertion_NotExist, 0, 0)
startTS, err = doTxn(kv.SetAssertExist, k(4))
s.NotNil(err)
checkAssertionFailError(err, startTS, k(4), kvrpcpb.Assertion_Exist, 0, 0)

// Multiple keys
startTS, err = doTxn(kv.SetAssertNotExist, k(5), k(6), k(7))
s.NotNil(err)
checkAssertionFailError(err, startTS, k(7), kvrpcpb.Assertion_NotExist, 0, 0)
startTS, err = doTxn(kv.SetAssertExist, k(8), k(9), k(10))
s.NotNil(err)
checkAssertionFailError(err, startTS, k(10), kvrpcpb.Assertion_Exist, 0, 0)
} else {
// Nothing will be detected.

// Single key.
_, err = doTxn(kv.SetAssertExist, k(1))
s.Nil(err)
_, err = doTxn(kv.SetAssertNotExist, k(2))
s.Nil(err)
_, err := doTxn(kv.SetAssertNotExist, k(3))
s.Nil(err)
_, err = doTxn(kv.SetAssertExist, k(4))
s.Nil(err)

// Multiple keys
_, err = doTxn(kv.SetAssertNotExist, k(5), k(6), k(7))
s.Nil(err)
_, err = doTxn(kv.SetAssertExist, k(8), k(9), k(10))
s.Nil(err)
}
}

func (s *testAssertionSuite) TestPrewriteAssertion() {
// When the test cases runs with TiKV, the TiKV cluster can be reused, thus there may be deleted versions caused by
// previous tests. This test case may meet different behavior if there are deleted versions. To avoid it, compose a
// key prefix with a timestamp to ensure the keys to be unique.
ts, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
s.Nil(err)
prefix := fmt.Sprintf("test-prewrite-assertion-%d-", ts)
s.testAssertionImpl(prefix+"a", false, false, kvrpcpb.AssertionLevel_Strict, false)
s.testAssertionImpl(prefix+"b", true, false, kvrpcpb.AssertionLevel_Strict, false)
s.testAssertionImpl(prefix+"c", true, true, kvrpcpb.AssertionLevel_Strict, false)
s.testAssertionImpl(prefix+"a", false, false, kvrpcpb.AssertionLevel_Strict, true)
s.testAssertionImpl(prefix+"b", true, false, kvrpcpb.AssertionLevel_Strict, true)
s.testAssertionImpl(prefix+"c", true, true, kvrpcpb.AssertionLevel_Strict, true)
}

func (s *testAssertionSuite) TestFastAssertion() {
// When the test cases runs with TiKV, the TiKV cluster can be reused, thus there may be deleted versions caused by
// previous tests. This test case may meet different behavior if there are deleted versions. To avoid it, compose a
// key prefix with a timestamp to ensure the keys to be unique.
ts, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
s.Nil(err)
prefix := fmt.Sprintf("test-fast-assertion-%d-", ts)
s.testAssertionImpl(prefix+"a", false, false, kvrpcpb.AssertionLevel_Fast, false)
s.testAssertionImpl(prefix+"b", true, false, kvrpcpb.AssertionLevel_Fast, false)
s.testAssertionImpl(prefix+"c", true, true, kvrpcpb.AssertionLevel_Fast, false)
s.testAssertionImpl(prefix+"a", false, false, kvrpcpb.AssertionLevel_Fast, true)
s.testAssertionImpl(prefix+"b", true, false, kvrpcpb.AssertionLevel_Fast, true)
s.testAssertionImpl(prefix+"c", true, true, kvrpcpb.AssertionLevel_Fast, true)
}
5 changes: 3 additions & 2 deletions integration_tests/go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -4,10 +4,11 @@ go 1.16

require (
github.com/ninedraft/israce v0.0.3
github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c
github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00
github.com/pingcap/kvproto v0.0.0-20211224055123-d1a140660c39
github.com/pingcap/tidb v1.1.0-beta.0.20220208061135-294a094d9055
github.com/pingcap/tidb/parser v0.0.0-20220208061135-294a094d9055 // indirect
github.com/pingcap/tidb v1.1.0-beta.0.20220209090336-37c9dc9e5206
github.com/pingcap/tidb/parser v0.0.0-20220209083136-a850b044c134 // indirect
github.com/pkg/errors v0.9.1
github.com/stretchr/testify v1.7.0
github.com/tikv/client-go/v2 v2.0.0
Expand Down
8 changes: 4 additions & 4 deletions integration_tests/go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -668,15 +668,15 @@ github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041
github.com/pingcap/sysutil v0.0.0-20211208032423-041a72e5860d/go.mod h1:7j18ezaWTao2LHOyMlsc2Dg1vW+mDY9dEbPzVyOlaeM=
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM=
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops=
github.com/pingcap/tidb v1.1.0-beta.0.20220208061135-294a094d9055 h1:T5B3nfeO98NRFkgacnXiQt1lZd7CYBfDRqBzhkTSF6w=
github.com/pingcap/tidb v1.1.0-beta.0.20220208061135-294a094d9055/go.mod h1:1ATxD29hPy7OXNIxyZWb9PqA12F5YjmYoO9uh/655OY=
github.com/pingcap/tidb v1.1.0-beta.0.20220209090336-37c9dc9e5206 h1:xioqMwQm+D9iSQjG2V+fEkZ++TCKd+mr4bf9xF7sXk8=
github.com/pingcap/tidb v1.1.0-beta.0.20220209090336-37c9dc9e5206/go.mod h1:1ATxD29hPy7OXNIxyZWb9PqA12F5YjmYoO9uh/655OY=
github.com/pingcap/tidb-dashboard v0.0.0-20211206031355-bcc43a01d537/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ=
github.com/pingcap/tidb-dashboard v0.0.0-20220117082709-e8076b5c79ba/go.mod h1:4hk/3owVGWdvI9Kx6yCqqvM1T5PVgwyQNyMQxD3rwfc=
github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible h1:c7+izmker91NkjkZ6FgTlmD4k1A5FLOAq+li6Ki2/GY=
github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM=
github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e/go.mod h1:e1MGCA9Sg3T8jid8PKAEq5eYVuMMCq4n8gJ+Kqp4Plg=
github.com/pingcap/tidb/parser v0.0.0-20220208061135-294a094d9055 h1:ZiFxY5X03o8bzt7BH/5vCLS/eK+pjv7uJBWXFBHtFco=
github.com/pingcap/tidb/parser v0.0.0-20220208061135-294a094d9055/go.mod h1:ElJiub4lRy6UZDb+0JHDkGEdr6aOli+ykhyej7VCLoI=
github.com/pingcap/tidb/parser v0.0.0-20220209083136-a850b044c134 h1:G/poRrk+Uqe1vwNjL0QMbgesxkBVucP+9/WxD7gm/OI=
github.com/pingcap/tidb/parser v0.0.0-20220209083136-a850b044c134/go.mod h1:ElJiub4lRy6UZDb+0JHDkGEdr6aOli+ykhyej7VCLoI=
github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7 h1:DHU4vw0o15qdKsf7d/Pyhun4YtX8FwoDQxG0plPByUg=
github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs=
github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98=
Expand Down
Loading

0 comments on commit b5eb031

Please sign in to comment.