github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/tests/integration_tests/resolve_lock/main.go (about)

     1  // Copyright 2020 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 main
    15  
    16  import (
    17  	"bytes"
    18  	"context"
    19  	"database/sql"
    20  	"encoding/json"
    21  	"flag"
    22  	"fmt"
    23  	"io"
    24  	"math/rand"
    25  	"net"
    26  	"net/http"
    27  	"os"
    28  	"strings"
    29  	"time"
    30  
    31  	"github.com/pingcap/errors"
    32  	"github.com/pingcap/kvproto/pkg/kvrpcpb"
    33  	"github.com/pingcap/log"
    34  	"github.com/pingcap/tidb/pkg/kv"
    35  	"github.com/pingcap/tidb/pkg/parser/model"
    36  	"github.com/pingcap/tidb/pkg/store/driver"
    37  	"github.com/pingcap/tidb/pkg/tablecodec"
    38  	"github.com/pingcap/tiflow/tests/integration_tests/util"
    39  	"github.com/tikv/client-go/v2/oracle"
    40  	"github.com/tikv/client-go/v2/tikv"
    41  	"github.com/tikv/client-go/v2/tikvrpc"
    42  	pd "github.com/tikv/pd/client"
    43  )
    44  
    45  func main() {
    46  	cfg := util.NewConfig()
    47  	err := cfg.Parse(os.Args[1:])
    48  	switch errors.Cause(err) {
    49  	case nil:
    50  	case flag.ErrHelp:
    51  		os.Exit(0)
    52  	default:
    53  		log.S().Errorf("parse cmd flags err %s\n", err)
    54  		os.Exit(2)
    55  	}
    56  
    57  	sourceDB, err := util.CreateDB(cfg.SourceDBCfg[0])
    58  	if err != nil {
    59  		log.S().Fatal(err)
    60  	}
    61  	defer func() {
    62  		if err := util.CloseDB(sourceDB); err != nil {
    63  			log.S().Errorf("Failed to close source database: %s\n", err)
    64  		}
    65  	}()
    66  	if err := prepare(sourceDB); err != nil {
    67  		log.S().Fatal(err)
    68  	}
    69  	ctx, cancel := context.WithCancel(context.Background())
    70  	defer cancel()
    71  	if err := addLock(ctx, cfg); err != nil {
    72  		log.S().Fatal(err)
    73  	}
    74  	time.Sleep(5 * time.Second)
    75  	if err := finishMark(sourceDB); err != nil {
    76  		log.S().Fatal(err)
    77  	}
    78  }
    79  
    80  func prepare(sourceDB *sql.DB) error {
    81  	sqls := []string{
    82  		"use test;",
    83  		"create table t1 (a int primary key);",
    84  	}
    85  	for _, sql := range sqls {
    86  		_, err := sourceDB.Exec(sql)
    87  		if err != nil {
    88  			return errors.Trace(err)
    89  		}
    90  	}
    91  	return nil
    92  }
    93  
    94  func finishMark(sourceDB *sql.DB) error {
    95  	sqls := []string{
    96  		"use test;",
    97  		"insert into t1 value (1);",
    98  		"create table t2 (a int primary key);",
    99  	}
   100  	for _, sql := range sqls {
   101  		_, err := sourceDB.Exec(sql)
   102  		if err != nil {
   103  			return errors.Trace(err)
   104  		}
   105  	}
   106  	return nil
   107  }
   108  
   109  func addLock(ctx context.Context, cfg *util.Config) error {
   110  	http.DefaultClient.Timeout = 10 * time.Second
   111  
   112  	tableID, err := getTableID(cfg.SourceDBCfg[0].Host, "test", "t1")
   113  	if err != nil {
   114  		return errors.Trace(err)
   115  	}
   116  
   117  	pdcli, err := pd.NewClientWithContext(
   118  		ctx, strings.Split(cfg.PDAddr, ","), pd.SecurityOption{})
   119  	if err != nil {
   120  		return errors.Trace(err)
   121  	}
   122  	defer pdcli.Close()
   123  
   124  	driver := driver.TiKVDriver{}
   125  	store, err := driver.Open(fmt.Sprintf("tikv://%s?disableGC=true", cfg.PDAddr))
   126  	if err != nil {
   127  		return errors.Trace(err)
   128  	}
   129  
   130  	locker := Locker{
   131  		tableID:   tableID,
   132  		tableSize: 1000,
   133  		lockTTL:   10 * time.Second,
   134  		pdcli:     pdcli,
   135  		kv:        store.(tikv.Storage),
   136  	}
   137  	return errors.Trace(locker.generateLocks(ctx, 10*time.Second))
   138  }
   139  
   140  // getTableID of the table with specified table name.
   141  func getTableID(dbAddr, dbName, table string) (int64, error) {
   142  	dbStatusAddr := net.JoinHostPort(dbAddr, "10080")
   143  	url := fmt.Sprintf("http://%s/schema/%s/%s", dbStatusAddr, dbName, table)
   144  
   145  	resp, err := http.Get(url) // #nosec G107
   146  	if err != nil {
   147  		return 0, errors.Trace(err)
   148  	}
   149  	defer resp.Body.Close()
   150  
   151  	body, err := io.ReadAll(resp.Body)
   152  	if err != nil {
   153  		return 0, errors.Trace(err)
   154  	}
   155  
   156  	if resp.StatusCode != 200 {
   157  		return 0, errors.Errorf("HTTP request to TiDB status reporter returns %v. Body: %v", resp.StatusCode, string(body))
   158  	}
   159  
   160  	var data model.TableInfo
   161  	err = json.Unmarshal(body, &data)
   162  	if err != nil {
   163  		return 0, errors.Trace(err)
   164  	}
   165  	return data.ID, nil
   166  }
   167  
   168  // Locker leaves locks on a table.
   169  type Locker struct {
   170  	tableID   int64
   171  	tableSize int64
   172  	lockTTL   time.Duration
   173  
   174  	pdcli pd.Client
   175  	kv    tikv.Storage
   176  }
   177  
   178  // generateLocks sends Prewrite requests to TiKV to generate locks, without committing and rolling back.
   179  func (c *Locker) generateLocks(ctx context.Context, genDur time.Duration) error {
   180  	log.Info("genLock started")
   181  
   182  	const maxTxnSize = 1000
   183  
   184  	// How many keys should be in the next transaction.
   185  	nextTxnSize := rand.Intn(maxTxnSize) + 1 // 0 is not allowed.
   186  
   187  	// How many keys has been scanned since last time sending request.
   188  	scannedKeys := 0
   189  	var batch []int64
   190  
   191  	// Send lock for 10 seconds
   192  	timer := time.After(genDur)
   193  	for rowID := int64(0); ; rowID = (rowID + 1) % c.tableSize {
   194  		select {
   195  		case <-timer:
   196  			log.Info("genLock done")
   197  			return nil
   198  		default:
   199  		}
   200  
   201  		scannedKeys++
   202  
   203  		// Randomly decide whether to lock current key.
   204  		lockThis := rand.Intn(2) == 0
   205  
   206  		if lockThis {
   207  			batch = append(batch, rowID)
   208  
   209  			if len(batch) >= nextTxnSize {
   210  				// The batch is large enough to start the transaction
   211  				err := c.lockKeys(ctx, batch)
   212  				if err != nil {
   213  					return errors.Annotate(err, "lock keys failed")
   214  				}
   215  
   216  				// Start the next loop
   217  				batch = batch[:0]
   218  				scannedKeys = 0
   219  				nextTxnSize = rand.Intn(maxTxnSize) + 1
   220  			}
   221  		}
   222  	}
   223  }
   224  
   225  func (c *Locker) lockKeys(ctx context.Context, rowIDs []int64) error {
   226  	keys := make([][]byte, 0, len(rowIDs))
   227  
   228  	keyPrefix := tablecodec.GenTableRecordPrefix(c.tableID)
   229  	for _, rowID := range rowIDs {
   230  		key := tablecodec.EncodeRecordKey(keyPrefix, kv.IntHandle(rowID))
   231  		keys = append(keys, key)
   232  	}
   233  
   234  	primary := keys[0]
   235  
   236  	for len(keys) > 0 {
   237  		lockedKeys, err := c.lockBatch(ctx, keys, primary)
   238  		if err != nil {
   239  			return errors.Trace(err)
   240  		}
   241  		keys = keys[lockedKeys:]
   242  	}
   243  	return nil
   244  }
   245  
   246  func (c *Locker) lockBatch(ctx context.Context, keys [][]byte, primary []byte) (int, error) {
   247  	const maxBatchSize = 16 * 1024
   248  
   249  	// TiKV client doesn't expose Prewrite interface directly. We need to manually locate the region and send the
   250  	// Prewrite requests.
   251  
   252  	bo := tikv.NewBackoffer(ctx, 20000)
   253  	for {
   254  		loc, err := c.kv.GetRegionCache().LocateKey(bo, keys[0])
   255  		if err != nil {
   256  			return 0, errors.Trace(err)
   257  		}
   258  
   259  		// Get a timestamp to use as the startTs
   260  		physical, logical, err := c.pdcli.GetTS(ctx)
   261  		if err != nil {
   262  			return 0, errors.Trace(err)
   263  		}
   264  		startTs := oracle.ComposeTS(physical, logical)
   265  
   266  		// Pick a batch of keys and make up the mutations
   267  		var mutations []*kvrpcpb.Mutation
   268  		batchSize := 0
   269  
   270  		for _, key := range keys {
   271  			if len(loc.EndKey) > 0 && bytes.Compare(key, loc.EndKey) >= 0 {
   272  				break
   273  			}
   274  			if bytes.Compare(key, loc.StartKey) < 0 {
   275  				break
   276  			}
   277  
   278  			value := randStr()
   279  			mutations = append(mutations, &kvrpcpb.Mutation{
   280  				Op:    kvrpcpb.Op_Put,
   281  				Key:   key,
   282  				Value: []byte(value),
   283  			})
   284  			batchSize += len(key) + len(value)
   285  
   286  			if batchSize >= maxBatchSize {
   287  				break
   288  			}
   289  		}
   290  
   291  		lockedKeys := len(mutations)
   292  		if lockedKeys == 0 {
   293  			return 0, nil
   294  		}
   295  
   296  		prewrite := &kvrpcpb.PrewriteRequest{
   297  			Mutations:    mutations,
   298  			PrimaryLock:  primary,
   299  			StartVersion: startTs,
   300  			LockTtl:      uint64(c.lockTTL.Milliseconds()),
   301  		}
   302  		req := tikvrpc.NewRequest(tikvrpc.CmdPrewrite, prewrite)
   303  
   304  		// Send the requests
   305  		resp, err := c.kv.SendReq(bo, req, loc.Region, time.Second*20)
   306  		if err != nil {
   307  			return 0, errors.Annotatef(
   308  				err,
   309  				"send request failed. region: %+v [%+q, %+q), keys: %+q",
   310  				loc.Region, loc.StartKey, loc.EndKey, keys[0:lockedKeys])
   311  		}
   312  		regionErr, err := resp.GetRegionError()
   313  		if err != nil {
   314  			return 0, errors.Trace(err)
   315  		}
   316  		if regionErr != nil {
   317  			err = bo.Backoff(tikv.BoRegionMiss(), errors.New(regionErr.String()))
   318  			if err != nil {
   319  				return 0, errors.Trace(err)
   320  			}
   321  			continue
   322  		}
   323  
   324  		prewriteResp := resp.Resp
   325  		if prewriteResp == nil {
   326  			return 0, errors.Errorf("response body missing")
   327  		}
   328  
   329  		// Ignore key errors since we never commit the transaction and we don't need to keep consistency here.
   330  		return lockedKeys, nil
   331  	}
   332  }
   333  
   334  func randStr() string {
   335  	length := rand.Intn(128)
   336  	res := ""
   337  	for i := 0; i < length; i++ {
   338  		res += fmt.Sprintf("a%d", rand.Intn(26))
   339  	}
   340  	return res
   341  }