github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/kv/testing.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 kv
    15  
    16  import (
    17  	"bytes"
    18  	"context"
    19  	"fmt"
    20  	"strconv"
    21  	"strings"
    22  	"time"
    23  
    24  	"github.com/pingcap/log"
    25  	"github.com/pingcap/ticdc/cdc/model"
    26  	"github.com/pingcap/ticdc/pkg/regionspan"
    27  	"github.com/pingcap/ticdc/pkg/security"
    28  	"github.com/pingcap/ticdc/pkg/txnutil"
    29  	"github.com/pingcap/tidb/kv"
    30  	"github.com/pingcap/tidb/store"
    31  	"github.com/pingcap/tidb/store/driver"
    32  	"github.com/pingcap/tidb/store/tikv"
    33  	"github.com/stretchr/testify/require"
    34  	pd "github.com/tikv/pd/client"
    35  	"go.uber.org/zap"
    36  )
    37  
    38  var genValueID int
    39  
    40  func genValue() []byte {
    41  	genValueID++
    42  	return []byte("value_" + strconv.Itoa(genValueID))
    43  }
    44  
    45  type eventChecker struct {
    46  	t       require.TestingT
    47  	eventCh chan model.RegionFeedEvent
    48  	closeCh chan struct{}
    49  
    50  	vals        []*model.RawKVEntry
    51  	checkpoints []*model.ResolvedSpan
    52  }
    53  
    54  func valInSlice(val *model.RawKVEntry, vals []*model.RawKVEntry) bool {
    55  	for _, v := range vals {
    56  		if val.CRTs == v.CRTs && bytes.Equal(val.Key, v.Key) {
    57  			return true
    58  		}
    59  	}
    60  	return false
    61  }
    62  
    63  func newEventChecker(t require.TestingT) *eventChecker {
    64  	ec := &eventChecker{
    65  		t:       t,
    66  		eventCh: make(chan model.RegionFeedEvent),
    67  		closeCh: make(chan struct{}),
    68  	}
    69  
    70  	go func() {
    71  		for {
    72  			select {
    73  			case e := <-ec.eventCh:
    74  				log.Debug("get event", zap.Reflect("event", e))
    75  				if e.Val != nil {
    76  					// check if the value event break the checkpoint guarantee
    77  					for _, cp := range ec.checkpoints {
    78  						if !regionspan.KeyInSpan(e.Val.Key, cp.Span) ||
    79  							e.Val.CRTs > cp.ResolvedTs {
    80  							continue
    81  						}
    82  
    83  						if !valInSlice(e.Val, ec.vals) {
    84  							require.FailNowf(t, "unexpected value event", "value: %+v checkpoint: %+v", e.Val, cp)
    85  						}
    86  					}
    87  
    88  					ec.vals = append(ec.vals, e.Val)
    89  				} else {
    90  					ec.checkpoints = append(ec.checkpoints, e.Resolved)
    91  				}
    92  			case <-ec.closeCh:
    93  				return
    94  			}
    95  		}
    96  	}()
    97  
    98  	return ec
    99  }
   100  
   101  // stop the checker
   102  func (ec *eventChecker) stop() {
   103  	close(ec.closeCh)
   104  }
   105  
   106  // CreateStorage creates a tikv Storage instance.
   107  func CreateStorage(pdAddr string) (storage kv.Storage, err error) {
   108  	tiPath := fmt.Sprintf("tikv://%s?disableGC=true", pdAddr)
   109  	err = store.Register("tikv", driver.TiKVDriver{})
   110  	if err != nil && !strings.Contains(err.Error(), "already registered") {
   111  		return
   112  	}
   113  	storage, err = store.New(tiPath)
   114  	return
   115  }
   116  
   117  func mustGetTimestamp(t require.TestingT, storage tikv.Storage) uint64 {
   118  	ts, err := storage.GetOracle().GetTimestamp(context.Background(), nil)
   119  	require.NoError(t, err)
   120  
   121  	return ts
   122  }
   123  
   124  func mustGetValue(t require.TestingT, eventCh <-chan model.RegionFeedEvent, value []byte) {
   125  	timeout := time.After(time.Second * 20)
   126  
   127  	for {
   128  		select {
   129  		case e := <-eventCh:
   130  			if e.Val != nil && bytes.Equal(e.Val.Value, value) {
   131  				return
   132  			}
   133  		case <-timeout:
   134  			require.FailNowf(t, "timeout to get value", "value: %v", value)
   135  		}
   136  	}
   137  }
   138  
   139  type mockPullerInit struct{}
   140  
   141  func (*mockPullerInit) IsInitialized() bool {
   142  	return true
   143  }
   144  
   145  // TestSplit try split on every region, and test can get value event from
   146  // every region after split.
   147  func TestSplit(t require.TestingT, pdCli pd.Client, storage tikv.Storage, kvStore kv.Storage) {
   148  	eventCh := make(chan model.RegionFeedEvent, 1<<20)
   149  	ctx, cancel := context.WithCancel(context.Background())
   150  	defer cancel()
   151  
   152  	grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{})
   153  	cli := NewCDCClient(context.Background(), pdCli, storage, grpcPool)
   154  	defer cli.Close()
   155  
   156  	startTS := mustGetTimestamp(t, storage)
   157  
   158  	lockresolver := txnutil.NewLockerResolver(storage)
   159  	isPullInit := &mockPullerInit{}
   160  	go func() {
   161  		err := cli.EventFeed(ctx, regionspan.ComparableSpan{Start: nil, End: nil}, startTS, false, lockresolver, isPullInit, eventCh)
   162  		require.Equal(t, err, context.Canceled)
   163  	}()
   164  
   165  	mockTableID := int64(999)
   166  	preRegions, err := pdCli.ScanRegions(context.Background(), nil, nil, 10000)
   167  	require.NoError(t, err)
   168  
   169  	for i := 0; i < 2; i++ {
   170  		regions := preRegions
   171  		// In second loop try split every region.
   172  		if i == 1 {
   173  			splitStore, ok := storage.(kv.SplittableStore)
   174  			require.True(t, ok)
   175  			for _, r := range preRegions {
   176  				splitKey := r.Meta.GetStartKey()
   177  				if len(splitKey) == 0 {
   178  					splitKey = []byte{0}
   179  				} else {
   180  					splitKey = append(splitKey, 0)
   181  				}
   182  				splitKeys := [][]byte{splitKey}
   183  				_, err := splitStore.SplitRegions(context.Background(), splitKeys, false, &mockTableID)
   184  				require.NoError(t, err)
   185  			}
   186  
   187  			time.Sleep(time.Second * 3)
   188  
   189  			var afterRegions []*pd.Region
   190  			afterRegions, err = pdCli.ScanRegions(context.Background(), nil, nil, 10000)
   191  			require.NoError(t, err)
   192  			require.Greater(t, len(afterRegions), len(preRegions))
   193  
   194  			regions = afterRegions
   195  		}
   196  
   197  		// Put a key on every region and check we can get the event.
   198  		for _, r := range regions {
   199  			key := r.Meta.GetStartKey()
   200  			if len(key) == 0 {
   201  				key = []byte{0}
   202  			}
   203  			value := genValue()
   204  
   205  			var tx kv.Transaction
   206  			tx, err = kvStore.Begin()
   207  			require.NoError(t, err)
   208  			err = tx.Set(key, value)
   209  			require.NoError(t, err)
   210  			err = tx.Commit(ctx)
   211  			require.NoError(t, err)
   212  
   213  			mustGetValue(t, eventCh, value)
   214  		}
   215  	}
   216  }
   217  
   218  func mustSetKey(t require.TestingT, storage kv.Storage, key []byte, value []byte) {
   219  	tx, err := storage.Begin()
   220  	require.NoError(t, err)
   221  	err = tx.Set(key, value)
   222  	require.NoError(t, err)
   223  	err = tx.Commit(context.Background())
   224  	require.NoError(t, err)
   225  }
   226  
   227  func mustDeleteKey(t require.TestingT, storage kv.Storage, key []byte) {
   228  	tx, err := storage.Begin()
   229  	require.NoError(t, err)
   230  	err = tx.Delete(key)
   231  	require.NoError(t, err)
   232  	err = tx.Commit(context.Background())
   233  	require.NoError(t, err)
   234  }
   235  
   236  // TestGetKVSimple test simple KV operations
   237  func TestGetKVSimple(t require.TestingT, pdCli pd.Client, storage tikv.Storage, kvStore kv.Storage) {
   238  	checker := newEventChecker(t)
   239  	ctx, cancel := context.WithCancel(context.Background())
   240  	defer cancel()
   241  
   242  	grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{})
   243  	cli := NewCDCClient(context.Background(), pdCli, storage, grpcPool)
   244  	defer cli.Close()
   245  
   246  	startTS := mustGetTimestamp(t, storage)
   247  	lockresolver := txnutil.NewLockerResolver(storage)
   248  	isPullInit := &mockPullerInit{}
   249  	go func() {
   250  		err := cli.EventFeed(ctx, regionspan.ComparableSpan{Start: nil, End: nil}, startTS, false, lockresolver, isPullInit, checker.eventCh)
   251  		require.Equal(t, err, context.Canceled)
   252  	}()
   253  
   254  	key := []byte("s1")
   255  	value := []byte("s1v")
   256  
   257  	// set
   258  	mustSetKey(t, kvStore, key, value)
   259  
   260  	// delete
   261  	mustDeleteKey(t, kvStore, key)
   262  
   263  	// set again
   264  	mustSetKey(t, kvStore, key, value)
   265  
   266  	for i := 0; i < 2; i++ {
   267  		// start a new EventFeed with the startTS before the kv operations should also get the same events.
   268  		// This can test the initialize case.
   269  		if i == 1 {
   270  			checker = newEventChecker(t)
   271  			go func() {
   272  				err := cli.EventFeed(ctx, regionspan.ComparableSpan{Start: nil, End: nil}, startTS, false, lockresolver, isPullInit, checker.eventCh)
   273  				require.Equal(t, err, context.Canceled)
   274  			}()
   275  		}
   276  
   277  		time.Sleep(5 * time.Second)
   278  		checker.stop()
   279  
   280  		// filter the unrelated keys event.
   281  		var vals []*model.RawKVEntry
   282  		for _, v := range checker.vals {
   283  			if bytes.Equal(v.Key, key) {
   284  				vals = append(vals, v)
   285  			}
   286  		}
   287  		checker.vals = vals
   288  
   289  		// check we can get the events.
   290  		require.Len(t, checker.vals, 3)
   291  		require.Equal(t, checker.vals[0].OpType, model.OpTypePut)
   292  		require.Equal(t, checker.vals[0].Key, key)
   293  		require.Equal(t, checker.vals[0].Value, value)
   294  
   295  		require.Equal(t, checker.vals[1].OpType, model.OpTypeDelete)
   296  		require.Equal(t, checker.vals[1].Key, key)
   297  
   298  		require.Equal(t, checker.vals[2].OpType, model.OpTypePut)
   299  		require.Equal(t, checker.vals[2].Key, key)
   300  		require.Equal(t, checker.vals[2].Value, value)
   301  	}
   302  }