github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/binlogstream/streamer_controller.go (about) 1 // Copyright 2019 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 binlogstream is used by syncer to read binlog. 15 // All information related to upstream binlog stream should be kept in this package, 16 // such as reset binlog to a location, maintain properties of the binlog event 17 // and stream, inject or delete binlog events with binlog stream, etc. 18 package binlogstream 19 20 import ( 21 "context" 22 "errors" 23 "sync" 24 "time" 25 26 "github.com/go-mysql-org/go-mysql/mysql" 27 "github.com/go-mysql-org/go-mysql/replication" 28 "github.com/pingcap/failpoint" 29 "github.com/pingcap/tiflow/dm/pb" 30 "github.com/pingcap/tiflow/dm/pkg/binlog" 31 "github.com/pingcap/tiflow/dm/pkg/binlog/reader" 32 "github.com/pingcap/tiflow/dm/pkg/conn" 33 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 34 "github.com/pingcap/tiflow/dm/pkg/log" 35 "github.com/pingcap/tiflow/dm/pkg/terror" 36 "github.com/pingcap/tiflow/dm/pkg/utils" 37 "github.com/pingcap/tiflow/dm/relay" 38 "github.com/pingcap/tiflow/dm/syncer/dbconn" 39 "go.uber.org/zap" 40 ) 41 42 // streamGenerator provides the ability to generate reader.Streamer from 43 // specified location. Current implementation of reader.Streamer are MySQL binlog 44 // streamer, relay log streamer and mock streamer. 45 type streamGenerator interface { 46 GenerateStreamFrom(location binlog.Location) (reader.Streamer, error) 47 } 48 49 type localBinlogReader struct { 50 reader *relay.BinlogReader 51 EnableGTID bool 52 } 53 54 func (l *localBinlogReader) GenerateStreamFrom(location binlog.Location) (reader.Streamer, error) { 55 if l.EnableGTID { 56 return l.reader.StartSyncByGTID(location.GetGTID().Clone()) 57 } 58 return l.reader.StartSyncByPos(location.Position) 59 } 60 61 type remoteBinlogReader struct { 62 reader *replication.BinlogSyncer 63 tctx *tcontext.Context 64 flavor string 65 EnableGTID bool 66 } 67 68 func (r *remoteBinlogReader) GenerateStreamFrom(location binlog.Location) (reader.Streamer, error) { 69 defer func() { 70 lastSlaveConnectionID := r.reader.LastConnectionID() 71 r.tctx.L().Info("last slave connection", zap.Uint32("connection ID", lastSlaveConnectionID)) 72 }() 73 74 if r.EnableGTID { 75 streamer, err := r.reader.StartSyncGTID(location.GetGTID().Clone()) 76 return streamer, terror.ErrSyncerUnitRemoteSteamerStartSync.Delegate(err) 77 } 78 79 // position's name may contain uuid, so need remove it 80 adjustedPos := binlog.RemoveRelaySubDirSuffix(location.Position) 81 streamer, err := r.reader.StartSync(adjustedPos) 82 return streamer, terror.ErrSyncerUnitRemoteSteamerStartSync.Delegate(err) 83 } 84 85 type locationStream struct { 86 stream reader.Streamer 87 *locationRecorder 88 } 89 90 func newLocationStream(generator streamGenerator, location binlog.Location) (locationStream, error) { 91 var ret locationStream 92 93 // strip injected event suffix 94 location.Suffix = 0 95 s, err := generator.GenerateStreamFrom(location) 96 if err != nil { 97 return ret, err 98 } 99 ret.stream = s 100 101 recorder := newLocationRecorder() 102 recorder.reset(location) 103 ret.locationRecorder = recorder 104 return ret, nil 105 } 106 107 func (l locationStream) GetEvent(ctx context.Context) (*replication.BinlogEvent, error) { 108 e, err := l.stream.GetEvent(ctx) 109 if err != nil { 110 return nil, err 111 } 112 113 failpoint.Inject("MakeFakeRotateEvent", func(val failpoint.Value) { 114 ev, ok := e.Event.(*replication.RotateEvent) 115 if ok { 116 e.Header.LogPos = 0 117 e.Header.Flags = replication.LOG_EVENT_ARTIFICIAL_F 118 ev.NextLogName = []byte(val.(string)) 119 log.L().Info("MakeFakeRotateEvent", zap.String("fake file name", string(ev.NextLogName))) 120 } 121 }) 122 123 l.locationRecorder.update(e) 124 return e, nil 125 } 126 127 // StreamerController controls the streamer for read binlog, include: 128 // 1. reset streamer to a binlog position or GTID 129 // 2. read next binlog event 130 // 3. transfer from local streamer to remote streamer. 131 type StreamerController struct { 132 sync.RWMutex 133 134 // the initial binlog type 135 initBinlogType BinlogType 136 137 // the current binlog type 138 currentBinlogType BinlogType 139 retryStrategy retryStrategy 140 141 syncCfg replication.BinlogSyncerConfig 142 enableGTID bool 143 localBinlogDir string 144 timezone *time.Location 145 146 streamProducer streamGenerator 147 upstream locationStream 148 149 *streamModifier 150 // streamModifier will also modify locations so they'll be different from upstreamLocations. 151 locations *locations 152 153 // lastEventFromUpstream is the last event from upstream, and not sent to caller 154 // yet. It should be set to nil after sent to caller. 155 lastEventFromUpstream *replication.BinlogEvent 156 157 // meetError means meeting error when get binlog event 158 // if binlogType is local and meetError is true, then need to create remote binlog stream 159 meetError bool 160 161 fromDB *dbconn.UpStreamConn 162 163 relaySubDirSuffix string 164 165 closed bool 166 167 // whether the server id is updated 168 serverIDUpdated bool 169 relay relay.Process 170 } 171 172 // NewStreamerController creates a new streamer controller. 173 func NewStreamerController( 174 syncCfg replication.BinlogSyncerConfig, 175 enableGTID bool, 176 fromDB *dbconn.UpStreamConn, 177 localBinlogDir string, 178 timezone *time.Location, 179 relay relay.Process, 180 logger log.Logger, 181 ) *StreamerController { 182 var strategy retryStrategy = alwaysRetryStrategy{} 183 binlogType := RelayToBinlogType(relay) 184 if binlogType != LocalBinlog { 185 strategy = &maxIntervalRetryStrategy{ 186 interval: minErrorRetryInterval, 187 } 188 } 189 // let local binlog also return error to avoid infinity loop 190 failpoint.Inject("GetEventError", func() { 191 strategy = &maxIntervalRetryStrategy{ 192 interval: minErrorRetryInterval, 193 } 194 }) 195 streamerController := &StreamerController{ 196 initBinlogType: binlogType, 197 currentBinlogType: binlogType, 198 retryStrategy: strategy, 199 syncCfg: syncCfg, 200 enableGTID: enableGTID, 201 localBinlogDir: localBinlogDir, 202 timezone: timezone, 203 fromDB: fromDB, 204 closed: true, 205 relay: relay, 206 streamModifier: newStreamModifier(logger), 207 locations: &locations{}, 208 } 209 210 return streamerController 211 } 212 213 // Start starts streamer controller. 214 func (c *StreamerController) Start(tctx *tcontext.Context, location binlog.Location) error { 215 c.Lock() 216 defer c.Unlock() 217 218 c.meetError = false 219 c.closed = false 220 c.currentBinlogType = c.initBinlogType 221 222 var err error 223 if c.serverIDUpdated { 224 err = c.resetReplicationSyncer(tctx, location) 225 } else { 226 err = c.updateServerIDAndResetReplication(tctx, location) 227 } 228 if err != nil { 229 c.close() 230 return err 231 } 232 233 return nil 234 } 235 236 // ResetReplicationSyncer reset the replication. 237 func (c *StreamerController) ResetReplicationSyncer(tctx *tcontext.Context, location binlog.Location) (err error) { 238 c.Lock() 239 defer c.Unlock() 240 241 tctx.L().Info("reset replication syncer", zap.Stringer("location", location)) 242 return c.resetReplicationSyncer(tctx, location) 243 } 244 245 func (c *StreamerController) GetStreamer() reader.Streamer { 246 c.Lock() 247 defer c.Unlock() 248 249 return c.upstream 250 } 251 252 func (c *StreamerController) resetReplicationSyncer(tctx *tcontext.Context, location binlog.Location) (err error) { 253 uuidSameWithUpstream := true 254 255 // close old streamProducer 256 if c.streamProducer != nil { 257 switch t := c.streamProducer.(type) { 258 case *remoteBinlogReader: 259 // unclosed conn bug already fixed in go-mysql, https://github.com/go-mysql-org/go-mysql/pull/411 260 t.reader.Close() 261 case *localBinlogReader: 262 // check the uuid before close 263 ctx, cancel := context.WithTimeout(tctx.Ctx, conn.DefaultDBTimeout) 264 defer cancel() 265 uuidSameWithUpstream, err = c.checkUUIDSameWithUpstream(ctx, location.Position, t.reader.GetSubDirs()) 266 if err != nil { 267 return err 268 } 269 t.reader.Close() 270 default: 271 // some other producers such as mockStreamerProducer, should not re-create 272 c.upstream, err = newLocationStream(c.streamProducer, location) 273 return err 274 } 275 } 276 277 if c.currentBinlogType == LocalBinlog && c.meetError { 278 // meetError is true means meets error when get binlog event, in this case use remote binlog as default 279 if !uuidSameWithUpstream { 280 // if the binlog position's uuid is different from the upstream, can not switch to remote binlog 281 tctx.L().Warn("may switch master in upstream, so can not switch local to remote") 282 } else { 283 c.currentBinlogType = RemoteBinlog 284 c.retryStrategy = &maxIntervalRetryStrategy{interval: minErrorRetryInterval} 285 tctx.L().Warn("meet error when read from local binlog, will switch to remote binlog") 286 } 287 } 288 289 if c.currentBinlogType == RemoteBinlog { 290 c.streamProducer = &remoteBinlogReader{replication.NewBinlogSyncer(c.syncCfg), tctx, c.syncCfg.Flavor, c.enableGTID} 291 } else { 292 c.streamProducer = &localBinlogReader{c.relay.NewReader(tctx.L(), &relay.BinlogReaderConfig{RelayDir: c.localBinlogDir, Timezone: c.timezone, Flavor: c.syncCfg.Flavor, RowsEventDecodeFunc: c.syncCfg.RowsEventDecodeFunc}), c.enableGTID} 293 } 294 295 c.upstream, err = newLocationStream(c.streamProducer, location) 296 if err == nil { 297 c.streamModifier.reset(location) 298 c.locations.reset(location) 299 c.lastEventFromUpstream = nil 300 } 301 return err 302 } 303 304 // GetEvent returns binlog event from upstream binlog or streamModifier. It's not 305 // concurrent safe. 306 // 307 // After GetEvent returns an event, GetCurStartLocation, GetCurEndLocation, GetTxnEndLocation 308 // will return the corresponding locations of the event. The definition of 3 locations 309 // can be found in the comment of locations struct in binlog_locations.go . 310 // 311 // When return events from streamModifier, 3 locations are maintained as below: 312 // 313 // - Inject 314 // if we inject events [DDL1, DDL2] at (start) position 900, where start position 315 // 900 has Insert1 event whose LogPos (end position) is 1000, we should return 316 // to caller like 317 // 318 // 1. DDL1, start (900, suffix 0) end (900, suffix 1) 319 // 2. DDL2, start (900, suffix 1) end (900, suffix 2) 320 // 3. Insert1, start (900, suffix 2) end (1000, suffix 0) 321 // 322 // The DDLs are placed before DML because user may want to use Inject to change 323 // table structure for DML. 324 // 325 // - Replace 326 // if we replace events [DDL1, DDL2] at (start) position 900, where start position 327 // 900 has DDL0 event whose LogPos (end position) is 1000, we should return to 328 // caller like 329 // 330 // 1. DDL1, start (900, suffix 0) end (900, suffix 1) 331 // 2. DDL2, start (900, suffix 1) end (1000, suffix 0) 332 // 333 // - Skip 334 // the skipped event will still be sent to caller, with op = pb.ErrorOp_Skip, 335 // to let caller track schema and save checkpoints. 336 func (c *StreamerController) GetEvent(tctx *tcontext.Context) (*replication.BinlogEvent, pb.ErrorOp, error) { 337 event, suffix, op, err := c.getEvent(tctx) 338 // if is local binlog but switch to remote on error, need to add uuid information in binlog's filename 339 if err == nil { 340 if ev, ok := event.Event.(*replication.RotateEvent); ok { 341 // nolint:dogsled 342 _, relaySubDirSuffix, _, _ := utils.SplitFilenameWithUUIDSuffix(string(ev.NextLogName)) 343 if relaySubDirSuffix != "" { 344 c.relaySubDirSuffix = relaySubDirSuffix 345 } else if c.relaySubDirSuffix != "" { 346 filename, err2 := utils.ParseFilename(string(ev.NextLogName)) 347 if err2 != nil { 348 return nil, pb.ErrorOp_InvalidErrorOp, terror.Annotate(err2, "fail to parse binlog file name from rotate event") 349 } 350 ev.NextLogName = []byte(utils.ConstructFilenameWithUUIDSuffix(filename, c.relaySubDirSuffix)) 351 } 352 } 353 } 354 355 if err != nil { 356 if err == context.Canceled || err == context.DeadlineExceeded { 357 return nil, pb.ErrorOp_InvalidErrorOp, err 358 } 359 c.Lock() 360 tctx.L().Error("meet error when get binlog event", zap.Error(err)) 361 c.meetError = true 362 c.Unlock() 363 return nil, pb.ErrorOp_InvalidErrorOp, err 364 } 365 366 if suffix != 0 { 367 c.locations.curStartLocation = c.upstream.curStartLocation 368 c.locations.curStartLocation.Suffix = suffix - 1 369 c.locations.curEndLocation = c.upstream.curStartLocation 370 c.locations.curEndLocation.Suffix = suffix 371 // we only allow injecting DDL, so txnEndLocation is end location of every injected event 372 c.locations.txnEndLocation = c.locations.curEndLocation 373 return event, op, nil 374 } 375 376 if isDataEvent(event) { 377 c.locations.curStartLocation = c.locations.curEndLocation 378 c.locations.curEndLocation = c.upstream.curEndLocation 379 c.locations.txnEndLocation = c.upstream.txnEndLocation 380 return event, op, nil 381 } 382 383 c.locations.curStartLocation = c.locations.curEndLocation 384 c.locations.curEndLocation.CopyWithoutSuffixFrom(c.upstream.curEndLocation) 385 c.locations.txnEndLocation.CopyWithoutSuffixFrom(c.upstream.txnEndLocation) 386 387 return event, op, nil 388 } 389 390 // getEvent gets event from upstream binlog or streamModifier. The maintaining of 391 // locations is on its caller. 392 func (c *StreamerController) getEvent(tctx *tcontext.Context) ( 393 event *replication.BinlogEvent, 394 suffix int, 395 op pb.ErrorOp, 396 err error, 397 ) { 398 failpoint.Inject("SyncerGetEventError", func(_ failpoint.Value) { 399 c.meetError = true 400 tctx.L().Info("mock upstream instance restart", zap.String("failpoint", "SyncerGetEventError")) 401 failpoint.Return(nil, 0, pb.ErrorOp_InvalidErrorOp, terror.ErrDBBadConn.Generate()) 402 }) 403 404 var status getEventFromFrontOpStatus 405 406 LOOP: 407 for frontOp := c.streamModifier.front(); frontOp != nil; frontOp = c.streamModifier.front() { 408 op = pb.ErrorOp_InvalidErrorOp 409 suffix = 0 410 411 if c.lastEventFromUpstream == nil { 412 c.lastEventFromUpstream, err = c.upstream.GetEvent(tctx.Context()) 413 failpoint.Inject("GetEventError", func() { 414 err = errors.New("go-mysql returned an error") 415 }) 416 if err != nil { 417 return 418 } 419 } 420 421 // fake rotate. binlog recorder should handle it 422 if c.lastEventFromUpstream.Header.LogPos == 0 { 423 event = c.lastEventFromUpstream 424 c.lastEventFromUpstream = nil 425 return 426 } 427 428 startPos := mysql.Position{ 429 Name: c.upstream.curEndLocation.Position.Name, 430 Pos: c.lastEventFromUpstream.Header.LogPos - c.lastEventFromUpstream.Header.EventSize, 431 } 432 cmp := binlog.ComparePosition(startPos, frontOp.pos) 433 switch cmp { 434 // when upstream event is earlier than any injected op. 435 case -1: 436 event = c.lastEventFromUpstream 437 c.lastEventFromUpstream = nil 438 break LOOP 439 // when upstream event is at the same location as injected op. 440 case 0: 441 // TODO: check it's DDL 442 switch frontOp.op { 443 case pb.ErrorOp_Skip: 444 // skipped event and op should be sent to caller, to let schema 445 // tracker and checkpoint work 446 event = c.lastEventFromUpstream 447 c.lastEventFromUpstream = nil 448 c.streamModifier.next() 449 op = pb.ErrorOp_Skip 450 break LOOP 451 case pb.ErrorOp_Replace: 452 event, status = c.streamModifier.getEventFromFrontOp() 453 // this op has been totally consumed, move to next op and also delete 454 // lastEventFromUpstream because it's Replace. 455 if status == eventsExhausted { 456 c.lastEventFromUpstream = nil 457 c.streamModifier.next() 458 continue 459 } 460 461 // for last event in Replace, we change the LogPos and EventSize 462 // to imitate the real event. otherwise, we use the LogPos from 463 // startPosition, zero EventSize and increase suffix. 464 if status == lastEvent { 465 event.Header.LogPos = c.lastEventFromUpstream.Header.LogPos 466 event.Header.EventSize = c.lastEventFromUpstream.Header.EventSize 467 suffix = 0 468 } else { 469 event.Header.LogPos = startPos.Pos 470 suffix = c.streamModifier.nextEventInOp 471 } 472 op = pb.ErrorOp_Replace 473 break LOOP 474 case pb.ErrorOp_Inject: 475 event, status = c.streamModifier.getEventFromFrontOp() 476 // this op has been totally consumed, move to next op and return 477 // original upstream event 478 if status == eventsExhausted { 479 c.streamModifier.next() 480 event = c.lastEventFromUpstream 481 c.lastEventFromUpstream = nil 482 break LOOP 483 } 484 485 event.Header.LogPos = startPos.Pos 486 suffix = c.streamModifier.nextEventInOp 487 op = pb.ErrorOp_Inject 488 break LOOP 489 default: 490 c.logger.DPanic("invalid error handle op", zap.Stringer("op", frontOp.op)) 491 } 492 // when upstream event is later than front op. Apart from Inject, 493 // This may happen when user use handle-error but forget to specify source, 494 // so all workers receive the handle-error command. 495 case 1: 496 switch frontOp.op { 497 case pb.ErrorOp_Inject: 498 event, status = c.streamModifier.getEventFromFrontOp() 499 if status == eventsExhausted { 500 c.streamModifier.next() 501 continue 502 } 503 op = pb.ErrorOp_Inject 504 break LOOP 505 default: 506 c.logger.Warn("mismatched handle op", 507 zap.Stringer("op", frontOp.op), 508 zap.Stringer("startPos", startPos), 509 zap.Stringer("frontOp", frontOp.pos), 510 ) 511 c.streamModifier.next() 512 } 513 } 514 } 515 516 if event != nil { 517 return 518 } 519 520 // above loop will be terminated when streamModifier is empty. We still need 521 // to send the last event from upstream. 522 523 if c.lastEventFromUpstream != nil { 524 event = c.lastEventFromUpstream 525 c.lastEventFromUpstream = nil 526 return 527 } 528 529 event, err = c.upstream.GetEvent(tctx.Context()) 530 failpoint.Inject("GetEventError", func() { 531 err = errors.New("go-mysql returned an error") 532 }) 533 // nolint:nakedret 534 return 535 } 536 537 // Close closes streamer. 538 func (c *StreamerController) Close() { 539 c.Lock() 540 c.close() 541 c.Unlock() 542 } 543 544 func (c *StreamerController) close() { 545 if c.closed { 546 return 547 } 548 549 if c.streamProducer != nil { 550 switch r := c.streamProducer.(type) { 551 case *remoteBinlogReader: 552 // process remote binlog reader 553 r.reader.Close() 554 case *localBinlogReader: 555 // process local binlog reader 556 r.reader.Close() 557 } 558 c.streamProducer = nil 559 } 560 561 c.closed = true 562 } 563 564 // IsClosed returns whether streamer controller is closed. 565 func (c *StreamerController) IsClosed() bool { 566 c.RLock() 567 defer c.RUnlock() 568 569 return c.closed 570 } 571 572 // UpdateSyncCfg updates sync config and fromDB. 573 func (c *StreamerController) UpdateSyncCfg(syncCfg replication.BinlogSyncerConfig, fromDB *dbconn.UpStreamConn) { 574 c.Lock() 575 c.fromDB = fromDB 576 c.syncCfg = syncCfg 577 c.Unlock() 578 } 579 580 // check whether the uuid in binlog position's name is same with upstream. 581 func (c *StreamerController) checkUUIDSameWithUpstream(ctx context.Context, pos mysql.Position, uuids []string) (bool, error) { 582 _, uuidSuffix, _, err := utils.SplitFilenameWithUUIDSuffix(pos.Name) 583 if err != nil { 584 // don't contain uuid in position's name 585 // nolint:nilerr 586 return true, nil 587 } 588 uuid := utils.GetUUIDBySuffix(uuids, uuidSuffix) 589 590 upstreamUUID, err := conn.GetServerUUID(tcontext.NewContext(ctx, log.L()), c.fromDB.BaseDB, c.syncCfg.Flavor) 591 if err != nil { 592 return false, terror.Annotate(err, "streamer controller check upstream uuid failed") 593 } 594 595 return uuid == upstreamUUID, nil 596 } 597 598 // GetBinlogType returns the binlog type used now. 599 func (c *StreamerController) GetBinlogType() BinlogType { 600 c.RLock() 601 defer c.RUnlock() 602 return c.currentBinlogType 603 } 604 605 // CanRetry returns true if can switch from local to remote and retry again. 606 func (c *StreamerController) CanRetry(err error) bool { 607 c.RLock() 608 defer c.RUnlock() 609 610 return c.retryStrategy.CanRetry(err) 611 } 612 613 func (c *StreamerController) updateServerID(tctx *tcontext.Context) error { 614 randomServerID, err := conn.GetRandomServerID(tctx, c.fromDB.BaseDB) 615 if err != nil { 616 // should never happened unless the master has too many slave 617 return terror.Annotate(err, "fail to get random server id for streamer controller") 618 } 619 620 c.syncCfg.ServerID = randomServerID 621 c.serverIDUpdated = true 622 return nil 623 } 624 625 // UpdateServerIDAndResetReplication updates the server id and reset replication. 626 func (c *StreamerController) UpdateServerIDAndResetReplication(tctx *tcontext.Context, location binlog.Location) error { 627 c.Lock() 628 defer c.Unlock() 629 630 return c.updateServerIDAndResetReplication(tctx, location) 631 } 632 633 func (c *StreamerController) updateServerIDAndResetReplication(tctx *tcontext.Context, location binlog.Location) error { 634 err := c.updateServerID(tctx) 635 if err != nil { 636 return err 637 } 638 639 err = c.resetReplicationSyncer(tctx, location) 640 if err != nil { 641 return err 642 } 643 644 return nil 645 } 646 647 func (c *StreamerController) GetCurStartLocation() binlog.Location { 648 return c.locations.curStartLocation 649 } 650 651 func (c *StreamerController) GetCurEndLocation() binlog.Location { 652 return c.locations.curEndLocation 653 } 654 655 func (c *StreamerController) GetTxnEndLocation() binlog.Location { 656 return c.locations.txnEndLocation 657 } 658 659 // NewStreamerController4Test is used in tests. 660 func NewStreamerController4Test( 661 streamerProducer streamGenerator, 662 streamer reader.Streamer, 663 ) *StreamerController { 664 return &StreamerController{ 665 streamProducer: streamerProducer, 666 upstream: locationStream{ 667 stream: streamer, 668 locationRecorder: newLocationRecorder(), 669 }, 670 closed: false, 671 streamModifier: &streamModifier{logger: log.L()}, 672 locations: &locations{}, 673 } 674 }