github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/compactor_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 syncer
    15  
    16  import (
    17  	"context"
    18  	"math/rand"
    19  	"time"
    20  
    21  	"github.com/go-mysql-org/go-mysql/mysql"
    22  	. "github.com/pingcap/check"
    23  	"github.com/pingcap/failpoint"
    24  	"github.com/pingcap/tidb/pkg/parser"
    25  	"github.com/pingcap/tidb/pkg/util/mock"
    26  	cdcmodel "github.com/pingcap/tiflow/cdc/model"
    27  	"github.com/pingcap/tiflow/dm/config"
    28  	"github.com/pingcap/tiflow/dm/pkg/binlog"
    29  	tcontext "github.com/pingcap/tiflow/dm/pkg/context"
    30  	"github.com/pingcap/tiflow/dm/pkg/log"
    31  	"github.com/pingcap/tiflow/dm/pkg/utils"
    32  	"github.com/pingcap/tiflow/dm/syncer/metrics"
    33  	"github.com/pingcap/tiflow/pkg/sqlmodel"
    34  )
    35  
    36  // mockExecute mock a kv store.
    37  func mockExecute(kv map[interface{}][]interface{}, dmls []*sqlmodel.RowChange) map[interface{}][]interface{} {
    38  	for _, dml := range dmls {
    39  		switch dml.Type() {
    40  		case sqlmodel.RowChangeInsert:
    41  			kv[dml.GetPostValues()[0]] = dml.GetPostValues()
    42  		case sqlmodel.RowChangeUpdate:
    43  			delete(kv, dml.GetPreValues()[0])
    44  			kv[dml.GetPostValues()[0]] = dml.GetPostValues()
    45  		case sqlmodel.RowChangeDelete:
    46  			delete(kv, dml.GetPreValues()[0])
    47  		}
    48  	}
    49  
    50  	return kv
    51  }
    52  
    53  func randString(n int) string {
    54  	letter := []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ")
    55  	b := make([]rune, n)
    56  	for i := range b {
    57  		b[i] = letter[rand.Intn(len(letter))]
    58  	}
    59  	return string(b)
    60  }
    61  
    62  func (s *testSyncerSuite) TestCompactJob(c *C) {
    63  	compactor := &compactor{
    64  		bufferSize:         10000,
    65  		logger:             log.L(),
    66  		keyMap:             make(map[string]map[string]int),
    67  		buffer:             make([]*job, 0, 10000),
    68  		updateJobMetricsFn: func(bool, string, *job) {},
    69  	}
    70  
    71  	location := binlog.MustZeroLocation(mysql.MySQLFlavor)
    72  	ec := &eventContext{startLocation: location, endLocation: location, lastLocation: location}
    73  	p := parser.New()
    74  	se := mock.NewContext()
    75  	sourceTable := &cdcmodel.TableName{Schema: "test", Table: "tb1"}
    76  	targetTable := &cdcmodel.TableName{Schema: "test", Table: "tb"}
    77  	schemaStr := "create table test.tb(id int primary key, col1 int, name varchar(24))"
    78  	ti, err := createTableInfo(p, se, 0, schemaStr)
    79  	c.Assert(err, IsNil)
    80  
    81  	var dml *sqlmodel.RowChange
    82  	var dmls []*sqlmodel.RowChange
    83  	dmlNum := 1000000
    84  	maxID := 1000
    85  	batch := 1000
    86  	updateIdentifyProbability := 0.1
    87  
    88  	// generate DMLs
    89  	kv := make(map[interface{}][]interface{})
    90  	for i := 0; i < dmlNum; i++ {
    91  		newID := rand.Intn(maxID)
    92  		newCol1 := rand.Intn(maxID * 10)
    93  		newName := randString(rand.Intn(20))
    94  		values := []interface{}{newID, newCol1, newName}
    95  		oldValues, ok := kv[newID]
    96  		if !ok {
    97  			// insert
    98  			dml = sqlmodel.NewRowChange(sourceTable, targetTable, nil, values, ti, nil, nil)
    99  		} else {
   100  			if rand.Int()%2 > 0 {
   101  				// update
   102  				// check whether to update ID
   103  				if rand.Float64() < updateIdentifyProbability {
   104  					for try := 0; try < 10; try++ {
   105  						newID := rand.Intn(maxID)
   106  						if _, ok := kv[newID]; !ok {
   107  							values[0] = newID
   108  							break
   109  						}
   110  					}
   111  				}
   112  				dml = sqlmodel.NewRowChange(sourceTable, targetTable, oldValues, values, ti, nil, nil)
   113  			} else {
   114  				// delete
   115  				dml = sqlmodel.NewRowChange(sourceTable, targetTable, oldValues, nil, ti, nil, nil)
   116  			}
   117  		}
   118  
   119  		kv = mockExecute(kv, []*sqlmodel.RowChange{dml})
   120  		dmls = append(dmls, dml)
   121  	}
   122  
   123  	kv = make(map[interface{}][]interface{})
   124  	compactKV := make(map[interface{}][]interface{})
   125  
   126  	// mock compactJob
   127  	for i := 0; i < len(dmls); i += batch {
   128  		end := i + batch
   129  		if end > len(dmls) {
   130  			end = len(dmls)
   131  		}
   132  		kv = mockExecute(kv, dmls[i:end])
   133  
   134  		for _, dml := range dmls[i:end] {
   135  			j := newDMLJob(dml, ec)
   136  			// if update job update its identify keys, turn it into delete + insert
   137  			if j.dml.IsIdentityUpdated() {
   138  				delDML, insertDML := j.dml.SplitUpdate()
   139  				delJob := j.clone()
   140  				delJob.dml = delDML
   141  
   142  				insertJob := j.clone()
   143  				insertJob.dml = insertDML
   144  
   145  				compactor.compactJob(delJob)
   146  				compactor.compactJob(insertJob)
   147  			} else {
   148  				compactor.compactJob(j)
   149  			}
   150  		}
   151  
   152  		noCompactNumber := end - i
   153  		compactNumber := 0
   154  		for _, dml := range dmls[i:end] {
   155  			c.Logf("before compact, dml: %s", dml.String())
   156  		}
   157  		for _, j := range compactor.buffer {
   158  			if j != nil {
   159  				compactKV = mockExecute(compactKV, []*sqlmodel.RowChange{j.dml})
   160  				compactNumber++
   161  				c.Logf("after compact, dml: %s", j.dml.String())
   162  			}
   163  		}
   164  		c.Logf("before compact: %d, after compact: %d", noCompactNumber, compactNumber)
   165  		c.Assert(compactKV, DeepEquals, kv)
   166  		compactor.keyMap = make(map[string]map[string]int)
   167  		compactor.buffer = compactor.buffer[0:0]
   168  	}
   169  }
   170  
   171  func (s *testSyncerSuite) TestCompactorSafeMode(c *C) {
   172  	p := parser.New()
   173  	se := mock.NewContext()
   174  	sourceTable := &cdcmodel.TableName{Schema: "test", Table: "tb"}
   175  	schemaStr := "create table test.tb(id int primary key, col1 int, name varchar(24))"
   176  	ti, err := createTableInfo(p, se, 0, schemaStr)
   177  	c.Assert(err, IsNil)
   178  
   179  	testCases := []struct {
   180  		input  []*job
   181  		output []*job
   182  	}{
   183  		// nolint:dupl
   184  		{
   185  			input: []*job{
   186  				newDMLJob(
   187  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil),
   188  					ecWithSafeMode,
   189  				),
   190  				newDMLJob(
   191  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "b"}, ti, nil, nil),
   192  					ecWithSafeMode,
   193  				),
   194  				newDMLJob(
   195  					sqlmodel.NewRowChange(sourceTable, nil, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti, nil, nil),
   196  					ecWithSafeMode,
   197  				),
   198  				newDMLJob(
   199  					sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, nil, ti, nil, nil),
   200  					ec,
   201  				),
   202  				newDMLJob(
   203  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil),
   204  					ec,
   205  				),
   206  			},
   207  			output: []*job{
   208  				newDMLJob(
   209  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{3, 3, "c"}, ti, nil, nil),
   210  					ecWithSafeMode,
   211  				),
   212  				newDMLJob(
   213  					sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, nil, ti, nil, nil),
   214  					ecWithSafeMode,
   215  				),
   216  				newDMLJob(
   217  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil),
   218  					ecWithSafeMode,
   219  				),
   220  			},
   221  		},
   222  		// nolint:dupl
   223  		{
   224  			input: []*job{
   225  				newDMLJob(
   226  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil),
   227  					ec,
   228  				),
   229  				newDMLJob(
   230  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "b"}, ti, nil, nil),
   231  					ec,
   232  				),
   233  				newDMLJob(
   234  					sqlmodel.NewRowChange(sourceTable, nil, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti, nil, nil),
   235  					ec,
   236  				),
   237  				newDMLJob(
   238  					sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, nil, ti, nil, nil),
   239  					ec,
   240  				),
   241  				newDMLJob(
   242  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil),
   243  					ec,
   244  				),
   245  				newDMLJob(
   246  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "b"}, ti, nil, nil),
   247  					ec,
   248  				),
   249  				newDMLJob(
   250  					sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, []interface{}{2, 2, "c"}, ti, nil, nil),
   251  					ec,
   252  				),
   253  			},
   254  			output: []*job{
   255  				newDMLJob(
   256  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{3, 3, "c"}, ti, nil, nil),
   257  					ec,
   258  				),
   259  				newDMLJob(
   260  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil),
   261  					ecWithSafeMode,
   262  				),
   263  				newDMLJob(
   264  					sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "c"}, ti, nil, nil),
   265  					ecWithSafeMode,
   266  				),
   267  			},
   268  		},
   269  	}
   270  
   271  	inCh := make(chan *job, 100)
   272  	syncer := &Syncer{
   273  		tctx: tcontext.NewContext(context.Background(), log.L()),
   274  		cfg: &config.SubTaskConfig{
   275  			Name:     "task",
   276  			SourceID: "source",
   277  			SyncerConfig: config.SyncerConfig{
   278  				QueueSize:   100,
   279  				WorkerCount: 100,
   280  			},
   281  		},
   282  		metricsProxies: metrics.DefaultMetricsProxies.CacheForOneTask("task", "worker", "source"),
   283  	}
   284  
   285  	c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/syncer/SkipFlushCompactor", `return()`), IsNil)
   286  	//nolint:errcheck
   287  	defer failpoint.Disable("github.com/pingcap/tiflow/dm/syncer/SkipFlushCompactor")
   288  
   289  	outCh := compactorWrap(inCh, syncer)
   290  
   291  	for _, tc := range testCases {
   292  		for _, j := range tc.input {
   293  			inCh <- j
   294  		}
   295  		inCh <- newFlushJob(syncer.cfg.WorkerCount, 1)
   296  		c.Assert(
   297  			utils.WaitSomething(10, time.Millisecond, func() bool {
   298  				return len(outCh) == len(tc.output)+1
   299  			}), Equals, true)
   300  		for i := 0; i <= len(tc.output); i++ {
   301  			j := <-outCh
   302  			if i < len(tc.output) {
   303  				c.Assert(j.String(), Equals, tc.output[i].String())
   304  			} else {
   305  				c.Assert(j.tp, Equals, flush)
   306  			}
   307  		}
   308  	}
   309  }