github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink_test.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 package cloudstorage 14 15 import ( 16 "context" 17 "fmt" 18 "net/url" 19 "os" 20 "path" 21 "sync/atomic" 22 "testing" 23 "time" 24 25 "github.com/pingcap/failpoint" 26 timodel "github.com/pingcap/tidb/pkg/parser/model" 27 "github.com/pingcap/tidb/pkg/parser/mysql" 28 "github.com/pingcap/tidb/pkg/parser/types" 29 "github.com/pingcap/tiflow/cdc/model" 30 "github.com/pingcap/tiflow/cdc/sink/dmlsink" 31 "github.com/pingcap/tiflow/cdc/sink/tablesink/state" 32 "github.com/pingcap/tiflow/engine/pkg/clock" 33 "github.com/pingcap/tiflow/pkg/config" 34 "github.com/pingcap/tiflow/pkg/pdutil" 35 "github.com/pingcap/tiflow/pkg/util" 36 "github.com/stretchr/testify/require" 37 ) 38 39 func setClock(s *DMLSink, clock clock.Clock) { 40 for _, w := range s.workers { 41 w.filePathGenerator.SetClock(pdutil.NewMonotonicClock(clock)) 42 } 43 } 44 45 func getTableFiles(t *testing.T, tableDir string) []string { 46 files, err := os.ReadDir(tableDir) 47 require.Nil(t, err) 48 49 fileNames := []string{} 50 for _, f := range files { 51 fileName := f.Name() 52 if f.IsDir() { 53 metaFiles, err := os.ReadDir(path.Join(tableDir, f.Name())) 54 require.Nil(t, err) 55 require.Len(t, metaFiles, 1) 56 fileName = metaFiles[0].Name() 57 } 58 fileNames = append(fileNames, fileName) 59 } 60 return fileNames 61 } 62 63 func generateTxnEvents( 64 cnt *uint64, 65 batch int, 66 tableStatus *state.TableSinkState, 67 ) []*dmlsink.TxnCallbackableEvent { 68 // assume we have a large transaction and it is splitted into 10 small transactions 69 txns := make([]*dmlsink.TxnCallbackableEvent, 0, 10) 70 71 for i := 0; i < 10; i++ { 72 txn := &dmlsink.TxnCallbackableEvent{ 73 Event: &model.SingleTableTxn{ 74 CommitTs: 100, 75 TableInfoVersion: 33, 76 TableInfo: &model.TableInfo{ 77 TableName: model.TableName{ 78 Schema: "test", Table: "table1", 79 }, 80 Version: 33, 81 TableInfo: &timodel.TableInfo{ 82 Columns: []*timodel.ColumnInfo{ 83 {ID: 1, Name: timodel.NewCIStr("c1"), FieldType: *types.NewFieldType(mysql.TypeLong)}, 84 {ID: 2, Name: timodel.NewCIStr("c2"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, 85 }, 86 }, 87 }, 88 }, 89 Callback: func() { 90 atomic.AddUint64(cnt, uint64(batch)) 91 }, 92 SinkState: tableStatus, 93 } 94 tidbTableInfo := &timodel.TableInfo{ 95 Name: timodel.NewCIStr("table1"), 96 Columns: []*timodel.ColumnInfo{ 97 {ID: 1, Name: timodel.NewCIStr("c1"), FieldType: *types.NewFieldType(mysql.TypeLong)}, 98 {ID: 2, Name: timodel.NewCIStr("c2"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, 99 }, 100 } 101 tableInfo := model.WrapTableInfo(100, "test", 33, tidbTableInfo) 102 for j := 0; j < batch; j++ { 103 row := &model.RowChangedEvent{ 104 CommitTs: 100, 105 TableInfo: tableInfo, 106 Columns: []*model.ColumnData{ 107 {ColumnID: 1, Value: i*batch + j}, 108 {ColumnID: 2, Value: "hello world"}, 109 }, 110 } 111 txn.Event.Rows = append(txn.Event.Rows, row) 112 } 113 txns = append(txns, txn) 114 } 115 116 return txns 117 } 118 119 func TestCloudStorageWriteEventsWithoutDateSeparator(t *testing.T) { 120 t.Parallel() 121 122 ctx, cancel := context.WithCancel(context.Background()) 123 parentDir := t.TempDir() 124 uri := fmt.Sprintf("file:///%s?flush-interval=2s", parentDir) 125 sinkURI, err := url.Parse(uri) 126 require.Nil(t, err) 127 128 replicaConfig := config.GetDefaultReplicaConfig() 129 replicaConfig.Sink.DateSeparator = util.AddressOf(config.DateSeparatorNone.String()) 130 replicaConfig.Sink.Protocol = util.AddressOf(config.ProtocolCsv.String()) 131 replicaConfig.Sink.FileIndexWidth = util.AddressOf(6) 132 errCh := make(chan error, 5) 133 s, err := NewDMLSink(ctx, 134 model.DefaultChangeFeedID("test"), 135 pdutil.NewMonotonicClock(clock.New()), 136 sinkURI, replicaConfig, errCh) 137 require.Nil(t, err) 138 var cnt uint64 = 0 139 batch := 100 140 tableStatus := state.TableSinkSinking 141 142 // generating one dml file. 143 txns := generateTxnEvents(&cnt, batch, &tableStatus) 144 err = s.WriteEvents(txns...) 145 require.Nil(t, err) 146 time.Sleep(3 * time.Second) 147 148 metaDir := path.Join(parentDir, "test/table1/meta") 149 files, err := os.ReadDir(metaDir) 150 require.Nil(t, err) 151 require.Len(t, files, 1) 152 153 tableDir := path.Join(parentDir, "test/table1/33") 154 fileNames := getTableFiles(t, tableDir) 155 require.Len(t, fileNames, 2) 156 require.ElementsMatch(t, []string{"CDC000001.csv", "CDC.index"}, fileNames) 157 content, err := os.ReadFile(path.Join(tableDir, "CDC000001.csv")) 158 require.Nil(t, err) 159 require.Greater(t, len(content), 0) 160 161 content, err = os.ReadFile(path.Join(tableDir, "meta/CDC.index")) 162 require.Nil(t, err) 163 require.Equal(t, "CDC000001.csv\n", string(content)) 164 require.Equal(t, uint64(1000), atomic.LoadUint64(&cnt)) 165 166 // generating another dml file. 167 err = s.WriteEvents(txns...) 168 require.Nil(t, err) 169 time.Sleep(3 * time.Second) 170 171 fileNames = getTableFiles(t, tableDir) 172 require.Len(t, fileNames, 3) 173 require.ElementsMatch(t, []string{ 174 "CDC000001.csv", "CDC000002.csv", "CDC.index", 175 }, fileNames) 176 content, err = os.ReadFile(path.Join(tableDir, "CDC000002.csv")) 177 require.Nil(t, err) 178 require.Greater(t, len(content), 0) 179 180 content, err = os.ReadFile(path.Join(tableDir, "meta/CDC.index")) 181 require.Nil(t, err) 182 require.Equal(t, "CDC000002.csv\n", string(content)) 183 require.Equal(t, uint64(2000), atomic.LoadUint64(&cnt)) 184 185 cancel() 186 s.Close() 187 } 188 189 func TestCloudStorageWriteEventsWithDateSeparator(t *testing.T) { 190 t.Parallel() 191 192 ctx, cancel := context.WithCancel(context.Background()) 193 parentDir := t.TempDir() 194 uri := fmt.Sprintf("file:///%s?flush-interval=4s", parentDir) 195 sinkURI, err := url.Parse(uri) 196 require.Nil(t, err) 197 198 replicaConfig := config.GetDefaultReplicaConfig() 199 replicaConfig.Sink.Protocol = util.AddressOf(config.ProtocolCsv.String()) 200 replicaConfig.Sink.DateSeparator = util.AddressOf(config.DateSeparatorDay.String()) 201 replicaConfig.Sink.FileIndexWidth = util.AddressOf(6) 202 203 errCh := make(chan error, 5) 204 mockClock := clock.NewMock() 205 s, err := NewDMLSink(ctx, 206 model.DefaultChangeFeedID("test"), 207 pdutil.NewMonotonicClock(mockClock), 208 sinkURI, replicaConfig, errCh) 209 require.Nil(t, err) 210 211 var cnt uint64 = 0 212 batch := 100 213 tableStatus := state.TableSinkSinking 214 215 mockClock.Set(time.Date(2023, 3, 8, 23, 59, 58, 0, time.UTC)) 216 txns := generateTxnEvents(&cnt, batch, &tableStatus) 217 tableDir := path.Join(parentDir, "test/table1/33/2023-03-08") 218 err = s.WriteEvents(txns...) 219 require.Nil(t, err) 220 time.Sleep(5 * time.Second) 221 222 fileNames := getTableFiles(t, tableDir) 223 require.Len(t, fileNames, 2) 224 require.ElementsMatch(t, []string{"CDC000001.csv", "CDC.index"}, fileNames) 225 content, err := os.ReadFile(path.Join(tableDir, "CDC000001.csv")) 226 require.Nil(t, err) 227 require.Greater(t, len(content), 0) 228 229 content, err = os.ReadFile(path.Join(tableDir, "meta/CDC.index")) 230 require.Nil(t, err) 231 require.Equal(t, "CDC000001.csv\n", string(content)) 232 require.Equal(t, uint64(1000), atomic.LoadUint64(&cnt)) 233 234 // test date (day) is NOT changed. 235 mockClock.Set(time.Date(2023, 3, 8, 23, 59, 59, 0, time.UTC)) 236 setClock(s, mockClock) 237 238 err = s.WriteEvents(txns...) 239 require.Nil(t, err) 240 time.Sleep(5 * time.Second) 241 242 fileNames = getTableFiles(t, tableDir) 243 require.Len(t, fileNames, 3) 244 require.ElementsMatch(t, []string{"CDC000001.csv", "CDC000002.csv", "CDC.index"}, fileNames) 245 content, err = os.ReadFile(path.Join(tableDir, "CDC000002.csv")) 246 require.Nil(t, err) 247 require.Greater(t, len(content), 0) 248 249 content, err = os.ReadFile(path.Join(tableDir, "meta/CDC.index")) 250 require.Nil(t, err) 251 require.Equal(t, "CDC000002.csv\n", string(content)) 252 require.Equal(t, uint64(2000), atomic.LoadUint64(&cnt)) 253 254 // test date (day) is changed. 255 mockClock.Set(time.Date(2023, 3, 9, 0, 0, 10, 0, time.UTC)) 256 setClock(s, mockClock) 257 258 failpoint.Enable("github.com/pingcap/tiflow/cdc/sink/dmlsink/cloudstorage/passTickerOnce", "1*return") 259 defer func() { 260 _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sink/dmlsink/cloudstorage/passTickerOnce") 261 }() 262 263 err = s.WriteEvents(txns...) 264 require.Nil(t, err) 265 time.Sleep(10 * time.Second) 266 267 tableDir = path.Join(parentDir, "test/table1/33/2023-03-09") 268 fileNames = getTableFiles(t, tableDir) 269 require.Len(t, fileNames, 2) 270 require.ElementsMatch(t, []string{"CDC000001.csv", "CDC.index"}, fileNames) 271 content, err = os.ReadFile(path.Join(tableDir, "CDC000001.csv")) 272 require.Nil(t, err) 273 require.Greater(t, len(content), 0) 274 275 content, err = os.ReadFile(path.Join(tableDir, "meta/CDC.index")) 276 require.Nil(t, err) 277 require.Equal(t, "CDC000001.csv\n", string(content)) 278 require.Equal(t, uint64(3000), atomic.LoadUint64(&cnt)) 279 cancel() 280 s.Close() 281 282 // test table is scheduled from one node to another 283 cnt = 0 284 ctx, cancel = context.WithCancel(context.Background()) 285 286 mockClock = clock.NewMock() 287 mockClock.Set(time.Date(2023, 3, 9, 0, 1, 10, 0, time.UTC)) 288 s, err = NewDMLSink(ctx, 289 model.DefaultChangeFeedID("test"), 290 pdutil.NewMonotonicClock(mockClock), 291 sinkURI, replicaConfig, errCh) 292 require.Nil(t, err) 293 294 err = s.WriteEvents(txns...) 295 require.Nil(t, err) 296 time.Sleep(5 * time.Second) 297 298 fileNames = getTableFiles(t, tableDir) 299 require.Len(t, fileNames, 3) 300 require.ElementsMatch(t, []string{"CDC000001.csv", "CDC000002.csv", "CDC.index"}, fileNames) 301 content, err = os.ReadFile(path.Join(tableDir, "CDC000002.csv")) 302 require.Nil(t, err) 303 require.Greater(t, len(content), 0) 304 305 content, err = os.ReadFile(path.Join(tableDir, "meta/CDC.index")) 306 require.Nil(t, err) 307 require.Equal(t, "CDC000002.csv\n", string(content)) 308 require.Equal(t, uint64(1000), atomic.LoadUint64(&cnt)) 309 310 cancel() 311 s.Close() 312 }