Skip to content

Commit

Permalink
lightning: move TestCheckRequirementsTiFlash out of local package (
Browse files Browse the repository at this point in the history
  • Loading branch information
tangenta committed Sep 8, 2022
1 parent cdcbfc8 commit e75a079
Show file tree
Hide file tree
Showing 4 changed files with 84 additions and 53 deletions.
1 change: 1 addition & 0 deletions br/pkg/lightning/backend/local/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,7 @@ go_test(
"engine_test.go",
"iterator_test.go",
"key_adapter_test.go",
"local_check_test.go",
"local_test.go",
"localhelper_test.go",
],
Expand Down
6 changes: 6 additions & 0 deletions br/pkg/lightning/backend/local/local.go
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,9 @@ func checkTiDBVersion(_ context.Context, versionStr string, requiredMinVersion,

var tiFlashReplicaQuery = "SELECT TABLE_SCHEMA, TABLE_NAME FROM information_schema.TIFLASH_REPLICA WHERE REPLICA_COUNT > 0;"

// TiFlashReplicaQueryForTest is only used for tests.
var TiFlashReplicaQueryForTest = tiFlashReplicaQuery

type tblName struct {
schema string
name string
Expand All @@ -299,6 +302,9 @@ func (t tblNames) String() string {
return b.String()
}

// CheckTiFlashVersionForTest is only used for tests.
var CheckTiFlashVersionForTest = checkTiFlashVersion

// check TiFlash replicas.
// local backend doesn't support TiFlash before tidb v4.0.5
func checkTiFlashVersion(ctx context.Context, g glue.Glue, checkCtx *backend.CheckCtx, tidbVersion semver.Version) error {
Expand Down
77 changes: 77 additions & 0 deletions br/pkg/lightning/backend/local/local_check_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
// Copyright 2022 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,
// 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 local_test

import (
"context"
"testing"

"github.com/coreos/go-semver/semver"
"github.com/golang/mock/gomock"
"github.com/pingcap/tidb/br/pkg/lightning/backend"
"github.com/pingcap/tidb/br/pkg/lightning/backend/local"
"github.com/pingcap/tidb/br/pkg/lightning/mydump"
"github.com/pingcap/tidb/br/pkg/mock"
"github.com/stretchr/testify/require"
)

func TestCheckRequirementsTiFlash(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
glue := mock.NewMockGlue(controller)
exec := mock.NewMockSQLExecutor(controller)
ctx := context.Background()

dbMetas := []*mydump.MDDatabaseMeta{
{
Name: "test",
Tables: []*mydump.MDTableMeta{
{
DB: "test",
Name: "t1",
DataFiles: []mydump.FileInfo{{}},
},
{
DB: "test",
Name: "tbl",
DataFiles: []mydump.FileInfo{{}},
},
},
},
{
Name: "test1",
Tables: []*mydump.MDTableMeta{
{
DB: "test1",
Name: "t",
DataFiles: []mydump.FileInfo{{}},
},
{
DB: "test1",
Name: "tbl",
DataFiles: []mydump.FileInfo{{}},
},
},
},
}
checkCtx := &backend.CheckCtx{DBMetas: dbMetas}

glue.EXPECT().GetSQLExecutor().Return(exec)
exec.EXPECT().QueryStringsWithLog(ctx, local.TiFlashReplicaQueryForTest, gomock.Any(), gomock.Any()).
Return([][]string{{"db", "tbl"}, {"test", "t1"}, {"test1", "tbl"}}, nil)

err := local.CheckTiFlashVersionForTest(ctx, glue, checkCtx, *semver.New("4.0.2"))
require.Regexp(t, "^lightning local backend doesn't support TiFlash in this TiDB version. conflict tables: \\[`test`.`t1`, `test1`.`tbl`\\]", err.Error())
}
53 changes: 0 additions & 53 deletions br/pkg/lightning/backend/local/local_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,7 @@ import (
"testing"

"github.com/cockroachdb/pebble"
"github.com/coreos/go-semver/semver"
"github.com/docker/go-units"
"github.com/golang/mock/gomock"
"github.com/google/uuid"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
Expand All @@ -44,9 +42,7 @@ import (
"github.com/pingcap/tidb/br/pkg/lightning/backend/kv"
"github.com/pingcap/tidb/br/pkg/lightning/common"
"github.com/pingcap/tidb/br/pkg/lightning/log"
"github.com/pingcap/tidb/br/pkg/lightning/mydump"
"github.com/pingcap/tidb/br/pkg/membuf"
"github.com/pingcap/tidb/br/pkg/mock"
"github.com/pingcap/tidb/br/pkg/pdutil"
"github.com/pingcap/tidb/br/pkg/restore/split"
"github.com/pingcap/tidb/br/pkg/utils"
Expand Down Expand Up @@ -653,55 +649,6 @@ func TestLocalIngestLoop(t *testing.T) {
require.Equal(t, atomic.LoadInt32(&maxMetaSeq), f.finishedMetaSeq.Load())
}

func TestCheckRequirementsTiFlash(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
glue := mock.NewMockGlue(controller)
exec := mock.NewMockSQLExecutor(controller)
ctx := context.Background()

dbMetas := []*mydump.MDDatabaseMeta{
{
Name: "test",
Tables: []*mydump.MDTableMeta{
{
DB: "test",
Name: "t1",
DataFiles: []mydump.FileInfo{{}},
},
{
DB: "test",
Name: "tbl",
DataFiles: []mydump.FileInfo{{}},
},
},
},
{
Name: "test1",
Tables: []*mydump.MDTableMeta{
{
DB: "test1",
Name: "t",
DataFiles: []mydump.FileInfo{{}},
},
{
DB: "test1",
Name: "tbl",
DataFiles: []mydump.FileInfo{{}},
},
},
},
}
checkCtx := &backend.CheckCtx{DBMetas: dbMetas}

glue.EXPECT().GetSQLExecutor().Return(exec)
exec.EXPECT().QueryStringsWithLog(ctx, tiFlashReplicaQuery, gomock.Any(), gomock.Any()).
Return([][]string{{"db", "tbl"}, {"test", "t1"}, {"test1", "tbl"}}, nil)

err := checkTiFlashVersion(ctx, glue, checkCtx, *semver.New("4.0.2"))
require.Regexp(t, "^lightning local backend doesn't support TiFlash in this TiDB version. conflict tables: \\[`test`.`t1`, `test1`.`tbl`\\]", err.Error())
}

func makeRanges(input []string) []Range {
ranges := make([]Range, 0, len(input)/2)
for i := 0; i < len(input)-1; i += 2 {
Expand Down

0 comments on commit e75a079

Please sign in to comment.