github.com/m3db/m3@v1.5.0/src/msg/producer/writer/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  
    28  	"github.com/m3db/m3/src/cluster/services"
    29  	"github.com/m3db/m3/src/msg/producer"
    30  	"github.com/m3db/m3/src/msg/topic"
    31  	xerrors "github.com/m3db/m3/src/x/errors"
    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  	errWriterClosed = errors.New("writer is closed")
    40  )
    41  
    42  type writerMetrics struct {
    43  	topicUpdateSuccess  tally.Counter
    44  	topicUpdateError    tally.Counter
    45  	invalidTopicUpdate  tally.Counter
    46  	invalidShard        tally.Counter
    47  	numConsumerServices tally.Gauge
    48  }
    49  
    50  func newWriterMetrics(scope tally.Scope) writerMetrics {
    51  	return writerMetrics{
    52  		topicUpdateSuccess: scope.Counter("topic-update-success"),
    53  		topicUpdateError:   scope.Counter("topic-update-error"),
    54  		invalidTopicUpdate: scope.Counter("invalid-topic"),
    55  		invalidShard: scope.Tagged(map[string]string{"reason": "invalid-shard"}).
    56  			Counter("invalid-shard-write"),
    57  		numConsumerServices: scope.Gauge("num-consumer-services"),
    58  	}
    59  }
    60  
    61  // nolint: maligned
    62  type writer struct {
    63  	sync.RWMutex
    64  
    65  	topic  string
    66  	ts     topic.Service
    67  	opts   Options
    68  	logger *zap.Logger
    69  
    70  	value                  watch.Value
    71  	initType               initType
    72  	numShards              uint32
    73  	consumerServiceWriters map[string]consumerServiceWriter
    74  	filterRegistry         map[string]producer.FilterFunc
    75  	isClosed               bool
    76  	m                      writerMetrics
    77  
    78  	processFn watch.ProcessFn
    79  }
    80  
    81  // NewWriter creates a new writer.
    82  func NewWriter(opts Options) producer.Writer {
    83  	w := &writer{
    84  		topic:                  opts.TopicName(),
    85  		ts:                     opts.TopicService(),
    86  		opts:                   opts,
    87  		logger:                 opts.InstrumentOptions().Logger(),
    88  		initType:               failOnError,
    89  		consumerServiceWriters: make(map[string]consumerServiceWriter),
    90  		filterRegistry:         make(map[string]producer.FilterFunc),
    91  		isClosed:               false,
    92  		m:                      newWriterMetrics(opts.InstrumentOptions().MetricsScope()),
    93  	}
    94  	w.processFn = w.process
    95  	return w
    96  }
    97  
    98  func (w *writer) Write(rm *producer.RefCountedMessage) error {
    99  	w.RLock()
   100  	if w.isClosed {
   101  		rm.Drop()
   102  		w.RUnlock()
   103  		return errWriterClosed
   104  	}
   105  	shard := rm.Shard()
   106  	if shard >= w.numShards {
   107  		w.m.invalidShard.Inc(1)
   108  		rm.Drop()
   109  		w.RUnlock()
   110  		return fmt.Errorf("could not write message for shard %d which is larger than max shard id %d", shard, w.numShards-1)
   111  	}
   112  	// NB(cw): Need to inc ref here in case a consumer service
   113  	// writes the message too fast and close the message.
   114  	rm.IncRef()
   115  	for _, csw := range w.consumerServiceWriters {
   116  		csw.Write(rm)
   117  	}
   118  	rm.DecRef()
   119  	w.RUnlock()
   120  	return nil
   121  }
   122  
   123  func (w *writer) Init() error {
   124  	newUpdatableFn := func() (watch.Updatable, error) {
   125  		return w.ts.Watch(w.topic)
   126  	}
   127  	getUpdateFn := func(value watch.Updatable) (interface{}, error) {
   128  		t, err := value.(topic.Watch).Get()
   129  		if err != nil {
   130  			w.m.invalidTopicUpdate.Inc(1)
   131  			return nil, err
   132  		}
   133  		return t, nil
   134  	}
   135  	vOptions := watch.NewOptions().
   136  		SetInitWatchTimeout(w.opts.TopicWatchInitTimeout()).
   137  		SetInstrumentOptions(w.opts.InstrumentOptions()).
   138  		SetNewUpdatableFn(newUpdatableFn).
   139  		SetGetUpdateFn(getUpdateFn).
   140  		SetProcessFn(w.processFn).
   141  		SetKey(w.opts.TopicName())
   142  	w.value = watch.NewValue(vOptions)
   143  	if err := w.value.Watch(); err != nil {
   144  		return fmt.Errorf("writer init error: %v", err)
   145  	}
   146  	return nil
   147  }
   148  
   149  func (w *writer) NumShards() uint32 {
   150  	w.RLock()
   151  	n := w.numShards
   152  	w.RUnlock()
   153  	return n
   154  }
   155  
   156  func (w *writer) process(update interface{}) error {
   157  	t := update.(topic.Topic)
   158  	if err := t.Validate(); err != nil {
   159  		return err
   160  	}
   161  	// We don't allow changing number of shards for topics, it will be
   162  	// prevented on topic service side, but also being defensive here as well.
   163  	numShards := w.NumShards()
   164  	if numShards != 0 && numShards != t.NumberOfShards() {
   165  		w.m.topicUpdateError.Inc(1)
   166  		return fmt.Errorf("invalid topic update with %d shards, expecting %d", t.NumberOfShards(), numShards)
   167  	}
   168  	var (
   169  		iOpts                     = w.opts.InstrumentOptions()
   170  		newConsumerServiceWriters = make(map[string]consumerServiceWriter, len(t.ConsumerServices()))
   171  		toBeClosed                []consumerServiceWriter
   172  		multiErr                  xerrors.MultiError
   173  	)
   174  	for _, cs := range t.ConsumerServices() {
   175  		key := cs.ServiceID().String()
   176  		csw, ok := w.consumerServiceWriters[key]
   177  		if ok {
   178  			csw.SetMessageTTLNanos(cs.MessageTTLNanos())
   179  			newConsumerServiceWriters[key] = csw
   180  			continue
   181  		}
   182  		scope := iOpts.MetricsScope().Tagged(map[string]string{
   183  			"consumer-service-name": cs.ServiceID().Name(),
   184  			"consumer-service-zone": cs.ServiceID().Zone(),
   185  			"consumer-service-env":  cs.ServiceID().Environment(),
   186  			"consumption-type":      cs.ConsumptionType().String(),
   187  		})
   188  		csw, err := newConsumerServiceWriter(cs, t.NumberOfShards(), w.opts.SetInstrumentOptions(iOpts.SetMetricsScope(scope)))
   189  		if err != nil {
   190  			w.logger.Error("could not create consumer service writer",
   191  				zap.String("writer", cs.String()), zap.Error(err))
   192  			multiErr = multiErr.Add(err)
   193  			continue
   194  		}
   195  		if err = csw.Init(w.initType); err != nil {
   196  			w.logger.Error("could not init consumer service writer",
   197  				zap.String("writer", cs.String()), zap.Error(err))
   198  			multiErr = multiErr.Add(err)
   199  			// Could not initialize the consumer service, simply close it.
   200  			csw.Close()
   201  			continue
   202  		}
   203  		csw.SetMessageTTLNanos(cs.MessageTTLNanos())
   204  		newConsumerServiceWriters[key] = csw
   205  		w.logger.Info("initialized consumer service writer", zap.String("writer", cs.String()))
   206  	}
   207  	for key, csw := range w.consumerServiceWriters {
   208  		if _, ok := newConsumerServiceWriters[key]; !ok {
   209  			toBeClosed = append(toBeClosed, csw)
   210  		}
   211  	}
   212  	// Allow InitValueError for any future topic updates after starting up.
   213  	// This is to handle the case when a new consumer service got added to
   214  	// the topic, but the producer could not get initial value for its
   215  	// placement. We will continue to watch for placement updates for the new
   216  	// consumer service in the background, so the producer can write to it once
   217  	// the placement came in.
   218  	w.initType = allowInitValueError
   219  	w.m.numConsumerServices.Update(float64(len(newConsumerServiceWriters)))
   220  
   221  	// Apply the new consumer service writers.
   222  	w.Lock()
   223  	for key, csw := range newConsumerServiceWriters {
   224  		if filter, ok := w.filterRegistry[key]; ok {
   225  			csw.RegisterFilter(filter)
   226  		}
   227  	}
   228  	w.consumerServiceWriters = newConsumerServiceWriters
   229  	w.numShards = t.NumberOfShards()
   230  	w.Unlock()
   231  
   232  	// Close removed consumer service.
   233  	go func() {
   234  		for _, csw := range toBeClosed {
   235  			csw.Close()
   236  		}
   237  	}()
   238  
   239  	if err := multiErr.FinalError(); err != nil {
   240  		w.m.topicUpdateError.Inc(1)
   241  		return err
   242  	}
   243  	w.m.topicUpdateSuccess.Inc(1)
   244  	return nil
   245  }
   246  
   247  func (w *writer) Close() {
   248  	w.Lock()
   249  	if w.isClosed {
   250  		w.Unlock()
   251  		return
   252  	}
   253  	w.isClosed = true
   254  	w.Unlock()
   255  
   256  	w.value.Unwatch()
   257  	for _, csw := range w.consumerServiceWriters {
   258  		csw.Close()
   259  	}
   260  }
   261  
   262  func (w *writer) RegisterFilter(sid services.ServiceID, filter producer.FilterFunc) {
   263  	w.Lock()
   264  	defer w.Unlock()
   265  
   266  	key := sid.String()
   267  	w.filterRegistry[key] = filter
   268  	csw, ok := w.consumerServiceWriters[key]
   269  	if ok {
   270  		csw.RegisterFilter(filter)
   271  	}
   272  }
   273  
   274  func (w *writer) UnregisterFilter(sid services.ServiceID) {
   275  	w.Lock()
   276  	defer w.Unlock()
   277  
   278  	key := sid.String()
   279  	delete(w.filterRegistry, key)
   280  	csw, ok := w.consumerServiceWriters[key]
   281  	if ok {
   282  		csw.UnregisterFilter()
   283  	}
   284  }