diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 7b4304c877f83..3ff94899d9ebc 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -1116,7 +1116,12 @@ func checkDiskAvail(ctx context.Context, store *pdhttp.StoreInfo) error { } ratio := available * 100 / capacity if ratio < 10 { - return errors.Errorf("the remaining storage capacity of TiKV(%s) is less than 10%%; please increase the storage capacity of TiKV and try again", store.Store.Address) + storeType := "TiKV" + if engine.IsTiFlashHTTPResp(&store.Store) { + storeType = "TiFlash" + } + return errors.Errorf("the remaining storage capacity of %s(%s) is less than 10%%; please increase the storage capacity of %s and try again", + storeType, store.Store.Address, storeType) } return nil } diff --git a/br/pkg/restore/BUILD.bazel b/br/pkg/restore/BUILD.bazel index 800e0d9cfa7ad..322ff3c6eed5c 100644 --- a/br/pkg/restore/BUILD.bazel +++ b/br/pkg/restore/BUILD.bazel @@ -64,6 +64,7 @@ go_library( "//pkg/util", "//pkg/util/codec", "//pkg/util/collate", + "//pkg/util/engine", "//pkg/util/hack", "//pkg/util/mathutil", "//pkg/util/sqlexec", diff --git a/br/pkg/restore/client.go b/br/pkg/restore/client.go index 399e807e73be0..1a9bdfd9af4e9 100644 --- a/br/pkg/restore/client.go +++ b/br/pkg/restore/client.go @@ -58,6 +58,7 @@ import ( "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util/codec" "github.com/pingcap/tidb/pkg/util/collate" + "github.com/pingcap/tidb/pkg/util/engine" "github.com/pingcap/tidb/pkg/util/sqlexec" filter "github.com/pingcap/tidb/pkg/util/table-filter" "github.com/tikv/client-go/v2/oracle" @@ -1917,10 +1918,8 @@ func (rc *Client) GoWaitTiFlashReady(ctx context.Context, inCh <-chan *CreatedTa } tiFlashStores := make(map[int64]pdhttp.StoreInfo) for _, store := range tikvStats.Stores { - for _, l := range store.Store.Labels { - if l.Key == "engine" && l.Value == "tiflash" { - tiFlashStores[store.Store.ID] = store - } + if engine.IsTiFlashHTTPResp(&store.Store) { + tiFlashStores[store.Store.ID] = store } } go concurrentHandleTablesCh(ctx, inCh, outCh, errCh, workers, func(c context.Context, tbl *CreatedTable) error { diff --git a/pkg/ddl/BUILD.bazel b/pkg/ddl/BUILD.bazel index 1f4b84fefd32d..23d556b27e446 100644 --- a/pkg/ddl/BUILD.bazel +++ b/pkg/ddl/BUILD.bazel @@ -137,6 +137,7 @@ go_library( "//pkg/util/dbterror", "//pkg/util/dbterror/exeerrors", "//pkg/util/domainutil", + "//pkg/util/engine", "//pkg/util/filter", "//pkg/util/gcutil", "//pkg/util/hack", diff --git a/pkg/ddl/ddl_tiflash_api.go b/pkg/ddl/ddl_tiflash_api.go index a16502ee04ae6..d1b76a8da0d7f 100644 --- a/pkg/ddl/ddl_tiflash_api.go +++ b/pkg/ddl/ddl_tiflash_api.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util" + "github.com/pingcap/tidb/pkg/util/engine" "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/logutil" pd "github.com/tikv/pd/client/http" @@ -340,11 +341,8 @@ func updateTiFlashStores(pollTiFlashContext *TiFlashManagementContext) error { } pollTiFlashContext.TiFlashStores = make(map[int64]pd.StoreInfo) for _, store := range tikvStats.Stores { - for _, l := range store.Store.Labels { - if l.Key == "engine" && l.Value == "tiflash" { - pollTiFlashContext.TiFlashStores[store.Store.ID] = store - logutil.BgLogger().Debug("Found tiflash store", zap.Int64("id", store.Store.ID), zap.String("Address", store.Store.Address), zap.String("StatusAddress", store.Store.StatusAddress)) - } + if engine.IsTiFlashHTTPResp(&store.Store) { + pollTiFlashContext.TiFlashStores[store.Store.ID] = store } } logutil.BgLogger().Debug("updateTiFlashStores finished", zap.Int("TiFlash store count", len(pollTiFlashContext.TiFlashStores))) diff --git a/pkg/domain/infosync/BUILD.bazel b/pkg/domain/infosync/BUILD.bazel index 1b20551cf539a..2c34d101101d2 100644 --- a/pkg/domain/infosync/BUILD.bazel +++ b/pkg/domain/infosync/BUILD.bazel @@ -34,6 +34,7 @@ go_library( "//pkg/util", "//pkg/util/codec", "//pkg/util/dbterror", + "//pkg/util/engine", "//pkg/util/hack", "//pkg/util/logutil", "//pkg/util/syncutil", diff --git a/pkg/domain/infosync/info.go b/pkg/domain/infosync/info.go index d5e21f9511e65..7bb57a5d268d5 100644 --- a/pkg/domain/infosync/info.go +++ b/pkg/domain/infosync/info.go @@ -48,6 +48,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/variable" util2 "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/dbterror" + "github.com/pingcap/tidb/pkg/util/engine" "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/versioninfo" @@ -438,11 +439,8 @@ func MustGetTiFlashProgress(tableID int64, replicaCount uint64, tiFlashStores *m } stores := make(map[int64]pdhttp.StoreInfo) for _, store := range tikvStats.Stores { - for _, l := range store.Store.Labels { - if l.Key == "engine" && l.Value == "tiflash" { - stores[store.Store.ID] = store - logutil.BgLogger().Debug("Found tiflash store", zap.Int64("id", store.Store.ID), zap.String("Address", store.Store.Address), zap.String("StatusAddress", store.Store.StatusAddress)) - } + if engine.IsTiFlashHTTPResp(&store.Store) { + stores[store.Store.ID] = store } } *tiFlashStores = stores diff --git a/pkg/util/engine/BUILD.bazel b/pkg/util/engine/BUILD.bazel index 7803c21218e6f..c21b6957317d3 100644 --- a/pkg/util/engine/BUILD.bazel +++ b/pkg/util/engine/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "engine", @@ -10,3 +10,15 @@ go_library( "@com_github_tikv_pd_client//http", ], ) + +go_test( + name = "engine_test", + timeout = "short", + srcs = ["engine_test.go"], + embed = [":engine"], + flaky = True, + deps = [ + "@com_github_stretchr_testify//require", + "@com_github_tikv_pd_client//http", + ], +) diff --git a/pkg/util/engine/engine.go b/pkg/util/engine/engine.go index 6873385d2b5af..06800328c81c6 100644 --- a/pkg/util/engine/engine.go +++ b/pkg/util/engine/engine.go @@ -19,7 +19,7 @@ import ( pdhttp "github.com/tikv/pd/client/http" ) -// IsTiFlash tests whether the store is based on tiflash engine. +// IsTiFlash check whether the store is based on tiflash engine. func IsTiFlash(store *metapb.Store) bool { for _, label := range store.Labels { if label.Key == "engine" && (label.Value == "tiflash_compute" || label.Value == "tiflash") { diff --git a/pkg/util/engine/engine_test.go b/pkg/util/engine/engine_test.go new file mode 100644 index 0000000000000..f15c28530c7a6 --- /dev/null +++ b/pkg/util/engine/engine_test.go @@ -0,0 +1,79 @@ +// Copyright 2024 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 engine + +import ( + "testing" + + "github.com/stretchr/testify/require" + pdhttp "github.com/tikv/pd/client/http" +) + +func TestIsTiFlashHTTPResp(t *testing.T) { + tests := []struct { + name string + store *pdhttp.MetaStore + want bool + }{ + { + name: "Test with TiFlash label", + store: &pdhttp.MetaStore{ + Labels: []pdhttp.StoreLabel{ + { + Key: "engine", + Value: "tiflash", + }, + }, + }, + want: true, + }, + { + name: "Test with TiFlash label 2", + store: &pdhttp.MetaStore{ + Labels: []pdhttp.StoreLabel{ + { + Key: "engine", + Value: "tiflash_compute", + }, + }, + }, + want: true, + }, + { + name: "Test without TiFlash label", + store: &pdhttp.MetaStore{ + Labels: []pdhttp.StoreLabel{ + { + Key: "engine", + Value: "not_tiflash", + }, + }, + }, + want: false, + }, + { + name: "Test with no labels", + store: &pdhttp.MetaStore{ + Labels: []pdhttp.StoreLabel{}, + }, + want: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + require.Equal(t, tt.want, IsTiFlashHTTPResp(tt.store)) + }) + } +}