vitess.io/vitess@v0.16.2/go/vt/vttablet/tabletserver/vstreamer/copy.go (about) 1 /* 2 Copyright 2020 The Vitess Authors. 3 4 Licensed under the Apache License, Version 2.0 (the "License"); 5 you may not use this file except in compliance with the License. 6 You may obtain a copy of the License at 7 8 http://www.apache.org/licenses/LICENSE-2.0 9 10 Unless required by applicable law or agreed to in writing, software 11 distributed under the License is distributed on an "AS IS" BASIS, 12 WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 See the License for the specific language governing permissions and 14 limitations under the License. 15 */ 16 17 package vstreamer 18 19 import ( 20 "context" 21 "fmt" 22 "io" 23 "math" 24 "time" 25 26 "vitess.io/vitess/go/mysql" 27 "vitess.io/vitess/go/sqltypes" 28 "vitess.io/vitess/go/vt/log" 29 binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" 30 querypb "vitess.io/vitess/go/vt/proto/query" 31 ) 32 33 // starts the copy phase for the first table in the (sorted) list. 34 // can be continuing the copy of a partially completed table or start a new one 35 func (uvs *uvstreamer) copy(ctx context.Context) error { 36 for len(uvs.tablesToCopy) > 0 { 37 tableName := uvs.tablesToCopy[0] 38 log.V(2).Infof("Copystate not empty starting catchupAndCopy on table %s", tableName) 39 if err := uvs.catchupAndCopy(ctx, tableName); err != nil { 40 uvs.vse.errorCounts.Add("Copy", 1) 41 return err 42 } 43 } 44 log.Info("No tables left to copy") 45 return nil 46 } 47 48 // first does a catchup for tables already fully or partially copied (upto last pk) 49 func (uvs *uvstreamer) catchupAndCopy(ctx context.Context, tableName string) error { 50 log.Infof("catchupAndCopy for %s", tableName) 51 if !uvs.pos.IsZero() { 52 if err := uvs.catchup(ctx); err != nil { 53 log.Infof("catchupAndCopy: catchup returned %v", err) 54 uvs.vse.errorCounts.Add("Catchup", 1) 55 return err 56 } 57 } 58 log.Infof("catchupAndCopy: before copyTable %s", tableName) 59 uvs.fields = nil 60 return uvs.copyTable(ctx, tableName) 61 } 62 63 // catchup on events for tables already fully or partially copied (upto last pk) until replication lag is small 64 func (uvs *uvstreamer) catchup(ctx context.Context) error { 65 log.Infof("starting catchup ...") 66 uvs.setReplicationLagSeconds(math.MaxInt64) 67 ctx, cancel := context.WithCancel(ctx) 68 defer cancel() 69 defer func() { 70 uvs.vse.vstreamerPhaseTimings.Record("catchup", time.Now()) 71 }() 72 73 errch := make(chan error, 1) 74 go func() { 75 startPos := mysql.EncodePosition(uvs.pos) 76 vs := newVStreamer(ctx, uvs.cp, uvs.se, startPos, "", uvs.filter, uvs.getVSchema(), uvs.send2, "catchup", uvs.vse) 77 uvs.setVs(vs) 78 errch <- vs.Stream() 79 uvs.setVs(nil) 80 log.Infof("catchup vs.stream returned with vs.pos %s", vs.pos.String()) 81 }() 82 83 // Wait for catchup. 84 tkr := time.NewTicker(uvs.config.CatchupRetryTime) 85 defer tkr.Stop() 86 seconds := int64(uvs.config.MaxReplicationLag / time.Second) 87 for { 88 sbm := uvs.getReplicationLagSeconds() 89 if sbm <= seconds { 90 log.Infof("Canceling context because lag is %d:%d", sbm, seconds) 91 cancel() 92 // Make sure vplayer returns before returning. 93 <-errch 94 return nil 95 } 96 select { 97 case err := <-errch: 98 if err != nil { 99 return err 100 } 101 return io.EOF 102 case <-ctx.Done(): 103 // Make sure vplayer returns before returning. 104 <-errch 105 return io.EOF 106 case <-tkr.C: 107 } 108 } 109 } 110 111 // field event is sent for every new rowevent or set of rowevents 112 func (uvs *uvstreamer) sendFieldEvent(ctx context.Context, gtid string, fieldEvent *binlogdatapb.FieldEvent) error { 113 evs := []*binlogdatapb.VEvent{{ 114 Type: binlogdatapb.VEventType_BEGIN, 115 }, { 116 Type: binlogdatapb.VEventType_FIELD, 117 FieldEvent: fieldEvent, 118 }} 119 log.V(2).Infof("Sending field event %v, gtid is %s", fieldEvent, gtid) 120 uvs.send(evs) 121 122 if err := uvs.setPosition(gtid, true); err != nil { 123 log.Infof("setPosition returned error %v", err) 124 return err 125 } 126 return nil 127 } 128 129 // send one RowEvent per row, followed by a LastPK (merged in VTGate with vgtid) 130 func (uvs *uvstreamer) sendEventsForRows(ctx context.Context, tableName string, rows *binlogdatapb.VStreamRowsResponse, qr *querypb.QueryResult) error { 131 var evs []*binlogdatapb.VEvent 132 for _, row := range rows.Rows { 133 ev := &binlogdatapb.VEvent{ 134 Type: binlogdatapb.VEventType_ROW, 135 Keyspace: uvs.vse.keyspace, 136 Shard: uvs.vse.shard, 137 RowEvent: &binlogdatapb.RowEvent{ 138 TableName: tableName, 139 Keyspace: uvs.vse.keyspace, 140 Shard: uvs.vse.shard, 141 RowChanges: []*binlogdatapb.RowChange{{ 142 Before: nil, 143 After: row, 144 }}, 145 }, 146 } 147 evs = append(evs, ev) 148 } 149 lastPKEvent := &binlogdatapb.LastPKEvent{ 150 TableLastPK: &binlogdatapb.TableLastPK{ 151 TableName: tableName, 152 Lastpk: qr, 153 }, 154 Completed: false, 155 } 156 157 ev := &binlogdatapb.VEvent{ 158 Type: binlogdatapb.VEventType_LASTPK, 159 Keyspace: uvs.vse.keyspace, 160 Shard: uvs.vse.shard, 161 LastPKEvent: lastPKEvent, 162 } 163 evs = append(evs, ev) 164 evs = append(evs, &binlogdatapb.VEvent{ 165 Type: binlogdatapb.VEventType_COMMIT, 166 Keyspace: uvs.vse.keyspace, 167 Shard: uvs.vse.shard, 168 }) 169 170 if err := uvs.send(evs); err != nil { 171 log.Infof("send returned error %v", err) 172 return err 173 } 174 return nil 175 } 176 177 // converts lastpk from proto to value 178 func getLastPKFromQR(qr *querypb.QueryResult) []sqltypes.Value { 179 if qr == nil { 180 return nil 181 } 182 var lastPK []sqltypes.Value 183 r := sqltypes.Proto3ToResult(qr) 184 if len(r.Rows) != 1 { 185 log.Errorf("unexpected lastpk input: %v", qr) 186 return nil 187 } 188 lastPK = r.Rows[0] 189 return lastPK 190 } 191 192 // converts lastpk from value to proto 193 func getQRFromLastPK(fields []*querypb.Field, lastPK []sqltypes.Value) *querypb.QueryResult { 194 row := sqltypes.RowToProto3(lastPK) 195 qr := &querypb.QueryResult{ 196 Fields: fields, 197 Rows: []*querypb.Row{row}, 198 } 199 return qr 200 } 201 202 // gets batch of rows to copy. size of batch is determined by max packetsize 203 func (uvs *uvstreamer) copyTable(ctx context.Context, tableName string) error { 204 ctx, cancel := context.WithCancel(ctx) 205 defer cancel() 206 defer func() { 207 uvs.vse.vstreamerPhaseTimings.Record("copy", time.Now()) 208 }() 209 210 var newLastPK *sqltypes.Result 211 lastPK := getLastPKFromQR(uvs.plans[tableName].tablePK.Lastpk) 212 filter := uvs.plans[tableName].rule.Filter 213 214 log.Infof("Starting copyTable for %s, PK %v", tableName, lastPK) 215 uvs.sendTestEvent(fmt.Sprintf("Copy Start %s", tableName)) 216 217 err := uvs.vse.StreamRows(ctx, filter, lastPK, func(rows *binlogdatapb.VStreamRowsResponse) error { 218 select { 219 case <-ctx.Done(): 220 log.Infof("Returning io.EOF in StreamRows") 221 return io.EOF 222 default: 223 } 224 if uvs.fields == nil { 225 if len(rows.Fields) == 0 { 226 return fmt.Errorf("expecting field event first, got: %v", rows) 227 } 228 pos, _ := mysql.DecodePosition(rows.Gtid) 229 if !uvs.pos.IsZero() && !uvs.pos.AtLeast(pos) { 230 if err := uvs.fastForward(rows.Gtid); err != nil { 231 uvs.setVs(nil) 232 log.Infof("fastForward returned error %v", err) 233 return err 234 } 235 uvs.setVs(nil) 236 if mysql.EncodePosition(uvs.pos) != rows.Gtid { 237 return fmt.Errorf("position after fastforward was %s but stopPos was %s", uvs.pos, rows.Gtid) 238 } 239 if err := uvs.setPosition(rows.Gtid, false); err != nil { 240 return err 241 } 242 } else { 243 log.V(2).Infof("Not starting fastforward pos is %s, uvs.pos is %s, rows.gtid %s", pos, uvs.pos, rows.Gtid) 244 } 245 246 fieldEvent := &binlogdatapb.FieldEvent{ 247 TableName: tableName, 248 Fields: rows.Fields, 249 Keyspace: uvs.vse.keyspace, 250 Shard: uvs.vse.shard, 251 } 252 uvs.fields = rows.Fields 253 uvs.pkfields = rows.Pkfields 254 if err := uvs.sendFieldEvent(ctx, rows.Gtid, fieldEvent); err != nil { 255 log.Infof("sendFieldEvent returned error %v", err) 256 return err 257 } 258 } 259 if len(rows.Rows) == 0 { 260 log.V(2).Infof("0 rows returned for table %s", tableName) 261 return nil 262 } 263 264 newLastPK = sqltypes.CustomProto3ToResult(uvs.pkfields, &querypb.QueryResult{ 265 Fields: rows.Fields, 266 Rows: []*querypb.Row{rows.Lastpk}, 267 }) 268 qrLastPK := sqltypes.ResultToProto3(newLastPK) 269 log.V(2).Infof("Calling sendEventForRows with gtid %s", rows.Gtid) 270 if err := uvs.sendEventsForRows(ctx, tableName, rows, qrLastPK); err != nil { 271 log.Infof("sendEventsForRows returned error %v", err) 272 return err 273 } 274 275 uvs.setCopyState(tableName, qrLastPK) 276 log.V(2).Infof("NewLastPK: %v", qrLastPK) 277 return nil 278 }) 279 if err != nil { 280 uvs.vse.errorCounts.Add("StreamRows", 1) 281 return err 282 } 283 284 select { 285 case <-ctx.Done(): 286 log.Infof("Context done: Copy of %v stopped at lastpk: %v", tableName, newLastPK) 287 return ctx.Err() 288 default: 289 } 290 291 log.Infof("Copy of %v finished at lastpk: %v", tableName, newLastPK) 292 if err := uvs.copyComplete(tableName); err != nil { 293 return err 294 } 295 return nil 296 } 297 298 // processes events between when a table was caught up and when a snapshot is taken for streaming a batch of rows 299 func (uvs *uvstreamer) fastForward(stopPos string) error { 300 defer func() { 301 uvs.vse.vstreamerPhaseTimings.Record("fastforward", time.Now()) 302 }() 303 log.Infof("starting fastForward from %s upto pos %s", mysql.EncodePosition(uvs.pos), stopPos) 304 uvs.stopPos, _ = mysql.DecodePosition(stopPos) 305 vs := newVStreamer(uvs.ctx, uvs.cp, uvs.se, mysql.EncodePosition(uvs.pos), "", uvs.filter, uvs.getVSchema(), uvs.send2, "fastforward", uvs.vse) 306 uvs.setVs(vs) 307 return vs.Stream() 308 }