github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/kv/client_bench_test.go (about)

     1  // Copyright 2021 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  	"context"
    18  	"fmt"
    19  	"net"
    20  	"sync"
    21  	"testing"
    22  	"time"
    23  
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/kvproto/pkg/cdcpb"
    26  	"github.com/pingcap/log"
    27  	"github.com/pingcap/ticdc/cdc/model"
    28  	"github.com/pingcap/ticdc/pkg/regionspan"
    29  	"github.com/pingcap/ticdc/pkg/retry"
    30  	"github.com/pingcap/ticdc/pkg/security"
    31  	"github.com/pingcap/ticdc/pkg/txnutil"
    32  	"github.com/pingcap/tidb/store/mockstore/mocktikv"
    33  	"github.com/pingcap/tidb/store/tikv"
    34  	"github.com/pingcap/tidb/store/tikv/oracle"
    35  	"go.uber.org/zap/zapcore"
    36  	"google.golang.org/grpc"
    37  )
    38  
    39  const batchResolvedSize = 100
    40  
    41  type mockChangeDataService2 struct {
    42  	b        *testing.B
    43  	ch       chan *cdcpb.ChangeDataEvent
    44  	recvLoop func(server cdcpb.ChangeData_EventFeedServer)
    45  }
    46  
    47  func newMockChangeDataService2(b *testing.B, ch chan *cdcpb.ChangeDataEvent) *mockChangeDataService2 {
    48  	s := &mockChangeDataService2{
    49  		b:  b,
    50  		ch: ch,
    51  	}
    52  	return s
    53  }
    54  
    55  func (s *mockChangeDataService2) EventFeed(server cdcpb.ChangeData_EventFeedServer) error {
    56  	if s.recvLoop != nil {
    57  		go func() {
    58  			s.recvLoop(server)
    59  		}()
    60  	}
    61  	for e := range s.ch {
    62  		if e == nil {
    63  			break
    64  		}
    65  		err := server.Send(e)
    66  		if err != nil {
    67  			s.b.Error(err)
    68  		}
    69  	}
    70  	return nil
    71  }
    72  
    73  func newMockService2(
    74  	ctx context.Context,
    75  	b *testing.B,
    76  	srv cdcpb.ChangeDataServer,
    77  	wg *sync.WaitGroup,
    78  ) (grpcServer *grpc.Server, addr string) {
    79  	lc := &net.ListenConfig{}
    80  	listenAddr := "127.0.0.1:0"
    81  	lis, err := lc.Listen(ctx, "tcp", listenAddr)
    82  	if err != nil {
    83  		b.Error(err)
    84  	}
    85  	addr = lis.Addr().String()
    86  	grpcServer = grpc.NewServer()
    87  	cdcpb.RegisterChangeDataServer(grpcServer, srv)
    88  	wg.Add(1)
    89  	go func() {
    90  		err := grpcServer.Serve(lis)
    91  		if err != nil {
    92  			b.Error(err)
    93  		}
    94  		wg.Done()
    95  	}()
    96  	return
    97  }
    98  
    99  func prepareBenchMultiStore(b *testing.B, storeNum, regionNum int) (
   100  	*sync.Map, /* regionID -> requestID/storeID */
   101  	*sync.WaitGroup, /* ensure eventfeed routine exit */
   102  	context.CancelFunc, /* cancle both mock server and cdc kv client */
   103  	chan model.RegionFeedEvent, /* kv client output channel */
   104  	[]chan *cdcpb.ChangeDataEvent, /* mock server data channels */
   105  ) {
   106  	ctx, cancel := context.WithCancel(context.Background())
   107  	wg := &sync.WaitGroup{}
   108  	requestIDs := new(sync.Map)
   109  
   110  	servers := make([]*grpc.Server, storeNum)
   111  	inputs := make([]chan *cdcpb.ChangeDataEvent, storeNum)
   112  	addrs := make([]string, storeNum)
   113  	for i := 0; i < storeNum; i++ {
   114  		mockSrvCh := make(chan *cdcpb.ChangeDataEvent, 100000)
   115  		srv := newMockChangeDataService2(b, mockSrvCh)
   116  		srv.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) {
   117  			for {
   118  				req, err := server.Recv()
   119  				if err != nil {
   120  					return
   121  				}
   122  				requestIDs.Store(req.RegionId, req.RequestId)
   123  			}
   124  		}
   125  		server, addr := newMockService2(ctx, b, srv, wg)
   126  		servers[i] = server
   127  		inputs[i] = mockSrvCh
   128  		addrs[i] = addr
   129  	}
   130  
   131  	for i := 0; i < storeNum; i++ {
   132  		wg.Add(1)
   133  		i := i
   134  		go func() {
   135  			defer wg.Done()
   136  			<-ctx.Done()
   137  			close(inputs[i])
   138  			servers[i].Stop()
   139  		}()
   140  	}
   141  
   142  	rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("")
   143  	if err != nil {
   144  		b.Error(err)
   145  	}
   146  	pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen}
   147  	tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0)
   148  	if err != nil {
   149  		b.Error(err)
   150  	}
   151  	kvStorage := newStorageWithCurVersionCache(tiStore, addrs[0])
   152  	defer kvStorage.Close() //nolint:errcheck
   153  
   154  	// we set each region has `storeNum` peers
   155  	regionID := uint64(1_000_000)
   156  	peers := make([]uint64, storeNum)
   157  	stores := make([]uint64, storeNum)
   158  	for i := 0; i < storeNum; i++ {
   159  		peers[i] = uint64(100_000_000*i) + regionID
   160  		stores[i] = uint64(i + 1)
   161  		cluster.AddStore(uint64(i+1), addrs[i])
   162  	}
   163  	// bootstrap cluster with the first region
   164  	cluster.Bootstrap(regionID, stores, peers, peers[0])
   165  	for i := 0; i < storeNum; i++ {
   166  		// first region of stores except for the first store
   167  		if i > 0 {
   168  			regionID += 1
   169  			peers := make([]uint64, storeNum)
   170  			for j := 0; j < storeNum; j++ {
   171  				peers[j] = uint64(100_000_000*j) + regionID
   172  			}
   173  			// peers[i] is the leader peer, locates in store with index i(storeID=i+1)
   174  			cluster.SplitRaw(regionID-1, regionID, []byte(fmt.Sprintf("a%d", regionID)), peers, peers[i])
   175  		}
   176  		// regions following, split from its previous region
   177  		for j := 1; j < regionNum; j++ {
   178  			regionID += 1
   179  			peers := make([]uint64, storeNum)
   180  			for k := 0; k < storeNum; k++ {
   181  				peers[k] = uint64(100_000_000*k) + regionID
   182  			}
   183  			// peers[i] is the leader peer, locates in store with index i(storeID=i+1)
   184  			cluster.SplitRaw(regionID-1, regionID, []byte(fmt.Sprintf("a%d", regionID)), peers, peers[i])
   185  		}
   186  	}
   187  
   188  	lockresolver := txnutil.NewLockerResolver(kvStorage)
   189  	isPullInit := &mockPullerInit{}
   190  	grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{})
   191  	defer grpcPool.Close()
   192  	cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool)
   193  	eventCh := make(chan model.RegionFeedEvent, 1000000)
   194  	wg.Add(1)
   195  	go func() {
   196  		err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh)
   197  		if errors.Cause(err) != context.Canceled {
   198  			b.Error(err)
   199  		}
   200  		cdcClient.Close() //nolint:errcheck
   201  		wg.Done()
   202  	}()
   203  
   204  	// wait all regions requested from cdc kv client
   205  	err = retry.Do(context.Background(), func() error {
   206  		count := 0
   207  		requestIDs.Range(func(_, _ interface{}) bool {
   208  			count++
   209  			return true
   210  		})
   211  		if count == regionNum*storeNum {
   212  			return nil
   213  		}
   214  		return errors.Errorf("region number %d is not as expected %d", count, regionNum)
   215  	}, retry.WithBackoffBaseDelay(500), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(20))
   216  	if err != nil {
   217  		b.Error(err)
   218  	}
   219  
   220  	return requestIDs, wg, cancel, eventCh, inputs
   221  }
   222  
   223  func prepareBench(b *testing.B, regionNum int) (
   224  	*sync.Map, /* regionID -> requestID */
   225  	*sync.WaitGroup, /* ensure eventfeed routine exit */
   226  	context.CancelFunc, /* cancle both mock server and cdc kv client */
   227  	chan model.RegionFeedEvent, /* kv client output channel */
   228  	chan *cdcpb.ChangeDataEvent, /* mock server data channel */
   229  ) {
   230  	ctx, cancel := context.WithCancel(context.Background())
   231  	wg := &sync.WaitGroup{}
   232  
   233  	requestIDs := new(sync.Map)
   234  	mockSrvCh := make(chan *cdcpb.ChangeDataEvent, 100000)
   235  	srv1 := newMockChangeDataService2(b, mockSrvCh)
   236  	srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) {
   237  		for {
   238  			req, err := server.Recv()
   239  			if err != nil {
   240  				return
   241  			}
   242  			requestIDs.Store(req.RegionId, req.RequestId)
   243  		}
   244  	}
   245  	server1, addr1 := newMockService2(ctx, b, srv1, wg)
   246  
   247  	wg.Add(1)
   248  	go func() {
   249  		defer wg.Done()
   250  		<-ctx.Done()
   251  		close(mockSrvCh)
   252  		server1.Stop()
   253  	}()
   254  
   255  	rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("")
   256  	if err != nil {
   257  		b.Error(err)
   258  	}
   259  	pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen}
   260  	tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0)
   261  	if err != nil {
   262  		b.Error(err)
   263  	}
   264  	kvStorage := newStorageWithCurVersionCache(tiStore, addr1)
   265  	defer kvStorage.Close() //nolint:errcheck
   266  
   267  	storeID := uint64(1)
   268  	cluster.AddStore(storeID, addr1)
   269  	// bootstrap with region 100_000(100k)
   270  	cluster.Bootstrap(uint64(100_000), []uint64{storeID}, []uint64{100_001}, 100_001)
   271  	for i := 1; i < regionNum; i++ {
   272  		regionID := uint64(i + 100_000)
   273  		peerID := regionID + 1
   274  		// split regions to [min, b100_001), [b100_001, b100_002), ... [bN, max)
   275  		cluster.SplitRaw(regionID-1, regionID, []byte(fmt.Sprintf("b%d", regionID)), []uint64{peerID}, peerID)
   276  	}
   277  
   278  	lockresolver := txnutil.NewLockerResolver(kvStorage)
   279  	isPullInit := &mockPullerInit{}
   280  	grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{})
   281  	defer grpcPool.Close()
   282  	cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool)
   283  	eventCh := make(chan model.RegionFeedEvent, 1000000)
   284  	wg.Add(1)
   285  	go func() {
   286  		err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("z")}, 100, false, lockresolver, isPullInit, eventCh)
   287  		if errors.Cause(err) != context.Canceled {
   288  			b.Error(err)
   289  		}
   290  		cdcClient.Close() //nolint:errcheck
   291  		wg.Done()
   292  	}()
   293  
   294  	// wait all regions requested from cdc kv client
   295  	err = retry.Do(context.Background(), func() error {
   296  		count := 0
   297  		requestIDs.Range(func(_, _ interface{}) bool {
   298  			count++
   299  			return true
   300  		})
   301  		if count == regionNum {
   302  			return nil
   303  		}
   304  		return errors.Errorf("region number %d is not as expected %d", count, regionNum)
   305  	}, retry.WithBackoffBaseDelay(500), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(20))
   306  	if err != nil {
   307  		b.Error(err)
   308  	}
   309  
   310  	return requestIDs, wg, cancel, eventCh, mockSrvCh
   311  }
   312  
   313  func benchmarkSingleWorkerResolvedTs(b *testing.B, clientV2 bool) {
   314  	enableKVClientV2 = clientV2
   315  	log.SetLevel(zapcore.ErrorLevel)
   316  	tests := []struct {
   317  		name      string
   318  		regionNum int
   319  	}{
   320  		{name: "10", regionNum: 10},
   321  		{name: "100", regionNum: 100},
   322  		{name: "1k", regionNum: 1000},
   323  		{name: "10k", regionNum: 10_000},
   324  		{name: "20k", regionNum: 20_000},
   325  	}
   326  
   327  	for _, test := range tests {
   328  		requestIDs, wg, cancel, eventCh, mockSrvCh := prepareBench(b, test.regionNum)
   329  
   330  		// copy to a normal map to reduce access latency
   331  		copyReqIDs := make(map[uint64]uint64, test.regionNum)
   332  		requestIDs.Range(func(key, value interface{}) bool {
   333  			regionID := key.(uint64)
   334  			requestID := value.(uint64)
   335  			initialized := mockInitializedEvent(regionID, requestID)
   336  			mockSrvCh <- initialized
   337  			copyReqIDs[regionID] = requestID
   338  			return true
   339  		})
   340  
   341  		b.Run(test.name, func(b *testing.B) {
   342  			b.ResetTimer()
   343  			for i := 0; i < b.N; i++ {
   344  				batch := 0
   345  				regions := make([]uint64, 0, batchResolvedSize)
   346  				rts := oracle.ComposeTS(oracle.GetPhysical(time.Now()), 0)
   347  				for regionID := range copyReqIDs {
   348  					batch++
   349  					regions = append(regions, regionID)
   350  					if batch == batchResolvedSize {
   351  						eventResolvedBatch := &cdcpb.ChangeDataEvent{
   352  							ResolvedTs: &cdcpb.ResolvedTs{
   353  								Regions: regions,
   354  								Ts:      rts,
   355  							},
   356  						}
   357  						mockSrvCh <- eventResolvedBatch
   358  						batch = 0
   359  						regions = regions[:0]
   360  					}
   361  				}
   362  				if len(regions) > 0 {
   363  					eventResolvedBatch := &cdcpb.ChangeDataEvent{
   364  						ResolvedTs: &cdcpb.ResolvedTs{
   365  							Regions: regions,
   366  							Ts:      rts,
   367  						},
   368  					}
   369  					mockSrvCh <- eventResolvedBatch
   370  				}
   371  				count := 0
   372  				for range eventCh {
   373  					count++
   374  					if count == test.regionNum {
   375  						break
   376  					}
   377  				}
   378  			}
   379  		})
   380  		err := retry.Do(context.Background(), func() error {
   381  			if len(mockSrvCh) == 0 {
   382  				return nil
   383  			}
   384  			return errors.New("not all events are sent yet")
   385  		}, retry.WithBackoffBaseDelay(500), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(20))
   386  		if err != nil {
   387  			b.Error(err)
   388  		}
   389  
   390  		cancel()
   391  		wg.Wait()
   392  	}
   393  }
   394  
   395  func benchmarkResolvedTsClientV2(b *testing.B) {
   396  	ctx, cancel := context.WithCancel(context.Background())
   397  	defer cancel()
   398  	InitWorkerPool()
   399  	go func() {
   400  		RunWorkerPool(ctx) //nolint:errcheck
   401  	}()
   402  	benchmarkSingleWorkerResolvedTs(b, true /* clientV2 */)
   403  }
   404  
   405  func BenchmarkResolvedTsClientV1(b *testing.B) {
   406  	benchmarkSingleWorkerResolvedTs(b, false /* clientV1 */)
   407  }
   408  
   409  func BenchmarkResolvedTsClientV2(b *testing.B) {
   410  	benchmarkResolvedTsClientV2(b)
   411  }
   412  
   413  func BenchmarkResolvedTsClientV2WorkerPool(b *testing.B) {
   414  	hwm := regionWorkerHighWatermark
   415  	lwm := regionWorkerLowWatermark
   416  	regionWorkerHighWatermark = 10000
   417  	regionWorkerLowWatermark = 2000
   418  	defer func() {
   419  		regionWorkerHighWatermark = hwm
   420  		regionWorkerLowWatermark = lwm
   421  	}()
   422  	benchmarkResolvedTsClientV2(b)
   423  }
   424  
   425  func benchmarkMultipleStoreResolvedTs(b *testing.B, clientV2 bool) {
   426  	enableKVClientV2 = clientV2
   427  	log.SetLevel(zapcore.ErrorLevel)
   428  	tests := []struct {
   429  		name      string
   430  		storeNum  int
   431  		regionNum int
   432  	}{
   433  		{name: "10", storeNum: 10, regionNum: 1},
   434  		{name: "100", storeNum: 10, regionNum: 10},
   435  		{name: "1k", storeNum: 10, regionNum: 100},
   436  		{name: "10k", storeNum: 10, regionNum: 1_000},
   437  		{name: "20k", storeNum: 10, regionNum: 2_000},
   438  	}
   439  
   440  	for _, test := range tests {
   441  		requestIDs, wg, cancel, eventCh, inputs := prepareBenchMultiStore(b, test.storeNum, test.regionNum)
   442  
   443  		// copy to a normal map to reduce access latency, mapping from store index to region id list
   444  		copyReqIDs := make(map[int][]uint64, test.regionNum*test.storeNum)
   445  		requestIDs.Range(func(key, value interface{}) bool {
   446  			regionID := key.(uint64)
   447  			requestID := value.(uint64)
   448  			initialized := mockInitializedEvent(regionID, requestID)
   449  			index := int(regionID-1_000_000) / test.regionNum
   450  			inputs[index] <- initialized
   451  			if _, ok := copyReqIDs[index]; !ok {
   452  				copyReqIDs[index] = make([]uint64, 0, test.regionNum)
   453  			}
   454  			copyReqIDs[index] = append(copyReqIDs[index], regionID)
   455  			return true
   456  		})
   457  
   458  		b.Run(test.name, func(b *testing.B) {
   459  			b.ResetTimer()
   460  			for i := 0; i < b.N; i++ {
   461  				rts := oracle.ComposeTS(oracle.GetPhysical(time.Now()), 0)
   462  				for storeID, regionIDs := range copyReqIDs {
   463  					batch := 0
   464  					regions := make([]uint64, 0, batchResolvedSize)
   465  					for _, regionID := range regionIDs {
   466  						batch++
   467  						regions = append(regions, regionID)
   468  						if batch == batchResolvedSize {
   469  							eventResolvedBatch := &cdcpb.ChangeDataEvent{
   470  								ResolvedTs: &cdcpb.ResolvedTs{
   471  									Regions: regions,
   472  									Ts:      rts,
   473  								},
   474  							}
   475  							inputs[storeID] <- eventResolvedBatch
   476  							batch = 0
   477  							regions = regions[:0]
   478  						}
   479  					}
   480  					if len(regions) > 0 {
   481  						eventResolvedBatch := &cdcpb.ChangeDataEvent{
   482  							ResolvedTs: &cdcpb.ResolvedTs{
   483  								Regions: regions,
   484  								Ts:      rts,
   485  							},
   486  						}
   487  						inputs[storeID] <- eventResolvedBatch
   488  					}
   489  				}
   490  				count := 0
   491  				for range eventCh {
   492  					count++
   493  					if count == test.regionNum*test.storeNum {
   494  						break
   495  					}
   496  				}
   497  			}
   498  		})
   499  
   500  		err := retry.Do(context.Background(), func() error {
   501  			for _, input := range inputs {
   502  				if len(input) != 0 {
   503  					return errors.New("not all events are sent yet")
   504  				}
   505  			}
   506  			return nil
   507  		}, retry.WithBackoffBaseDelay(500), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(1000))
   508  		if err != nil {
   509  			b.Error(err)
   510  		}
   511  
   512  		cancel()
   513  		wg.Wait()
   514  	}
   515  }
   516  
   517  func benchmarkMultiStoreResolvedTsClientV2(b *testing.B) {
   518  	ctx, cancel := context.WithCancel(context.Background())
   519  	defer cancel()
   520  	InitWorkerPool()
   521  	go func() {
   522  		RunWorkerPool(ctx) //nolint:errcheck
   523  	}()
   524  	benchmarkMultipleStoreResolvedTs(b, true /* clientV2 */)
   525  }
   526  
   527  func BenchmarkMultiStoreResolvedTsClientV1(b *testing.B) {
   528  	benchmarkMultipleStoreResolvedTs(b, false /* clientV1 */)
   529  }
   530  
   531  func BenchmarkMultiStoreResolvedTsClientV2(b *testing.B) {
   532  	benchmarkMultiStoreResolvedTsClientV2(b)
   533  }
   534  
   535  func BenchmarkMultiStoreResolvedTsClientV2WorkerPool(b *testing.B) {
   536  	hwm := regionWorkerHighWatermark
   537  	lwm := regionWorkerLowWatermark
   538  	regionWorkerHighWatermark = 1000
   539  	regionWorkerLowWatermark = 200
   540  	defer func() {
   541  		regionWorkerHighWatermark = hwm
   542  		regionWorkerLowWatermark = lwm
   543  	}()
   544  	benchmarkMultiStoreResolvedTsClientV2(b)
   545  }