vitess.io/vitess@v0.16.2/go/vt/vttablet/tabletmanager/vreplication/relaylog.go (about) 1 /* 2 Copyright 2019 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 vreplication 18 19 import ( 20 "io" 21 "sync" 22 "time" 23 24 "context" 25 26 binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" 27 ) 28 29 type relayLog struct { 30 ctx context.Context 31 maxItems int 32 maxSize int 33 34 // mu controls all variables below and is shared by canAccept and hasItems. 35 // Broadcasting must be done while holding mu. This is mainly necessary because both 36 // conditions depend on ctx.Done(), which can change state asynchronously. 37 mu sync.Mutex 38 curSize int 39 items [][]*binlogdatapb.VEvent 40 timedout bool 41 // canAccept is true if: curSize<=maxSize, len(items)<maxItems, and ctx is not Done. 42 canAccept sync.Cond 43 // hasItems is true if len(items)>0, ctx is not Done, and interuptFetch is false. 44 hasItems sync.Cond 45 } 46 47 func newRelayLog(ctx context.Context, maxItems, maxSize int) *relayLog { 48 rl := &relayLog{ 49 ctx: ctx, 50 maxItems: maxItems, 51 maxSize: maxSize, 52 } 53 rl.canAccept.L = &rl.mu 54 rl.hasItems.L = &rl.mu 55 56 // Any time context is done, wake up all waiters to make them exit. 57 go func() { 58 <-ctx.Done() 59 rl.mu.Lock() 60 defer rl.mu.Unlock() 61 rl.canAccept.Broadcast() 62 rl.hasItems.Broadcast() 63 }() 64 return rl 65 } 66 67 // Send writes events to the relay log 68 func (rl *relayLog) Send(events []*binlogdatapb.VEvent) error { 69 rl.mu.Lock() 70 defer rl.mu.Unlock() 71 72 if err := rl.checkDone(); err != nil { 73 return err 74 } 75 for rl.curSize > rl.maxSize || len(rl.items) >= rl.maxItems { 76 rl.canAccept.Wait() 77 if err := rl.checkDone(); err != nil { 78 return err 79 } 80 } 81 rl.items = append(rl.items, events) 82 rl.curSize += eventsSize(events) 83 rl.hasItems.Broadcast() 84 return nil 85 } 86 87 // Fetch returns all existing items in the relay log, and empties the log 88 func (rl *relayLog) Fetch() ([][]*binlogdatapb.VEvent, error) { 89 rl.mu.Lock() 90 defer rl.mu.Unlock() 91 92 if err := rl.checkDone(); err != nil { 93 return nil, err 94 } 95 cancelTimer := rl.startTimer() 96 defer cancelTimer() 97 for len(rl.items) == 0 && !rl.timedout { 98 rl.hasItems.Wait() 99 if err := rl.checkDone(); err != nil { 100 return nil, err 101 } 102 } 103 rl.timedout = false 104 items := rl.items 105 rl.items = nil 106 rl.curSize = 0 107 rl.canAccept.Broadcast() 108 return items, nil 109 } 110 111 func (rl *relayLog) checkDone() error { 112 select { 113 case <-rl.ctx.Done(): 114 return io.EOF 115 default: 116 } 117 return nil 118 } 119 120 func (rl *relayLog) startTimer() (cancel func()) { 121 timer := time.NewTimer(idleTimeout) 122 timerDone := make(chan struct{}) 123 go func() { 124 select { 125 case <-timer.C: 126 rl.mu.Lock() 127 defer rl.mu.Unlock() 128 rl.timedout = true 129 rl.hasItems.Broadcast() 130 case <-timerDone: 131 } 132 }() 133 return func() { 134 timer.Stop() 135 close(timerDone) 136 } 137 } 138 139 func eventsSize(events []*binlogdatapb.VEvent) int { 140 size := 0 141 for _, event := range events { 142 if event.Type != binlogdatapb.VEventType_ROW { 143 continue 144 } 145 for _, rowChange := range event.RowEvent.RowChanges { 146 if rowChange.Before != nil { 147 size += len(rowChange.Before.Values) 148 } 149 if rowChange.After != nil { 150 size += len(rowChange.After.Values) 151 } 152 } 153 } 154 return size 155 }