github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/checker/lightning.go (about)

     1  // Copyright 2022 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package checker
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  
    20  	"github.com/docker/go-units"
    21  	"github.com/pingcap/tidb/lightning/pkg/importer"
    22  	"github.com/pingcap/tidb/lightning/pkg/precheck"
    23  	"github.com/pingcap/tiflow/dm/pkg/log"
    24  )
    25  
    26  func convertLightningPrecheck(
    27  	ctx context.Context,
    28  	dmResult *Result,
    29  	lightningPrechecker precheck.Checker,
    30  	failLevel State,
    31  	instruction string,
    32  ) {
    33  	lightningResult, err := lightningPrechecker.Check(ctx)
    34  	if err != nil {
    35  		markCheckError(dmResult, err)
    36  		return
    37  	}
    38  	if !lightningResult.Passed {
    39  		dmResult.State = failLevel
    40  		dmResult.Instruction = instruction
    41  		dmResult.Errors = append(dmResult.Errors, &Error{Severity: failLevel, ShortErr: lightningResult.Message})
    42  		return
    43  	}
    44  	dmResult.State = StateSuccess
    45  }
    46  
    47  // LightningEmptyRegionChecker checks whether there are too many empty regions in the cluster.
    48  type LightningEmptyRegionChecker struct {
    49  	inner precheck.Checker
    50  }
    51  
    52  // NewLightningEmptyRegionChecker creates a new LightningEmptyRegionChecker.
    53  func NewLightningEmptyRegionChecker(lightningChecker precheck.Checker) RealChecker {
    54  	return &LightningEmptyRegionChecker{inner: lightningChecker}
    55  }
    56  
    57  // Name implements the RealChecker interface.
    58  func (c *LightningEmptyRegionChecker) Name() string {
    59  	return "lightning_empty_region"
    60  }
    61  
    62  // Check implements the RealChecker interface.
    63  func (c *LightningEmptyRegionChecker) Check(ctx context.Context) *Result {
    64  	result := &Result{
    65  		Name:  c.Name(),
    66  		Desc:  "check whether there are too many empty regions in the TiKV under physical import mode",
    67  		State: StateFailure,
    68  	}
    69  	convertLightningPrecheck(
    70  		ctx,
    71  		result,
    72  		c.inner,
    73  		StateWarning,
    74  		`you can change "region merge" related configuration in PD to speed up eliminating empty regions`,
    75  	)
    76  	return result
    77  }
    78  
    79  // LightningRegionDistributionChecker checks whether the region distribution is balanced.
    80  type LightningRegionDistributionChecker struct {
    81  	inner precheck.Checker
    82  }
    83  
    84  // NewLightningRegionDistributionChecker creates a new LightningRegionDistributionChecker.
    85  func NewLightningRegionDistributionChecker(lightningChecker precheck.Checker) RealChecker {
    86  	return &LightningRegionDistributionChecker{inner: lightningChecker}
    87  }
    88  
    89  // Name implements the RealChecker interface.
    90  func (c *LightningRegionDistributionChecker) Name() string {
    91  	return "lightning_region_distribution"
    92  }
    93  
    94  // Check implements the RealChecker interface.
    95  func (c *LightningRegionDistributionChecker) Check(ctx context.Context) *Result {
    96  	result := &Result{
    97  		Name:  c.Name(),
    98  		Desc:  "check whether the Regions in the TiKV cluster are distributed evenly under physical import mode",
    99  		State: StateFailure,
   100  	}
   101  	convertLightningPrecheck(
   102  		ctx,
   103  		result,
   104  		c.inner,
   105  		StateWarning,
   106  		`you can change "region schedule" related configuration in PD to speed up balancing regions`,
   107  	)
   108  	return result
   109  }
   110  
   111  // LightningClusterVersionChecker checks whether the cluster version is compatible with Lightning.
   112  type LightningClusterVersionChecker struct {
   113  	inner precheck.Checker
   114  }
   115  
   116  // NewLightningClusterVersionChecker creates a new LightningClusterVersionChecker.
   117  func NewLightningClusterVersionChecker(lightningChecker precheck.Checker) RealChecker {
   118  	return &LightningClusterVersionChecker{inner: lightningChecker}
   119  }
   120  
   121  // Name implements the RealChecker interface.
   122  func (c *LightningClusterVersionChecker) Name() string {
   123  	return "lightning_cluster_version"
   124  }
   125  
   126  // Check implements the RealChecker interface.
   127  func (c *LightningClusterVersionChecker) Check(ctx context.Context) *Result {
   128  	result := &Result{
   129  		Name:  c.Name(),
   130  		Desc:  "check whether the downstream TiDB/PD/TiKV version meets the requirements of physical import mode",
   131  		State: StateFailure,
   132  	}
   133  	convertLightningPrecheck(
   134  		ctx,
   135  		result,
   136  		c.inner,
   137  		StateFailure,
   138  		`you can switch to logical import mode which has no requirements on downstream cluster version`,
   139  	)
   140  	return result
   141  }
   142  
   143  // LightningFreeSpaceChecker checks whether the cluster has enough free space.
   144  type LightningFreeSpaceChecker struct {
   145  	sourceDataSize int64
   146  	infoGetter     importer.TargetInfoGetter
   147  }
   148  
   149  // NewLightningFreeSpaceChecker creates a new LightningFreeSpaceChecker.
   150  func NewLightningFreeSpaceChecker(sourceDataSize int64, getter importer.TargetInfoGetter) RealChecker {
   151  	return &LightningFreeSpaceChecker{
   152  		sourceDataSize: sourceDataSize,
   153  		infoGetter:     getter,
   154  	}
   155  }
   156  
   157  // Name implements the RealChecker interface.
   158  func (c *LightningFreeSpaceChecker) Name() string {
   159  	return "lightning_free_space"
   160  }
   161  
   162  // Check implements the RealChecker interface.
   163  func (c *LightningFreeSpaceChecker) Check(ctx context.Context) *Result {
   164  	result := &Result{
   165  		Name:  c.Name(),
   166  		Desc:  "check whether the downstream has enough free space to store the data to be migrated",
   167  		State: StateFailure,
   168  	}
   169  	storeInfo, err := c.infoGetter.GetStorageInfo(ctx)
   170  	if err != nil {
   171  		markCheckError(result, err)
   172  		return result
   173  	}
   174  	var (
   175  		clusterAvail uint64
   176  		avail        int64
   177  	)
   178  	for _, store := range storeInfo.Stores {
   179  		avail, err = units.RAMInBytes(store.Status.Available)
   180  		if err != nil {
   181  			markCheckError(result, err)
   182  			return result
   183  		}
   184  		clusterAvail += uint64(avail)
   185  	}
   186  	if clusterAvail < uint64(c.sourceDataSize) {
   187  		result.State = StateFailure
   188  		result.Errors = append(result.Errors, &Error{
   189  			Severity: StateFailure,
   190  			ShortErr: fmt.Sprintf("Downstream doesn't have enough space, available is %s, but we need %s",
   191  				units.BytesSize(float64(clusterAvail)), units.BytesSize(float64(c.sourceDataSize))),
   192  		})
   193  		result.Instruction = "you can try to scale-out TiKV storage or TiKV instance to gain more storage space"
   194  		return result
   195  	}
   196  
   197  	maxReplicas, err := c.infoGetter.GetMaxReplica(ctx)
   198  	if err != nil {
   199  		markCheckError(result, err)
   200  		return result
   201  	}
   202  	safeSize := uint64(c.sourceDataSize) * maxReplicas * 2
   203  	if clusterAvail < safeSize {
   204  		result.State = StateWarning
   205  		result.Errors = append(result.Errors, &Error{
   206  			Severity: StateWarning,
   207  			ShortErr: fmt.Sprintf("Cluster may not have enough space, available is %s, but we need %s",
   208  				units.BytesSize(float64(clusterAvail)), units.BytesSize(float64(safeSize))),
   209  		})
   210  		result.Instruction = "you can try to scale-out TiKV storage or TiKV instance to gain more storage space"
   211  		return result
   212  	}
   213  	result.State = StateSuccess
   214  	return result
   215  }
   216  
   217  // LightningCDCPiTRChecker checks whether the cluster has running CDC PiTR tasks.
   218  type LightningCDCPiTRChecker struct {
   219  	inner precheck.Checker
   220  }
   221  
   222  // NewLightningCDCPiTRChecker creates a new LightningCDCPiTRChecker.
   223  func NewLightningCDCPiTRChecker(lightningChecker precheck.Checker) RealChecker {
   224  	c, ok := lightningChecker.(*importer.CDCPITRCheckItem)
   225  	if ok {
   226  		c.Instruction = "physical import mode is not compatible with them. Please switch to logical import mode then try again."
   227  	} else {
   228  		log.L().DPanic("lightningChecker is not CDCPITRCheckItem")
   229  	}
   230  	return &LightningCDCPiTRChecker{inner: lightningChecker}
   231  }
   232  
   233  // Name implements the RealChecker interface.
   234  func (c *LightningCDCPiTRChecker) Name() string {
   235  	return "lightning_downstream_mutex_features"
   236  }
   237  
   238  // Check implements the RealChecker interface.
   239  func (c *LightningCDCPiTRChecker) Check(ctx context.Context) *Result {
   240  	result := &Result{
   241  		Name:  c.Name(),
   242  		Desc:  "check whether the downstream has tasks incompatible with physical import mode",
   243  		State: StateFailure,
   244  	}
   245  	convertLightningPrecheck(
   246  		ctx,
   247  		result,
   248  		c.inner,
   249  		StateFailure,
   250  		`you can switch to logical import mode which has no requirements on this`,
   251  	)
   252  	return result
   253  }
   254  
   255  // LightningTableEmptyChecker checks whether the cluster's target table is empty.
   256  type LightningTableEmptyChecker struct {
   257  	inner precheck.Checker
   258  }
   259  
   260  // NewLightningEmptyTableChecker creates a new LightningEmptyTableChecker.
   261  func NewLightningEmptyTableChecker(lightningChecker precheck.Checker) RealChecker {
   262  	return &LightningTableEmptyChecker{inner: lightningChecker}
   263  }
   264  
   265  // Name implements the RealChecker interface.
   266  func (c *LightningTableEmptyChecker) Name() string {
   267  	return "lightning_downstream_empty_table"
   268  }
   269  
   270  // Check implements the RealChecker interface.
   271  func (c *LightningTableEmptyChecker) Check(ctx context.Context) *Result {
   272  	result := &Result{
   273  		Name:  c.Name(),
   274  		Desc:  "check whether the downstream table not empty which is not compatible with physical import mode",
   275  		State: StateFailure,
   276  	}
   277  	convertLightningPrecheck(
   278  		ctx,
   279  		result,
   280  		c.inner,
   281  		StateFailure,
   282  		`you can switch to logical import mode which has no requirements on this`,
   283  	)
   284  	return result
   285  }