github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/common/flow_control.go (about)

     1  // Copyright 2021 PingCAP, Inc.
     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  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package common
    15  
    16  import (
    17  	"log"
    18  	"sync"
    19  	"sync/atomic"
    20  
    21  	"github.com/edwingeng/deque"
    22  	"github.com/pingcap/errors"
    23  	cerrors "github.com/pingcap/ticdc/pkg/errors"
    24  	"go.uber.org/zap"
    25  )
    26  
    27  // TableMemoryQuota is designed to curb the total memory consumption of processing
    28  // the event streams in a table.
    29  // A higher-level controller more suitable for direct use by the processor is TableFlowController.
    30  type TableMemoryQuota struct {
    31  	Quota uint64 // should not be changed once intialized
    32  
    33  	IsAborted uint32
    34  
    35  	mu       sync.Mutex
    36  	Consumed uint64
    37  
    38  	cond *sync.Cond
    39  }
    40  
    41  // NewTableMemoryQuota creates a new TableMemoryQuota
    42  // quota: max advised memory consumption in bytes.
    43  func NewTableMemoryQuota(quota uint64) *TableMemoryQuota {
    44  	ret := &TableMemoryQuota{
    45  		Quota:    quota,
    46  		mu:       sync.Mutex{},
    47  		Consumed: 0,
    48  	}
    49  
    50  	ret.cond = sync.NewCond(&ret.mu)
    51  	return ret
    52  }
    53  
    54  // ConsumeWithBlocking is called when a hard-limit is needed. The method will
    55  // block until enough memory has been freed up by Release.
    56  // blockCallBack will be called if the function will block.
    57  // Should be used with care to prevent deadlock.
    58  func (c *TableMemoryQuota) ConsumeWithBlocking(nBytes uint64, blockCallBack func() error) error {
    59  	if nBytes >= c.Quota {
    60  		return cerrors.ErrFlowControllerEventLargerThanQuota.GenWithStackByArgs(nBytes, c.Quota)
    61  	}
    62  
    63  	c.mu.Lock()
    64  	if c.Consumed+nBytes >= c.Quota {
    65  		c.mu.Unlock()
    66  		err := blockCallBack()
    67  		if err != nil {
    68  			return errors.Trace(err)
    69  		}
    70  	} else {
    71  		c.mu.Unlock()
    72  	}
    73  
    74  	c.mu.Lock()
    75  	defer c.mu.Unlock()
    76  
    77  	for {
    78  		if atomic.LoadUint32(&c.IsAborted) == 1 {
    79  			return cerrors.ErrFlowControllerAborted.GenWithStackByArgs()
    80  		}
    81  
    82  		if c.Consumed+nBytes < c.Quota {
    83  			break
    84  		}
    85  		c.cond.Wait()
    86  	}
    87  
    88  	c.Consumed += nBytes
    89  	return nil
    90  }
    91  
    92  // ForceConsume is called when blocking is not acceptable and the limit can be violated
    93  // for the sake of avoid deadlock. It merely records the increased memory consumption.
    94  func (c *TableMemoryQuota) ForceConsume(nBytes uint64) error {
    95  	c.mu.Lock()
    96  	defer c.mu.Unlock()
    97  
    98  	if atomic.LoadUint32(&c.IsAborted) == 1 {
    99  		return cerrors.ErrFlowControllerAborted.GenWithStackByArgs()
   100  	}
   101  
   102  	c.Consumed += nBytes
   103  	return nil
   104  }
   105  
   106  // Release is called when a chuck of memory is done being used.
   107  func (c *TableMemoryQuota) Release(nBytes uint64) {
   108  	c.mu.Lock()
   109  
   110  	if c.Consumed < nBytes {
   111  		c.mu.Unlock()
   112  		log.Panic("TableMemoryQuota: releasing more than consumed, report a bug",
   113  			zap.Uint64("consumed", c.Consumed),
   114  			zap.Uint64("released", nBytes))
   115  	}
   116  
   117  	c.Consumed -= nBytes
   118  	if c.Consumed < c.Quota {
   119  		c.mu.Unlock()
   120  		c.cond.Signal()
   121  		return
   122  	}
   123  
   124  	c.mu.Unlock()
   125  }
   126  
   127  // Abort interrupts any ongoing ConsumeWithBlocking call
   128  func (c *TableMemoryQuota) Abort() {
   129  	atomic.StoreUint32(&c.IsAborted, 1)
   130  	c.cond.Signal()
   131  }
   132  
   133  // GetConsumption returns the current memory consumption
   134  func (c *TableMemoryQuota) GetConsumption() uint64 {
   135  	c.mu.Lock()
   136  	defer c.mu.Unlock()
   137  
   138  	return c.Consumed
   139  }
   140  
   141  // TableFlowController provides a convenient interface to control the memory consumption of a per table event stream
   142  type TableFlowController struct {
   143  	memoryQuota *TableMemoryQuota
   144  
   145  	mu    sync.Mutex
   146  	queue deque.Deque
   147  
   148  	lastCommitTs uint64
   149  }
   150  
   151  type commitTsSizeEntry struct {
   152  	CommitTs uint64
   153  	Size     uint64
   154  }
   155  
   156  // NewTableFlowController creates a new TableFlowController
   157  func NewTableFlowController(quota uint64) *TableFlowController {
   158  	return &TableFlowController{
   159  		memoryQuota: NewTableMemoryQuota(quota),
   160  		queue:       deque.NewDeque(),
   161  	}
   162  }
   163  
   164  // Consume is called when an event has arrived for being processed by the sink.
   165  // It will handle transaction boundaries automatically, and will not block intra-transaction.
   166  func (c *TableFlowController) Consume(commitTs uint64, size uint64, blockCallBack func() error) error {
   167  	lastCommitTs := atomic.LoadUint64(&c.lastCommitTs)
   168  
   169  	if commitTs < lastCommitTs {
   170  		log.Panic("commitTs regressed, report a bug",
   171  			zap.Uint64("commitTs", commitTs),
   172  			zap.Uint64("lastCommitTs", c.lastCommitTs))
   173  	}
   174  
   175  	if commitTs > lastCommitTs {
   176  		atomic.StoreUint64(&c.lastCommitTs, commitTs)
   177  		err := c.memoryQuota.ConsumeWithBlocking(size, blockCallBack)
   178  		if err != nil {
   179  			return errors.Trace(err)
   180  		}
   181  	} else {
   182  		// Here commitTs == lastCommitTs, which means that we are not crossing
   183  		// a transaction boundary. In this situation, we use `ForceConsume` because
   184  		// blocking the event stream mid-transaction is highly likely to cause
   185  		// a deadlock.
   186  		// TODO fix this in the future, after we figure out how to elegantly support large txns.
   187  		err := c.memoryQuota.ForceConsume(size)
   188  		if err != nil {
   189  			return errors.Trace(err)
   190  		}
   191  	}
   192  
   193  	c.mu.Lock()
   194  	defer c.mu.Unlock()
   195  	c.queue.PushBack(&commitTsSizeEntry{
   196  		CommitTs: commitTs,
   197  		Size:     size,
   198  	})
   199  
   200  	return nil
   201  }
   202  
   203  // Release is called when all events committed before resolvedTs has been freed from memory.
   204  func (c *TableFlowController) Release(resolvedTs uint64) {
   205  	var nBytesToRelease uint64
   206  
   207  	c.mu.Lock()
   208  	for c.queue.Len() > 0 {
   209  		if peeked := c.queue.Front().(*commitTsSizeEntry); peeked.CommitTs <= resolvedTs {
   210  			nBytesToRelease += peeked.Size
   211  			c.queue.PopFront()
   212  		} else {
   213  			break
   214  		}
   215  	}
   216  	c.mu.Unlock()
   217  
   218  	c.memoryQuota.Release(nBytesToRelease)
   219  }
   220  
   221  // Abort interrupts any ongoing Consume call
   222  func (c *TableFlowController) Abort() {
   223  	c.memoryQuota.Abort()
   224  }
   225  
   226  // GetConsumption returns the current memory consumption
   227  func (c *TableFlowController) GetConsumption() uint64 {
   228  	return c.memoryQuota.GetConsumption()
   229  }