github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/msg/producer/writer/consumer_service_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 "errors" 25 "fmt" 26 "sync" 27 "time" 28 29 "github.com/m3db/m3/src/cluster/placement" 30 "github.com/m3db/m3/src/msg/producer" 31 "github.com/m3db/m3/src/msg/topic" 32 "github.com/m3db/m3/src/x/watch" 33 34 "github.com/uber-go/tally" 35 "go.uber.org/zap" 36 ) 37 38 var ( 39 acceptAllFilter = producer.FilterFunc( 40 func(m producer.Message) bool { 41 return true 42 }, 43 ) 44 45 errUnknownConsumptionType = errors.New("unknown consumption type") 46 ) 47 48 type initType int 49 50 const ( 51 // failOnError will fail the initialization when any error is encountered. 52 failOnError initType = iota 53 54 // allowInitValueError will not fail the initialization when the initial 55 // value could not be obtained within timeout. 56 // This could be used to initialize a new consumer service writer during 57 // runtime so it allows the consumer service writer to continue waiting 58 // for the placement update in the background. 59 allowInitValueError 60 ) 61 62 type consumerServiceWriter interface { 63 // Write writes a message. 64 Write(rm *producer.RefCountedMessage) 65 66 // Init will initialize the consumer service writer. 67 Init(initType) error 68 69 // Close closes the writer and the background watch thread. 70 Close() 71 72 // SetMessageTTLNanos sets the message ttl nanoseconds. 73 SetMessageTTLNanos(value int64) 74 75 // RegisterFilter registers a filter for the consumer service. 76 RegisterFilter(fn producer.FilterFunc) 77 78 // UnregisterFilters unregisters the filters for the consumer service. 79 UnregisterFilters() 80 } 81 82 type consumerServiceWriterMetrics struct { 83 placementError tally.Counter 84 placementUpdate tally.Counter 85 filterAccepted tally.Counter 86 filterNotAccepted tally.Counter 87 queueSize tally.Gauge 88 } 89 90 func newConsumerServiceWriterMetrics(scope tally.Scope) consumerServiceWriterMetrics { 91 return consumerServiceWriterMetrics{ 92 placementUpdate: scope.Counter("placement-update"), 93 placementError: scope.Counter("placement-error"), 94 filterAccepted: scope.Counter("filter-accepted"), 95 filterNotAccepted: scope.Counter("filter-not-accepted"), 96 queueSize: scope.Gauge("queue-size"), 97 } 98 } 99 100 type consumerServiceWriterImpl struct { 101 sync.Mutex 102 103 cs topic.ConsumerService 104 ps placement.Service 105 shardWriters []shardWriter 106 opts Options 107 logger *zap.Logger 108 109 value watch.Value 110 dataFilters []producer.FilterFunc 111 router ackRouter 112 consumerWriters map[string]consumerWriter 113 closed bool 114 doneCh chan struct{} 115 wg sync.WaitGroup 116 m consumerServiceWriterMetrics 117 cm consumerWriterMetrics 118 119 processFn watch.ProcessFn 120 } 121 122 func newConsumerServiceWriter( 123 cs topic.ConsumerService, 124 numShards uint32, 125 opts Options, 126 ) (consumerServiceWriter, error) { 127 ps, err := opts.ServiceDiscovery(). 128 PlacementService(cs.ServiceID(), opts.PlacementOptions()) 129 if err != nil { 130 return nil, err 131 } 132 ct := cs.ConsumptionType() 133 if ct == topic.Unknown { 134 return nil, errUnknownConsumptionType 135 } 136 router := newAckRouter(int(numShards)) 137 w := &consumerServiceWriterImpl{ 138 cs: cs, 139 ps: ps, 140 shardWriters: initShardWriters(router, ct, numShards, opts), 141 opts: opts, 142 logger: opts.InstrumentOptions().Logger(), 143 dataFilters: []producer.FilterFunc{acceptAllFilter}, 144 router: router, 145 consumerWriters: make(map[string]consumerWriter), 146 closed: false, 147 doneCh: make(chan struct{}), 148 m: newConsumerServiceWriterMetrics(opts.InstrumentOptions().MetricsScope()), 149 cm: newConsumerWriterMetrics(opts.InstrumentOptions().MetricsScope()), 150 } 151 w.processFn = w.process 152 return w, nil 153 } 154 155 func initShardWriters( 156 router ackRouter, 157 ct topic.ConsumptionType, 158 numberOfShards uint32, 159 opts Options, 160 ) []shardWriter { 161 var ( 162 sws = make([]shardWriter, numberOfShards) 163 m = newMessageWriterMetrics( 164 opts.InstrumentOptions().MetricsScope(), 165 opts.InstrumentOptions().TimerOptions(), 166 opts.WithoutConsumerScope(), 167 ) 168 mPool = newMessagePool() 169 ) 170 for i := range sws { 171 switch ct { 172 case topic.Shared: 173 sws[i] = newSharedShardWriter(uint32(i), router, mPool, opts, m) 174 case topic.Replicated: 175 sws[i] = newReplicatedShardWriter(uint32(i), numberOfShards, router, mPool, opts, m) 176 } 177 } 178 return sws 179 } 180 181 func (w *consumerServiceWriterImpl) Write(rm *producer.RefCountedMessage) { 182 if rm.Accept(w.dataFilters) { 183 w.shardWriters[rm.Shard()].Write(rm) 184 w.m.filterAccepted.Inc(1) 185 return 186 } 187 // It is not an error if the message does not pass the filter. 188 w.m.filterNotAccepted.Inc(1) 189 } 190 191 func (w *consumerServiceWriterImpl) Init(t initType) error { 192 w.wg.Add(1) 193 go func() { 194 w.reportMetrics() 195 w.wg.Done() 196 }() 197 198 updatableFn := func() (watch.Updatable, error) { 199 return w.ps.Watch() 200 } 201 getFn := func(updatable watch.Updatable) (interface{}, error) { 202 update, err := updatable.(placement.Watch).Get() 203 if err != nil { 204 w.m.placementError.Inc(1) 205 w.logger.Error("invalid placement update from kv", zap.Error(err)) 206 return nil, err 207 } 208 w.m.placementUpdate.Inc(1) 209 return update, nil 210 } 211 vOptions := watch.NewOptions(). 212 SetInitWatchTimeout(w.opts.PlacementWatchInitTimeout()). 213 SetInstrumentOptions(w.opts.InstrumentOptions()). 214 SetNewUpdatableFn(updatableFn). 215 SetGetUpdateFn(getFn). 216 SetProcessFn(w.processFn). 217 SetKey(w.opts.TopicName()) 218 w.value = watch.NewValue(vOptions) 219 err := w.value.Watch() 220 if err == nil { 221 return nil 222 } 223 if t == allowInitValueError { 224 if _, ok := err.(watch.InitValueError); ok { 225 w.logger.Warn("invalid placement update, continue to watch for placement updates", 226 zap.Error(err)) 227 return nil 228 } 229 } 230 return fmt.Errorf("consumer service writer init error: %v", err) 231 } 232 233 func (w *consumerServiceWriterImpl) process(update interface{}) error { 234 var ( 235 p = update.(placement.Placement) 236 isSharded = p.IsSharded() 237 ) 238 // Non sharded placement is only allowed for Shared consumption type. 239 if w.cs.ConsumptionType() == topic.Replicated && !isSharded { 240 return fmt.Errorf("non-sharded placement for replicated consumer %s", w.cs.String()) 241 } 242 // NB(cw): Lock can be removed as w.consumerWriters is only accessed in this thread. 243 w.Lock() 244 newConsumerWriters, tobeDeleted := w.diffPlacementWithLock(p) 245 for i, sw := range w.shardWriters { 246 if isSharded { 247 sw.UpdateInstances(p.InstancesForShard(uint32(i)), newConsumerWriters) 248 continue 249 } 250 sw.UpdateInstances(p.Instances(), newConsumerWriters) 251 } 252 oldConsumerWriters := w.consumerWriters 253 w.consumerWriters = newConsumerWriters 254 w.Unlock() 255 go func() { 256 for _, addr := range tobeDeleted { 257 cw, ok := oldConsumerWriters[addr] 258 if ok { 259 cw.Close() 260 } 261 } 262 }() 263 return nil 264 } 265 266 func (w *consumerServiceWriterImpl) diffPlacementWithLock(newPlacement placement.Placement) (map[string]consumerWriter, []string) { 267 var ( 268 newInstances = newPlacement.Instances() 269 newConsumerWriters = make(map[string]consumerWriter, len(newInstances)) 270 toBeDeleted []string 271 ) 272 for _, instance := range newInstances { 273 id := instance.Endpoint() 274 cw, ok := w.consumerWriters[id] 275 if ok { 276 newConsumerWriters[id] = cw 277 continue 278 } 279 cw = newConsumerWriter(instance.Endpoint(), w.router, w.opts, w.cm) 280 cw.Init() 281 newConsumerWriters[id] = cw 282 } 283 284 for id := range w.consumerWriters { 285 if _, ok := newConsumerWriters[id]; !ok { 286 toBeDeleted = append(toBeDeleted, id) 287 } 288 } 289 return newConsumerWriters, toBeDeleted 290 } 291 292 func (w *consumerServiceWriterImpl) Close() { 293 w.Lock() 294 if w.closed { 295 w.Unlock() 296 return 297 } 298 w.closed = true 299 w.Unlock() 300 301 w.logger.Info("closing consumer service writer", zap.String("writer", w.cs.String())) 302 close(w.doneCh) 303 // Blocks until all messages consuemd. 304 var shardWriterWG sync.WaitGroup 305 for _, sw := range w.shardWriters { 306 sw := sw 307 shardWriterWG.Add(1) 308 go func() { 309 sw.Close() 310 shardWriterWG.Done() 311 }() 312 } 313 shardWriterWG.Wait() 314 315 w.value.Unwatch() 316 for _, cw := range w.consumerWriters { 317 cw.Close() 318 } 319 w.wg.Wait() 320 w.logger.Info("closed consumer service writer", zap.String("writer", w.cs.String())) 321 } 322 323 func (w *consumerServiceWriterImpl) SetMessageTTLNanos(value int64) { 324 for _, sw := range w.shardWriters { 325 sw.SetMessageTTLNanos(value) 326 } 327 } 328 329 func (w *consumerServiceWriterImpl) RegisterFilter(filter producer.FilterFunc) { 330 w.Lock() 331 w.dataFilters = append(w.dataFilters, filter) 332 w.Unlock() 333 } 334 335 func (w *consumerServiceWriterImpl) UnregisterFilters() { 336 w.Lock() 337 w.dataFilters[0] = acceptAllFilter 338 w.dataFilters = w.dataFilters[:1] 339 w.Unlock() 340 } 341 342 func (w *consumerServiceWriterImpl) reportMetrics() { 343 t := time.NewTicker(w.opts.InstrumentOptions().ReportInterval()) 344 defer t.Stop() 345 346 for { 347 select { 348 case <-w.doneCh: 349 return 350 case <-t.C: 351 var l int 352 for _, sw := range w.shardWriters { 353 l += sw.QueueSize() 354 } 355 w.m.queueSize.Update(float64(l)) 356 } 357 } 358 }