github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/msg/producer/writer/shard_writer.go (about) 1 // Copyright (c) 2018 Uber Technologies, Inc. 2 // 3 // Permission is hereby granted, free of charge, to any person obtaining a copy 4 // of this software and associated documentation files (the "Software"), to deal 5 // in the Software without restriction, including without limitation the rights 6 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 // copies of the Software, and to permit persons to whom the Software is 8 // furnished to do so, subject to the following conditions: 9 // 10 // The above copyright notice and this permission notice shall be included in 11 // all copies or substantial portions of the Software. 12 // 13 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 // THE SOFTWARE. 20 21 package writer 22 23 import ( 24 "sync" 25 26 "github.com/m3db/m3/src/cluster/placement" 27 "github.com/m3db/m3/src/msg/producer" 28 29 "go.uber.org/atomic" 30 "go.uber.org/zap" 31 ) 32 33 type shardWriter interface { 34 // Write writes the reference counted message, this needs to be thread safe. 35 Write(rm *producer.RefCountedMessage) 36 37 // UpdateInstances updates the instances responsible for this shard. 38 UpdateInstances( 39 instances []placement.Instance, 40 cws map[string]consumerWriter, 41 ) 42 43 // SetMessageTTLNanos sets the message ttl nanoseconds. 44 SetMessageTTLNanos(value int64) 45 46 // Close closes the shard writer. 47 Close() 48 49 // QueueSize returns the number of messages queued for the shard. 50 QueueSize() int 51 } 52 53 type sharedShardWriter struct { 54 instances map[string]struct{} 55 mw *messageWriter 56 isClosed atomic.Bool 57 } 58 59 func newSharedShardWriter( 60 shard uint32, 61 router ackRouter, 62 mPool *messagePool, 63 opts Options, 64 m *messageWriterMetrics, 65 ) shardWriter { 66 replicatedShardID := uint64(shard) 67 mw := newMessageWriter(replicatedShardID, mPool, opts, m) 68 mw.Init() 69 router.Register(replicatedShardID, mw) 70 return &sharedShardWriter{ 71 instances: make(map[string]struct{}), 72 mw: mw, 73 } 74 } 75 76 func (w *sharedShardWriter) Write(rm *producer.RefCountedMessage) { 77 w.mw.Write(rm) 78 } 79 80 // This is not thread safe, must be called in one thread. 81 func (w *sharedShardWriter) UpdateInstances( 82 instances []placement.Instance, 83 cws map[string]consumerWriter, 84 ) { 85 var ( 86 newInstancesMap = make(map[string]struct{}, len(instances)) 87 toBeDeleted = w.instances 88 ) 89 for _, instance := range instances { 90 id := instance.Endpoint() 91 newInstancesMap[id] = struct{}{} 92 if _, ok := toBeDeleted[id]; ok { 93 // Existing instance. 94 delete(toBeDeleted, id) 95 continue 96 } 97 // Add the consumer writer to the message writer. 98 w.mw.AddConsumerWriter(cws[id]) 99 } 100 for id := range toBeDeleted { 101 w.mw.RemoveConsumerWriter(id) 102 } 103 w.instances = newInstancesMap 104 } 105 106 func (w *sharedShardWriter) Close() { 107 if !w.isClosed.CAS(false, true) { 108 return 109 } 110 w.mw.Close() 111 } 112 113 func (w *sharedShardWriter) QueueSize() int { 114 return w.mw.QueueSize() 115 } 116 117 func (w *sharedShardWriter) SetMessageTTLNanos(value int64) { 118 w.mw.SetMessageTTLNanos(value) 119 } 120 121 // nolint: maligned 122 type replicatedShardWriter struct { 123 sync.RWMutex 124 125 shard uint32 126 numberOfShards uint32 127 mPool *messagePool 128 ackRouter ackRouter 129 opts Options 130 logger *zap.Logger 131 m *messageWriterMetrics 132 133 messageWriters map[string]*messageWriter 134 messageTTLNanos int64 135 replicaID uint32 136 isClosed bool 137 } 138 139 func newReplicatedShardWriter( 140 shard, numberOfShards uint32, 141 router ackRouter, 142 mPool *messagePool, 143 opts Options, 144 m *messageWriterMetrics, 145 ) shardWriter { 146 return &replicatedShardWriter{ 147 shard: shard, 148 numberOfShards: numberOfShards, 149 mPool: mPool, 150 opts: opts, 151 logger: opts.InstrumentOptions().Logger(), 152 ackRouter: router, 153 replicaID: 0, 154 messageWriters: make(map[string]*messageWriter), 155 isClosed: false, 156 m: m, 157 } 158 } 159 160 func (w *replicatedShardWriter) Write(rm *producer.RefCountedMessage) { 161 w.RLock() 162 if len(w.messageWriters) == 0 { 163 w.RUnlock() 164 w.m.noWritersError.Inc(1) 165 w.logger.Error("no message writers available for shard", zap.Uint32("shard", rm.Shard())) 166 return 167 } 168 for _, mw := range w.messageWriters { 169 mw.Write(rm) 170 } 171 w.RUnlock() 172 } 173 174 // This is not thread safe, must be called in one thread. 175 func (w *replicatedShardWriter) UpdateInstances( 176 instances []placement.Instance, 177 cws map[string]consumerWriter, 178 ) { 179 // TODO: Schedule time after shardcutoff to clean up message writers that 180 // are already cutoff. Otherwise it will wait until next placement change 181 // to clean up. 182 var ( 183 newMessageWriters = make(map[string]*messageWriter, len(instances)) 184 toBeClosed []*messageWriter 185 toBeAdded = make(map[placement.Instance]consumerWriter, len(instances)) 186 oldMessageWriters = w.messageWriters 187 ) 188 for _, instance := range instances { 189 key := instance.Endpoint() 190 if mw, ok := oldMessageWriters[key]; ok { 191 newMessageWriters[key] = mw 192 // Existing instance, try to update cutover cutoff times. 193 w.updateCutoverCutoffNanos(mw, instance) 194 continue 195 } 196 // This is a new instance. 197 toBeAdded[instance] = cws[key] 198 } 199 for id, mw := range oldMessageWriters { 200 if _, ok := newMessageWriters[id]; ok { 201 // Still in the new placement. 202 continue 203 } 204 // Keep the existing message writer and swap the consumer writer in it 205 // with a new consumer writer in the placement update, so that the 206 // messages buffered in the existing message writer can be tried on 207 // the new consumer writer. 208 if instance, cw, ok := anyKeyValueInMap(toBeAdded); ok { 209 mw.AddConsumerWriter(cw) 210 mw.RemoveConsumerWriter(id) 211 // a replicated writer only has a single downstream consumer instance at a time so we can update the 212 // metrics with a useful consumer label. 213 mw.SetMetrics(mw.Metrics().withConsumer(instance.ID())) 214 w.updateCutoverCutoffNanos(mw, instance) 215 newMessageWriters[instance.Endpoint()] = mw 216 delete(toBeAdded, instance) 217 continue 218 } 219 toBeClosed = append(toBeClosed, mw) 220 } 221 222 // If there are more instances for this shard, this happens when user 223 // increased replication factor for the placement or just this shard. 224 for instance, cw := range toBeAdded { 225 replicatedShardID := uint64(w.replicaID*w.numberOfShards + w.shard) 226 w.replicaID++ 227 mw := newMessageWriter(replicatedShardID, w.mPool, w.opts, w.m) 228 mw.AddConsumerWriter(cw) 229 mw.SetMetrics(mw.Metrics().withConsumer(instance.ID())) 230 w.updateCutoverCutoffNanos(mw, instance) 231 mw.Init() 232 w.ackRouter.Register(replicatedShardID, mw) 233 newMessageWriters[instance.Endpoint()] = mw 234 } 235 236 w.Lock() 237 w.messageWriters = newMessageWriters 238 w.setMessageTTLNanosWithLock(w.messageTTLNanos) 239 w.Unlock() 240 241 // If there are less instances for this shard, this happens when user 242 // reduced replication factor for the placement or just this shard. 243 for _, mw := range toBeClosed { 244 mw := mw 245 // This needs to be in done in a go routine as closing a message writer will 246 // block until all messages consumed. 247 go func() { 248 mw.Close() 249 w.ackRouter.Unregister(mw.ReplicatedShardID()) 250 }() 251 } 252 } 253 254 func (w *replicatedShardWriter) updateCutoverCutoffNanos( 255 mw *messageWriter, 256 instance placement.Instance, 257 ) { 258 s, ok := instance.Shards().Shard(w.shard) 259 if !ok { 260 // Unexpected. 261 w.logger.Error("could not find shard on instance", 262 zap.Uint32("shard", w.shard), zap.String("instance", instance.Endpoint())) 263 return 264 } 265 mw.SetCutoffNanos(s.CutoffNanos()) 266 mw.SetCutoverNanos(s.CutoverNanos()) 267 } 268 269 func (w *replicatedShardWriter) Close() { 270 w.Lock() 271 defer w.Unlock() 272 273 if w.isClosed { 274 return 275 } 276 w.isClosed = true 277 for _, mw := range w.messageWriters { 278 mw.Close() 279 } 280 } 281 282 func (w *replicatedShardWriter) QueueSize() int { 283 w.RLock() 284 mws := w.messageWriters 285 var l int 286 for _, mw := range mws { 287 l += mw.QueueSize() 288 } 289 w.RUnlock() 290 return l 291 } 292 293 func (w *replicatedShardWriter) SetMessageTTLNanos(value int64) { 294 w.Lock() 295 w.messageTTLNanos = value 296 w.setMessageTTLNanosWithLock(value) 297 w.Unlock() 298 } 299 300 func (w *replicatedShardWriter) setMessageTTLNanosWithLock(value int64) { 301 for _, mw := range w.messageWriters { 302 mw.SetMessageTTLNanos(value) 303 } 304 } 305 306 func anyKeyValueInMap( 307 m map[placement.Instance]consumerWriter, 308 ) (placement.Instance, consumerWriter, bool) { 309 for key, value := range m { 310 return key, value, true 311 } 312 return nil, nil, false 313 }