github.com/matrixorigin/matrixone@v1.2.0/pkg/util/export/batch_processor.go (about)

     1  // Copyright 2022 Matrix Origin
     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  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package export
    16  
    17  import (
    18  	"bytes"
    19  	"context"
    20  	"fmt"
    21  	"math"
    22  	"runtime"
    23  	"sync"
    24  	"sync/atomic"
    25  	"time"
    26  
    27  	"github.com/matrixorigin/matrixone/pkg/common/log"
    28  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    29  	morun "github.com/matrixorigin/matrixone/pkg/common/runtime"
    30  	"github.com/matrixorigin/matrixone/pkg/config"
    31  	"github.com/matrixorigin/matrixone/pkg/logutil"
    32  	"github.com/matrixorigin/matrixone/pkg/util/batchpipe"
    33  	"github.com/matrixorigin/matrixone/pkg/util/errutil"
    34  	"github.com/matrixorigin/matrixone/pkg/util/export/table"
    35  	v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2"
    36  	"github.com/matrixorigin/matrixone/pkg/util/trace"
    37  	"github.com/matrixorigin/matrixone/pkg/util/trace/impl/motrace"
    38  
    39  	"go.uber.org/zap"
    40  )
    41  
    42  const defaultQueueSize = 1310720 // queue mem cost = 10MB
    43  
    44  const LoggerNameMOCollector = "MOCollector"
    45  
    46  const discardCollectTimeout = time.Millisecond
    47  
    48  // bufferHolder hold ItemBuffer content, handle buffer's new/flush/reset/reminder(base on timer) operations.
    49  // work like:
    50  // ---> Add ---> ShouldFlush or trigger.signal -----> StopAndGetBatch ---> FlushAndReset ---> Add ---> ...
    51  // #     ^                   |No                |Yes, go next call
    52  // #     |<------------------/Accept next Add
    53  type bufferHolder struct {
    54  	c   *MOCollector
    55  	ctx context.Context
    56  	// name like a type
    57  	name string
    58  	// buffer is instance of batchpipe.ItemBuffer with its own elimination algorithm(like LRU, LFU)
    59  	buffer batchpipe.ItemBuffer[batchpipe.HasName, any]
    60  	// bufferPool
    61  	bufferPool *sync.Pool
    62  	bufferCnt  atomic.Int32
    63  	discardCnt atomic.Int32
    64  	// reminder
    65  	reminder batchpipe.Reminder
    66  	// signal send signal to Collector
    67  	signal bufferSignalFunc // see awakeBufferFactory
    68  	// impl NewItemBatchHandler
    69  	impl motrace.PipeImpl
    70  	// trigger handle Reminder strategy
    71  	trigger *time.Timer
    72  
    73  	mux sync.Mutex
    74  	// stopped mark
    75  	stopped bool
    76  }
    77  
    78  type bufferSignalFunc func(*bufferHolder)
    79  
    80  func newBufferHolder(ctx context.Context, name batchpipe.HasName, impl motrace.PipeImpl, signal bufferSignalFunc, c *MOCollector) *bufferHolder {
    81  	b := &bufferHolder{
    82  		c:      c,
    83  		ctx:    ctx,
    84  		name:   name.GetName(),
    85  		signal: signal,
    86  		impl:   impl,
    87  	}
    88  	b.bufferPool = &sync.Pool{}
    89  	b.bufferCnt.Swap(0)
    90  	b.bufferPool.New = func() interface{} {
    91  		return b.impl.NewItemBuffer(b.name)
    92  	}
    93  	b.buffer = b.getBuffer()
    94  	b.reminder = b.buffer.(batchpipe.Reminder)
    95  	b.mux.Lock()
    96  	defer b.mux.Unlock()
    97  	b.trigger = time.AfterFunc(time.Hour, func() {})
    98  	return b
    99  }
   100  
   101  // Start separated from newBufferHolder, should call only once, fix trigger started before first Add
   102  func (b *bufferHolder) Start() {
   103  	b.mux.Lock()
   104  	defer b.mux.Unlock()
   105  	b.stopped = false
   106  	b.trigger.Stop()
   107  	b.trigger = time.AfterFunc(b.reminder.RemindNextAfter(), func() {
   108  		if b.mux.TryLock() {
   109  			b.mux.Unlock()
   110  		}
   111  		b.signal(b)
   112  	})
   113  }
   114  
   115  func (b *bufferHolder) getBuffer() batchpipe.ItemBuffer[batchpipe.HasName, any] {
   116  	b.c.allocBuffer()
   117  	b.bufferCnt.Add(1)
   118  	buffer := b.bufferPool.Get().(batchpipe.ItemBuffer[batchpipe.HasName, any])
   119  	b.logStatus("new buffer")
   120  	return buffer
   121  }
   122  
   123  func (b *bufferHolder) putBuffer(buffer batchpipe.ItemBuffer[batchpipe.HasName, any]) {
   124  	buffer.Reset()
   125  	b.bufferPool.Put(buffer)
   126  	b.bufferCnt.Add(-1)
   127  	b.c.releaseBuffer()
   128  	b.logStatus("release buffer")
   129  }
   130  
   131  func (b *bufferHolder) logStatus(msg string) {
   132  	if b.c.logger.Enabled(zap.DebugLevel) {
   133  		b.c.logger.Debug(msg,
   134  			zap.String("item", b.name),
   135  			zap.Int32("cnt", b.bufferCnt.Load()),
   136  			zap.Int32("using", b.c.bufferTotal.Load()),
   137  		)
   138  	}
   139  }
   140  
   141  func (b *bufferHolder) discardBuffer(buffer batchpipe.ItemBuffer[batchpipe.HasName, any]) {
   142  	b.discardCnt.Add(1)
   143  	b.putBuffer(buffer)
   144  }
   145  
   146  // Add call buffer.Add(), while bufferHolder is NOT readonly
   147  func (b *bufferHolder) Add(item batchpipe.HasName) {
   148  	b.mux.Lock()
   149  	if b.stopped {
   150  		b.mux.Unlock()
   151  		return
   152  	}
   153  	if b.buffer == nil {
   154  		b.buffer = b.getBuffer()
   155  	}
   156  	buf := b.buffer
   157  	buf.Add(item)
   158  	b.mux.Unlock()
   159  	if buf.ShouldFlush() {
   160  		b.signal(b)
   161  	} else if checker, is := item.(table.NeedSyncWrite); is && checker.NeedSyncWrite() {
   162  		b.signal(b)
   163  	}
   164  }
   165  
   166  var _ generateReq = (*bufferGenerateReq)(nil)
   167  
   168  type bufferGenerateReq struct {
   169  	buffer batchpipe.ItemBuffer[batchpipe.HasName, any]
   170  	// impl NewItemBatchHandler
   171  	b *bufferHolder
   172  }
   173  
   174  func (r *bufferGenerateReq) handle(buf *bytes.Buffer) (exportReq, error) {
   175  	batch := r.buffer.GetBatch(r.b.ctx, buf)
   176  	r.b.putBuffer(r.buffer)
   177  	return &bufferExportReq{
   178  		batch: batch,
   179  		b:     r.b,
   180  	}, nil
   181  }
   182  
   183  func (r *bufferGenerateReq) callback(err error) {}
   184  
   185  var _ exportReq = (*bufferExportReq)(nil)
   186  
   187  type bufferExportReq struct {
   188  	batch any
   189  	b     *bufferHolder
   190  }
   191  
   192  func (r *bufferExportReq) handle() error {
   193  	if r.batch != nil {
   194  		var flush = r.b.impl.NewItemBatchHandler(context.Background())
   195  		flush(r.batch)
   196  	} else {
   197  		r.b.c.logger.Debug("batch is nil", zap.String("item", r.b.name))
   198  	}
   199  	return nil
   200  }
   201  
   202  func (r *bufferExportReq) callback(err error) {}
   203  
   204  // getGenerateReq get req to do generate logic
   205  // return nil, if b.buffer is nil
   206  func (b *bufferHolder) getGenerateReq() generateReq {
   207  	b.mux.Lock()
   208  	defer b.mux.Unlock()
   209  	defer b.resetTrigger()
   210  	if b.buffer == nil || b.buffer.IsEmpty() {
   211  		return nil
   212  	}
   213  	req := &bufferGenerateReq{
   214  		buffer: b.buffer,
   215  		b:      b,
   216  	}
   217  	b.buffer = nil
   218  	return req
   219  }
   220  
   221  func (b *bufferHolder) resetTrigger() {
   222  	if b.stopped {
   223  		return
   224  	}
   225  	b.trigger.Reset(b.reminder.RemindNextAfter())
   226  }
   227  
   228  func (b *bufferHolder) Stop() {
   229  	b.mux.Lock()
   230  	defer b.mux.Unlock()
   231  	b.stopped = true
   232  	b.trigger.Stop()
   233  }
   234  
   235  var _ motrace.BatchProcessor = (*MOCollector)(nil)
   236  
   237  // MOCollector handle all bufferPipe
   238  type MOCollector struct {
   239  	motrace.BatchProcessor
   240  	ctx    context.Context
   241  	logger *log.MOLogger
   242  
   243  	// mux control all changes on buffers
   244  	mux sync.RWMutex
   245  	// buffers maintain working buffer for each type
   246  	buffers map[string]*bufferHolder
   247  	// awakeCollect handle collect signal
   248  	awakeCollect chan batchpipe.HasName
   249  	// awakeGenerate handle generate signal
   250  	awakeGenerate chan generateReq
   251  	// awakeBatch handle export signal
   252  	awakeBatch chan exportReq
   253  
   254  	collectorCnt int // WithCollectorCnt
   255  	generatorCnt int // WithGeneratorCnt
   256  	exporterCnt  int // WithExporterCnt
   257  	// num % of (#cpu * 0.1)
   258  	collectorCntP int // default: 10
   259  	generatorCntP int // default: 20
   260  	exporterCntP  int // default: 80
   261  	// pipeImplHolder hold implement
   262  	pipeImplHolder *PipeImplHolder
   263  
   264  	statsInterval time.Duration // WithStatsInterval
   265  
   266  	maxBufferCnt int32 // cooperate with bufferCond
   267  	bufferTotal  atomic.Int32
   268  	bufferMux    sync.Mutex
   269  	bufferCond   *sync.Cond
   270  
   271  	// flow control
   272  	started  uint32
   273  	stopOnce sync.Once
   274  	stopWait sync.WaitGroup
   275  	stopCh   chan struct{}
   276  	stopDrop atomic.Uint64
   277  }
   278  
   279  type MOCollectorOption func(*MOCollector)
   280  
   281  func NewMOCollector(ctx context.Context, opts ...MOCollectorOption) *MOCollector {
   282  	c := &MOCollector{
   283  		ctx:            ctx,
   284  		logger:         morun.ProcessLevelRuntime().Logger().Named(LoggerNameMOCollector).With(logutil.Discardable()),
   285  		buffers:        make(map[string]*bufferHolder),
   286  		awakeCollect:   make(chan batchpipe.HasName, defaultQueueSize),
   287  		awakeGenerate:  make(chan generateReq, 16),
   288  		awakeBatch:     make(chan exportReq),
   289  		stopCh:         make(chan struct{}),
   290  		collectorCntP:  10,
   291  		generatorCntP:  20,
   292  		exporterCntP:   80,
   293  		pipeImplHolder: newPipeImplHolder(),
   294  		statsInterval:  time.Minute,
   295  		maxBufferCnt:   int32(runtime.NumCPU()),
   296  	}
   297  	c.bufferCond = sync.NewCond(&c.bufferMux)
   298  	for _, opt := range opts {
   299  		opt(c)
   300  	}
   301  	// calculate collectorCnt, generatorCnt, exporterCnt
   302  	c.calculateDefaultWorker(runtime.NumCPU())
   303  	return c
   304  }
   305  
   306  const maxPercentValue = 1000
   307  
   308  // calculateDefaultWorker
   309  // totalNum = int( #cpu * 0.1 + 0.5 )
   310  // default collectorCntP : generatorCntP : exporterCntP = 10 : 20 : 80.
   311  // It means collectorCnt = int( $totalNum * $collectorCntP / 100 + 0.5 )
   312  //
   313  // For example
   314  // | #cpu | #totalNum | collectorCnt | generatorCnt | exporterCnt |
   315  // | --   | --        | --           | --           | --          |
   316  // | 6    | 0.6 =~ 1  | 1            | 1            | 3           |
   317  // | 30   | 3.0       | 1            | 1            | 2           |
   318  // | 50   | 5.0       | 1            | 1            | 4           |
   319  // | 60   | 6.0       | 1            | 2            | 5           |
   320  func (c *MOCollector) calculateDefaultWorker(numCpu int) {
   321  	var totalNum = math.Ceil(float64(numCpu) * 0.1)
   322  	unit := float64(totalNum) / (100.0)
   323  	// set default value if non-set
   324  	c.collectorCnt = int(math.Round(unit * float64(c.collectorCntP)))
   325  	c.generatorCnt = int(math.Round(unit * float64(c.generatorCntP)))
   326  	c.exporterCnt = int(math.Round(unit * float64(c.exporterCntP)))
   327  	if c.collectorCnt <= 0 {
   328  		c.collectorCnt = 1
   329  	}
   330  	if c.generatorCnt <= 0 {
   331  		c.generatorCnt = 1
   332  	}
   333  	if c.exporterCnt <= 0 {
   334  		c.exporterCnt = 1
   335  	}
   336  
   337  	// check max value < numCpu
   338  	if c.collectorCnt > numCpu {
   339  		c.collectorCnt = numCpu
   340  	}
   341  	if c.generatorCnt > numCpu {
   342  		c.generatorCnt = numCpu
   343  	}
   344  	if c.exporterCnt > numCpu {
   345  		c.exporterCnt = numCpu
   346  	}
   347  
   348  	// last check: disable calculation
   349  	if c.collectorCnt >= maxPercentValue {
   350  		c.collectorCnt = numCpu
   351  	}
   352  	if c.generatorCntP >= maxPercentValue {
   353  		c.generatorCnt = numCpu
   354  	}
   355  	if c.exporterCnt >= maxPercentValue {
   356  		c.generatorCnt = numCpu
   357  	}
   358  }
   359  
   360  func WithCollectorCntP(p int) MOCollectorOption {
   361  	return MOCollectorOption(func(c *MOCollector) { c.collectorCntP = p })
   362  }
   363  func WithGeneratorCntP(p int) MOCollectorOption {
   364  	return MOCollectorOption(func(c *MOCollector) { c.generatorCntP = p })
   365  }
   366  func WithExporterCntP(p int) MOCollectorOption {
   367  	return MOCollectorOption(func(c *MOCollector) { c.exporterCntP = p })
   368  }
   369  
   370  func WithOBCollectorConfig(cfg *config.OBCollectorConfig) MOCollectorOption {
   371  	return MOCollectorOption(func(c *MOCollector) {
   372  		c.statsInterval = cfg.ShowStatsInterval.Duration
   373  		c.maxBufferCnt = cfg.BufferCnt
   374  		if c.maxBufferCnt == -1 {
   375  			c.maxBufferCnt = math.MaxInt32
   376  		} else if c.maxBufferCnt == 0 {
   377  			c.maxBufferCnt = int32(runtime.NumCPU())
   378  		}
   379  		c.collectorCntP = cfg.CollectorCntPercent
   380  		c.generatorCntP = cfg.GeneratorCntPercent
   381  		c.exporterCntP = cfg.ExporterCntPercent
   382  	})
   383  }
   384  
   385  func (c *MOCollector) initCnt() {
   386  	if c.collectorCnt <= 0 {
   387  		c.collectorCnt = c.pipeImplHolder.Size() * 2
   388  	}
   389  	if c.generatorCnt <= 0 {
   390  		c.generatorCnt = c.pipeImplHolder.Size()
   391  	}
   392  	if c.exporterCnt <= 0 {
   393  		c.exporterCnt = c.pipeImplHolder.Size()
   394  	}
   395  }
   396  
   397  func (c *MOCollector) Register(name batchpipe.HasName, impl motrace.PipeImpl) {
   398  	_ = c.pipeImplHolder.Put(name.GetName(), impl)
   399  }
   400  
   401  // Collect item in chan, if collector is stopped then return error
   402  func (c *MOCollector) Collect(ctx context.Context, item batchpipe.HasName) error {
   403  	select {
   404  	case <-c.stopCh:
   405  		c.stopDrop.Add(1)
   406  		ctx = errutil.ContextWithNoReport(ctx, true)
   407  		return moerr.NewInternalError(ctx, "MOCollector stopped")
   408  	case c.awakeCollect <- item:
   409  		return nil
   410  	}
   411  }
   412  
   413  // DiscardableCollect implements motrace.DiscardableCollector
   414  // cooperate with logutil.Discardable() field
   415  func (c *MOCollector) DiscardableCollect(ctx context.Context, item batchpipe.HasName) error {
   416  	select {
   417  	case <-c.stopCh:
   418  		c.stopDrop.Add(1)
   419  		ctx = errutil.ContextWithNoReport(ctx, true)
   420  		return moerr.NewInternalError(ctx, "MOCollector stopped")
   421  	case c.awakeCollect <- item:
   422  		return nil
   423  	case <-time.After(discardCollectTimeout):
   424  		return nil
   425  	}
   426  }
   427  
   428  // Start all goroutine worker, including collector, generator, and exporter
   429  func (c *MOCollector) Start() bool {
   430  	if atomic.LoadUint32(&c.started) != 0 {
   431  		return false
   432  	}
   433  	c.mux.Lock()
   434  	defer c.mux.Unlock()
   435  	if c.started != 0 {
   436  		return false
   437  	}
   438  	defer atomic.StoreUint32(&c.started, 1)
   439  
   440  	c.initCnt()
   441  
   442  	c.logger.Info("MOCollector Start")
   443  	for i := 0; i < c.collectorCnt; i++ {
   444  		c.stopWait.Add(1)
   445  		go c.doCollect(i)
   446  	}
   447  	for i := 0; i < c.generatorCnt; i++ {
   448  		c.stopWait.Add(1)
   449  		go c.doGenerate(i)
   450  	}
   451  	for i := 0; i < c.exporterCnt; i++ {
   452  		c.stopWait.Add(1)
   453  		go c.doExport(i)
   454  	}
   455  	c.stopWait.Add(1)
   456  	go c.showStats()
   457  	return true
   458  }
   459  
   460  func (c *MOCollector) allocBuffer() {
   461  	c.bufferCond.L.Lock()
   462  	for c.bufferTotal.Load() == c.maxBufferCnt {
   463  		c.bufferCond.Wait()
   464  	}
   465  	c.bufferTotal.Add(1)
   466  	c.bufferCond.L.Unlock()
   467  }
   468  
   469  func (c *MOCollector) releaseBuffer() {
   470  	c.bufferCond.L.Lock()
   471  	c.bufferTotal.Add(-1)
   472  	c.bufferCond.Signal()
   473  	c.bufferCond.L.Unlock()
   474  }
   475  
   476  // doCollect handle all item accept work, send it to the corresponding buffer
   477  // goroutine worker
   478  func (c *MOCollector) doCollect(idx int) {
   479  	defer c.stopWait.Done()
   480  	ctx, span := trace.Start(c.ctx, "MOCollector.doCollect")
   481  	defer span.End()
   482  	c.logger.Debug("doCollect %dth: start", zap.Int("idx", idx))
   483  loop:
   484  	for {
   485  		select {
   486  		case i := <-c.awakeCollect:
   487  			start := time.Now()
   488  			c.mux.RLock()
   489  			if buf, has := c.buffers[i.GetName()]; !has {
   490  				c.logger.Debug("doCollect: init buffer", zap.Int("idx", idx), zap.String("item", i.GetName()))
   491  				c.mux.RUnlock()
   492  				c.mux.Lock()
   493  				if _, has := c.buffers[i.GetName()]; !has {
   494  					c.logger.Debug("doCollect: init buffer done.", zap.Int("idx", idx))
   495  					if impl, has := c.pipeImplHolder.Get(i.GetName()); !has {
   496  						c.logger.Panic("unknown item type", zap.String("item", i.GetName()))
   497  					} else {
   498  						buf = newBufferHolder(ctx, i, impl, awakeBufferFactory(c), c)
   499  						c.buffers[i.GetName()] = buf
   500  						buf.Add(i)
   501  						buf.Start()
   502  					}
   503  				}
   504  				c.mux.Unlock()
   505  			} else {
   506  				buf.Add(i)
   507  				c.mux.RUnlock()
   508  			}
   509  			v2.TraceCollectorCollectDurationHistogram.Observe(time.Since(start).Seconds())
   510  		case <-c.stopCh:
   511  			break loop
   512  		}
   513  	}
   514  	c.logger.Debug("doCollect: Done.", zap.Int("idx", idx))
   515  }
   516  
   517  type generateReq interface {
   518  	handle(*bytes.Buffer) (exportReq, error)
   519  	callback(error)
   520  }
   521  
   522  type exportReq interface {
   523  	handle() error
   524  	callback(error)
   525  }
   526  
   527  // awakeBufferFactory frozen buffer, send GenRequest to awake
   528  var awakeBufferFactory = func(c *MOCollector) func(holder *bufferHolder) {
   529  	return func(holder *bufferHolder) {
   530  		start := time.Now()
   531  		defer func() {
   532  			v2.TraceCollectorGenerateAwareDurationHistogram.Observe(time.Since(start).Seconds())
   533  		}()
   534  		req := holder.getGenerateReq()
   535  		if req == nil {
   536  			return
   537  		}
   538  		if holder.name != motrace.RawLogTbl {
   539  			select {
   540  			case c.awakeGenerate <- req:
   541  			case <-time.After(time.Second * 3):
   542  				c.logger.Warn("awakeBufferFactory: timeout after 3 seconds")
   543  				goto discardL
   544  			}
   545  		} else {
   546  			select {
   547  			case c.awakeGenerate <- req:
   548  			default:
   549  				c.logger.Warn("awakeBufferFactory: awakeGenerate chan is full")
   550  				goto discardL
   551  			}
   552  		}
   553  		return
   554  	discardL:
   555  		if r, ok := req.(*bufferGenerateReq); ok {
   556  			r.b.discardBuffer(r.buffer)
   557  		}
   558  		v2.TraceCollectorGenerateAwareDiscardDurationHistogram.Observe(time.Since(start).Seconds())
   559  	}
   560  }
   561  
   562  // doGenerate handle buffer gen BatchRequest, which could be anything
   563  // goroutine worker
   564  func (c *MOCollector) doGenerate(idx int) {
   565  	defer c.stopWait.Done()
   566  	var buf = new(bytes.Buffer)
   567  	c.logger.Debug("doGenerate start", zap.Int("idx", idx))
   568  loop:
   569  	for {
   570  		select {
   571  		case req := <-c.awakeGenerate:
   572  			start := time.Now()
   573  			if req == nil {
   574  				c.logger.Warn("generate req is nil")
   575  			} else if exportReq, err := req.handle(buf); err != nil {
   576  				req.callback(err)
   577  				v2.TraceCollectorGenerateDurationHistogram.Observe(time.Since(start).Seconds())
   578  			} else {
   579  				startDelay := time.Now()
   580  				select {
   581  				case c.awakeBatch <- exportReq:
   582  				case <-c.stopCh:
   583  				case <-time.After(time.Second * 10):
   584  					c.logger.Info("awakeBatch: timeout after 10 seconds")
   585  					v2.TraceCollectorGenerateDiscardDurationHistogram.Observe(time.Since(start).Seconds())
   586  				}
   587  				end := time.Now()
   588  				v2.TraceCollectorGenerateDelayDurationHistogram.Observe(end.Sub(startDelay).Seconds())
   589  				v2.TraceCollectorGenerateDurationHistogram.Observe(end.Sub(start).Seconds())
   590  			}
   591  		case <-c.stopCh:
   592  			break loop
   593  		}
   594  	}
   595  	c.logger.Debug("doGenerate: Done.", zap.Int("idx", idx))
   596  }
   597  
   598  // doExport handle BatchRequest
   599  func (c *MOCollector) doExport(idx int) {
   600  	defer c.stopWait.Done()
   601  	c.logger.Debug("doExport %dth: start", zap.Int("idx", idx))
   602  loop:
   603  	for {
   604  		select {
   605  		case req := <-c.awakeBatch:
   606  			start := time.Now()
   607  			if req == nil {
   608  				c.logger.Warn("export req is nil")
   609  			} else if err := req.handle(); err != nil {
   610  				req.callback(err)
   611  			}
   612  			v2.TraceCollectorExportDurationHistogram.Observe(time.Since(start).Seconds())
   613  		case <-c.stopCh:
   614  			c.mux.Lock()
   615  			for len(c.awakeBatch) > 0 {
   616  				<-c.awakeBatch
   617  			}
   618  			c.mux.Unlock()
   619  			break loop
   620  		}
   621  	}
   622  	c.logger.Debug("doExport Done.", zap.Int("idx", idx))
   623  }
   624  
   625  func (c *MOCollector) showStats() {
   626  	defer c.stopWait.Done()
   627  	c.logger.Debug("start showStats")
   628  
   629  loop:
   630  	for {
   631  		select {
   632  		case <-time.After(c.statsInterval):
   633  			fields := make([]zap.Field, 0, 16)
   634  			fields = append(fields, zap.Int32("MaxBufferCnt", c.maxBufferCnt))
   635  			fields = append(fields, zap.Int32("TotalBufferCnt", c.bufferTotal.Load()))
   636  			fields = append(fields, zap.Int("QueueLength", len(c.awakeCollect)))
   637  			for _, b := range c.buffers {
   638  				fields = append(fields, zap.Int32(fmt.Sprintf("%sBufferCnt", b.name), b.bufferCnt.Load()))
   639  				fields = append(fields, zap.Int32(fmt.Sprintf("%sDiscardCnt", b.name), b.discardCnt.Load()))
   640  			}
   641  			c.logger.Info("stats", fields...)
   642  		case <-c.stopCh:
   643  			break loop
   644  		}
   645  	}
   646  	c.logger.Debug("showStats Done.")
   647  }
   648  
   649  func (c *MOCollector) Stop(graceful bool) error {
   650  	var err error
   651  	var buf = new(bytes.Buffer)
   652  	c.stopOnce.Do(func() {
   653  		for len(c.awakeCollect) > 0 && graceful {
   654  			c.logger.Debug(fmt.Sprintf("doCollect left %d job", len(c.awakeCollect)))
   655  			time.Sleep(250 * time.Millisecond)
   656  		}
   657  		c.mux.Lock()
   658  		for _, buffer := range c.buffers {
   659  			buffer.Stop()
   660  		}
   661  		c.mux.Unlock()
   662  		close(c.stopCh)
   663  		c.stopWait.Wait()
   664  		close(c.awakeGenerate)
   665  		close(c.awakeBatch)
   666  		if !graceful {
   667  			// shutdown directly
   668  			return
   669  		}
   670  		// handle remain data
   671  		handleExport := func(req exportReq) {
   672  			if err = req.handle(); err != nil {
   673  				req.callback(err)
   674  			}
   675  		}
   676  		handleGen := func(req generateReq) {
   677  			if export, err := req.handle(buf); err != nil {
   678  				req.callback(err)
   679  			} else {
   680  				handleExport(export)
   681  			}
   682  		}
   683  		for req := range c.awakeBatch {
   684  			handleExport(req)
   685  		}
   686  		for req := range c.awakeGenerate {
   687  			handleGen(req)
   688  		}
   689  		for _, buffer := range c.buffers {
   690  			if generate := buffer.getGenerateReq(); generate != nil {
   691  				handleGen(generate)
   692  			}
   693  		}
   694  	})
   695  	return err
   696  }
   697  
   698  type PipeImplHolder struct {
   699  	mux   sync.RWMutex
   700  	impls map[string]motrace.PipeImpl
   701  }
   702  
   703  func newPipeImplHolder() *PipeImplHolder {
   704  	return &PipeImplHolder{
   705  		impls: make(map[string]motrace.PipeImpl),
   706  	}
   707  }
   708  
   709  func (h *PipeImplHolder) Get(name string) (motrace.PipeImpl, bool) {
   710  	h.mux.RLock()
   711  	defer h.mux.RUnlock()
   712  	impl, has := h.impls[name]
   713  	return impl, has
   714  }
   715  
   716  func (h *PipeImplHolder) Put(name string, impl motrace.PipeImpl) bool {
   717  	h.mux.Lock()
   718  	defer h.mux.Unlock()
   719  	_, has := h.impls[name]
   720  	h.impls[name] = impl
   721  	return has
   722  }
   723  
   724  func (h *PipeImplHolder) Size() int {
   725  	h.mux.Lock()
   726  	defer h.mux.Unlock()
   727  	return len(h.impls)
   728  }