github.com/matrixorigin/matrixone@v1.2.0/pkg/txn/client/client.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 client
    16  
    17  import (
    18  	"context"
    19  	"encoding/hex"
    20  	"math"
    21  	"runtime/debug"
    22  	"sync"
    23  	"sync/atomic"
    24  	"time"
    25  
    26  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    27  	"github.com/matrixorigin/matrixone/pkg/common/runtime"
    28  	"github.com/matrixorigin/matrixone/pkg/lockservice"
    29  	"github.com/matrixorigin/matrixone/pkg/pb/timestamp"
    30  	"github.com/matrixorigin/matrixone/pkg/pb/txn"
    31  	"github.com/matrixorigin/matrixone/pkg/txn/clock"
    32  	"github.com/matrixorigin/matrixone/pkg/txn/rpc"
    33  	"github.com/matrixorigin/matrixone/pkg/txn/util"
    34  	v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2"
    35  	"go.uber.org/ratelimit"
    36  	"go.uber.org/zap"
    37  )
    38  
    39  // WithTxnIDGenerator setup txn id generator
    40  func WithTxnIDGenerator(generator TxnIDGenerator) TxnClientCreateOption {
    41  	return func(tc *txnClient) {
    42  		tc.generator = generator
    43  	}
    44  }
    45  
    46  // WithLockService setup lock service
    47  func WithLockService(lockService lockservice.LockService) TxnClientCreateOption {
    48  	return func(tc *txnClient) {
    49  		tc.lockService = lockService
    50  	}
    51  }
    52  
    53  // WithEnableSacrificingFreshness sacrifice freshness to reduce the waiting time for transaction start,
    54  // which will help to improve the latency of the transaction, but will sacrifice some data freshness.
    55  //
    56  // In Push mode, TN will bba to push Logtail to CN, if we need to guarantee the freshness of data, then
    57  // we need to use the current latest time as the start time of the transaction, this will ensure that
    58  // enough logtail is collected before the transaction read/write starts, but this will have some delayed
    59  // waiting time.
    60  //
    61  // But if we can accept to sacrifice some data freshness, we can optimize this waiting time, we just need to
    62  // use the latest logtail timestamp received + 1 as the transaction start timestamp, so we can not wait.
    63  
    64  // When making this optimization, there are some scenarios where data consistency must be guaranteed, such as
    65  // a database connection in a session where the latter transaction must be able to read the data committed by
    66  // the previous transaction, then it is necessary to maintain a Session-level transaction last commit time, and
    67  // the start time of the next transaction cannot be less than this value.
    68  //
    69  // If we need to ensure that all the transactions on a CN can read the writes of the previous committed
    70  // transaction, then we can use WithEnableCNBasedConsistency to turn on.
    71  func WithEnableSacrificingFreshness() TxnClientCreateOption {
    72  	return func(tc *txnClient) {
    73  		tc.enableSacrificingFreshness = true
    74  	}
    75  }
    76  
    77  // WithTimestampWaiter setup timestamp waiter to get the latest applied committed timestamp from logtail.
    78  func WithTimestampWaiter(waiter TimestampWaiter) TxnClientCreateOption {
    79  	return func(tc *txnClient) {
    80  		tc.timestampWaiter = waiter
    81  	}
    82  }
    83  
    84  // WithEnableCNBasedConsistency let all transactions on a CN see writes committed by other
    85  // transactions before them. When this feature is enabled, the client maintains a CN-Based
    86  // commit timestamp, and when opening a new transaction, it adjusts the transaction's snapshot
    87  // timestamp to at least >= lastCommitTimestamp, so that it can see the writes of the previously
    88  // committed transaction
    89  func WithEnableCNBasedConsistency() TxnClientCreateOption {
    90  	return func(tc *txnClient) {
    91  		tc.enableCNBasedConsistency = true
    92  	}
    93  }
    94  
    95  // WithEnableRefreshExpression in RC mode, in the event of a conflict, the later transaction needs
    96  // to see the latest data after the previous transaction commits. At this time we need to re-read
    97  // the data, re-read the latest data, and re-compute the expression.
    98  func WithEnableRefreshExpression() TxnClientCreateOption {
    99  	return func(tc *txnClient) {
   100  		tc.enableRefreshExpression = true
   101  	}
   102  }
   103  
   104  // WithEnableLeakCheck enable txn leak check. Used to found any txn is not committed or rolled back.
   105  func WithEnableLeakCheck(
   106  	maxActiveAges time.Duration,
   107  	leakHandleFunc func([]ActiveTxn)) TxnClientCreateOption {
   108  	return func(tc *txnClient) {
   109  		tc.leakChecker = newLeakCheck(maxActiveAges, leakHandleFunc)
   110  	}
   111  }
   112  
   113  // WithTxnLimit flow control of transaction creation, maximum number of transactions per second
   114  func WithTxnLimit(n int) TxnClientCreateOption {
   115  	return func(tc *txnClient) {
   116  		tc.limiter = ratelimit.New(n, ratelimit.Per(time.Second))
   117  	}
   118  }
   119  
   120  // WithMaxActiveTxn is the count of max active txn in current cn.  If reached max value, the txn is
   121  // added to a FIFO queue. Default is unlimited.
   122  func WithMaxActiveTxn(n int) TxnClientCreateOption {
   123  	return func(tc *txnClient) {
   124  		tc.maxActiveTxn = n
   125  	}
   126  }
   127  
   128  // WithNormalStateNoWait sets the normalStateNoWait value of txnClient.
   129  func WithNormalStateNoWait(t bool) TxnClientCreateOption {
   130  	return func(tc *txnClient) {
   131  		tc.normalStateNoWait = t
   132  	}
   133  }
   134  
   135  func WithTxnOpenedCallback(callbacks []func(op TxnOperator)) TxnClientCreateOption {
   136  	return func(tc *txnClient) {
   137  		tc.txnOpenedCallbacks = callbacks
   138  	}
   139  }
   140  
   141  func WithCheckDup() TxnClientCreateOption {
   142  	return func(tc *txnClient) {
   143  		tc.enableCheckDup = true
   144  	}
   145  }
   146  
   147  var _ TxnClient = (*txnClient)(nil)
   148  
   149  type status int
   150  
   151  const (
   152  	paused        = status(0)
   153  	normal status = status(1)
   154  )
   155  
   156  type txnClient struct {
   157  	clock                      clock.Clock
   158  	sender                     rpc.TxnSender
   159  	generator                  TxnIDGenerator
   160  	lockService                lockservice.LockService
   161  	timestampWaiter            TimestampWaiter
   162  	leakChecker                *leakChecker
   163  	limiter                    ratelimit.Limiter
   164  	maxActiveTxn               int
   165  	enableCheckDup             bool
   166  	enableCNBasedConsistency   bool
   167  	enableSacrificingFreshness bool
   168  	enableRefreshExpression    bool
   169  	txnOpenedCallbacks         []func(TxnOperator)
   170  
   171  	// normalStateNoWait is used to control if wait for the txn client's
   172  	// state to be normal. If it is false, which is default value, wait
   173  	// until the txn client's state to be normal; otherwise, if it is true,
   174  	// do not wait, and just return an error.
   175  	normalStateNoWait bool
   176  
   177  	atomic struct {
   178  		// we maintain a CN-based last commit timestamp to ensure that
   179  		// a txn with that CN can see previous writes.
   180  		// FIXME(fagongzi): this is a remedial solution to disable the
   181  		// cn-based commit ts when the session-level last commit ts have
   182  		// been processed.
   183  		latestCommitTS atomic.Pointer[timestamp.Timestamp]
   184  		// just for bvt testing
   185  		forceSyncCommitTimes atomic.Uint64
   186  	}
   187  
   188  	mu struct {
   189  		sync.RWMutex
   190  		// cond is used to control if we can create new txn and notify
   191  		// if the state is changed.
   192  		cond *sync.Cond
   193  		// indicate whether the CN can provide service normally.
   194  		state status
   195  		// user active txns
   196  		users int
   197  		// all active txns
   198  		activeTxns map[string]*txnOperator
   199  		// FIFO queue for ready to active txn
   200  		waitActiveTxns []*txnOperator
   201  	}
   202  }
   203  
   204  func (client *txnClient) GetState() TxnState {
   205  	client.mu.Lock()
   206  	defer client.mu.Unlock()
   207  	at := make([]string, 0, len(client.mu.activeTxns))
   208  	for k := range client.mu.activeTxns {
   209  		at = append(at, hex.EncodeToString([]byte(k)))
   210  	}
   211  	wt := make([]string, 0, len(client.mu.waitActiveTxns))
   212  	for _, v := range client.mu.waitActiveTxns {
   213  		wt = append(wt, hex.EncodeToString(v.txnID))
   214  	}
   215  	return TxnState{
   216  		State:          int(client.mu.state),
   217  		Users:          client.mu.users,
   218  		ActiveTxns:     at,
   219  		WaitActiveTxns: wt,
   220  		LatestTS:       client.timestampWaiter.LatestTS(),
   221  	}
   222  }
   223  
   224  // NewTxnClient create a txn client with TxnSender and Options
   225  func NewTxnClient(
   226  	sender rpc.TxnSender,
   227  	options ...TxnClientCreateOption) TxnClient {
   228  	c := &txnClient{
   229  		clock:  runtime.ProcessLevelRuntime().Clock(),
   230  		sender: sender,
   231  	}
   232  	c.mu.state = paused
   233  	c.mu.cond = sync.NewCond(&c.mu)
   234  	c.mu.activeTxns = make(map[string]*txnOperator, 100000)
   235  	for _, opt := range options {
   236  		opt(c)
   237  	}
   238  	c.adjust()
   239  	c.startLeakChecker()
   240  	return c
   241  }
   242  
   243  func (client *txnClient) adjust() {
   244  	if client.generator == nil {
   245  		client.generator = newUUIDTxnIDGenerator()
   246  	}
   247  	if runtime.ProcessLevelRuntime().Clock() == nil {
   248  		panic("txn clock not set")
   249  	}
   250  	if client.limiter == nil {
   251  		client.limiter = ratelimit.NewUnlimited()
   252  	}
   253  	if client.maxActiveTxn == 0 {
   254  		client.maxActiveTxn = math.MaxInt
   255  	}
   256  }
   257  
   258  func (client *txnClient) New(
   259  	ctx context.Context,
   260  	minTS timestamp.Timestamp,
   261  	options ...TxnOption) (TxnOperator, error) {
   262  	start := time.Now()
   263  	defer func() {
   264  		v2.TxnCreateTotalDurationHistogram.Observe(time.Since(start).Seconds())
   265  	}()
   266  
   267  	// we take a token from the limiter to control the number of transactions created per second.
   268  	client.limiter.Take()
   269  
   270  	txnMeta := txn.TxnMeta{}
   271  	txnMeta.ID = client.generator.Generate()
   272  	txnMeta.Mode = client.getTxnMode()
   273  	txnMeta.Isolation = client.getTxnIsolation()
   274  	if client.lockService != nil {
   275  		txnMeta.LockService = client.lockService.GetServiceID()
   276  	}
   277  
   278  	options = append(options,
   279  		WithTxnCNCoordinator(),
   280  		WithTxnLockService(client.lockService))
   281  	if client.enableCheckDup {
   282  		options = append(options, WithTxnEnableCheckDup())
   283  	}
   284  
   285  	op := newTxnOperator(
   286  		client.clock,
   287  		client.sender,
   288  		txnMeta,
   289  		options...)
   290  	op.timestampWaiter = client.timestampWaiter
   291  	op.AppendEventCallback(ClosedEvent,
   292  		client.updateLastCommitTS,
   293  		client.closeTxn)
   294  
   295  	if err := client.openTxn(op); err != nil {
   296  		return nil, err
   297  	}
   298  
   299  	for _, cb := range client.txnOpenedCallbacks {
   300  		cb(op)
   301  	}
   302  
   303  	ts, err := client.determineTxnSnapshot(minTS)
   304  	if err != nil {
   305  		_ = op.Rollback(ctx)
   306  		return nil, err
   307  	}
   308  	if err := op.UpdateSnapshot(ctx, ts); err != nil {
   309  		_ = op.Rollback(ctx)
   310  		return nil, err
   311  	}
   312  
   313  	util.LogTxnSnapshotTimestamp(
   314  		minTS,
   315  		ts)
   316  
   317  	if err := op.waitActive(ctx); err != nil {
   318  		_ = op.Rollback(ctx)
   319  		return nil, err
   320  	}
   321  	return op, nil
   322  }
   323  
   324  func (client *txnClient) NewWithSnapshot(snapshot []byte) (TxnOperator, error) {
   325  	op, err := newTxnOperatorWithSnapshot(client.sender, snapshot)
   326  	if err != nil {
   327  		return nil, err
   328  	}
   329  	op.timestampWaiter = client.timestampWaiter
   330  	return op, nil
   331  }
   332  
   333  func (client *txnClient) Close() error {
   334  	if client.leakChecker != nil {
   335  		client.leakChecker.close()
   336  	}
   337  	return client.sender.Close()
   338  }
   339  
   340  func (client *txnClient) MinTimestamp() timestamp.Timestamp {
   341  	client.mu.RLock()
   342  	defer client.mu.RUnlock()
   343  
   344  	min := timestamp.Timestamp{}
   345  	for _, op := range client.mu.activeTxns {
   346  		if min.IsEmpty() ||
   347  			op.Txn().SnapshotTS.Less(min) {
   348  			min = op.Txn().SnapshotTS
   349  		}
   350  	}
   351  	return min
   352  }
   353  
   354  func (client *txnClient) WaitLogTailAppliedAt(
   355  	ctx context.Context,
   356  	ts timestamp.Timestamp) (timestamp.Timestamp, error) {
   357  	if client.timestampWaiter == nil {
   358  		return timestamp.Timestamp{}, nil
   359  	}
   360  	return client.timestampWaiter.GetTimestamp(ctx, ts)
   361  }
   362  
   363  func (client *txnClient) getTxnIsolation() txn.TxnIsolation {
   364  	if v, ok := runtime.ProcessLevelRuntime().GetGlobalVariables(runtime.TxnIsolation); ok {
   365  		return v.(txn.TxnIsolation)
   366  	}
   367  	return txn.TxnIsolation_RC
   368  }
   369  
   370  func (client *txnClient) getTxnMode() txn.TxnMode {
   371  	if v, ok := runtime.ProcessLevelRuntime().GetGlobalVariables(runtime.TxnMode); ok {
   372  		return v.(txn.TxnMode)
   373  	}
   374  	return txn.TxnMode_Pessimistic
   375  }
   376  
   377  func (client *txnClient) updateLastCommitTS(event TxnEvent) {
   378  	var old *timestamp.Timestamp
   379  	new := &event.Txn.CommitTS
   380  	for {
   381  		old = client.atomic.latestCommitTS.Load()
   382  		if old != nil && old.GreaterEq(event.Txn.CommitTS) {
   383  			return
   384  		}
   385  
   386  		if client.atomic.latestCommitTS.CompareAndSwap(old, new) {
   387  			return
   388  		}
   389  	}
   390  }
   391  
   392  // determineTxnSnapshot assuming we determine the timestamp to be ts, the final timestamp
   393  // returned will be ts+1. This is because we need to see the submitted data for ts, and the
   394  // timestamp for all things is ts+1.
   395  func (client *txnClient) determineTxnSnapshot(minTS timestamp.Timestamp) (timestamp.Timestamp, error) {
   396  	start := time.Now()
   397  	defer func() {
   398  		v2.TxnDetermineSnapshotDurationHistogram.Observe(time.Since(start).Seconds())
   399  	}()
   400  
   401  	// always use the current ts as txn's snapshot ts is enableSacrificingFreshness
   402  	if !client.enableSacrificingFreshness {
   403  		// TODO: Consider how to handle clock offsets. If use Clock-SI, can use the current
   404  		// time minus the maximum clock offset as the transaction's snapshotTimestamp to avoid
   405  		// conflicts due to clock uncertainty.
   406  		now, _ := client.clock.Now()
   407  		minTS = now
   408  	} else if client.enableCNBasedConsistency {
   409  		minTS = client.adjustTimestamp(minTS)
   410  	}
   411  
   412  	return minTS, nil
   413  }
   414  
   415  func (client *txnClient) adjustTimestamp(ts timestamp.Timestamp) timestamp.Timestamp {
   416  	v := client.atomic.latestCommitTS.Load()
   417  	if v != nil && v.Greater(ts) {
   418  		return *v
   419  	}
   420  	return ts
   421  }
   422  
   423  func (client *txnClient) GetLatestCommitTS() timestamp.Timestamp {
   424  	return client.adjustTimestamp(timestamp.Timestamp{})
   425  }
   426  
   427  func (client *txnClient) SyncLatestCommitTS(ts timestamp.Timestamp) {
   428  	client.updateLastCommitTS(TxnEvent{Txn: txn.TxnMeta{CommitTS: ts}})
   429  	if client.timestampWaiter != nil {
   430  		ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5)
   431  		defer cancel()
   432  		_, err := client.timestampWaiter.GetTimestamp(ctx, ts)
   433  		if err != nil {
   434  			util.GetLogger().Fatal("wait latest commit ts failed", zap.Error(err))
   435  		}
   436  	}
   437  	client.atomic.forceSyncCommitTimes.Add(1)
   438  }
   439  
   440  func (client *txnClient) GetSyncLatestCommitTSTimes() uint64 {
   441  	return client.atomic.forceSyncCommitTimes.Load()
   442  }
   443  
   444  func (client *txnClient) openTxn(op *txnOperator) error {
   445  	client.mu.Lock()
   446  	defer func() {
   447  		v2.TxnActiveQueueSizeGauge.Set(float64(len(client.mu.activeTxns)))
   448  		v2.TxnWaitActiveQueueSizeGauge.Set(float64(len(client.mu.waitActiveTxns)))
   449  		client.mu.Unlock()
   450  	}()
   451  
   452  	for client.mu.state == paused {
   453  		if client.normalStateNoWait {
   454  			return moerr.NewInternalErrorNoCtx("cn service is not ready, retry later")
   455  		}
   456  
   457  		util.GetLogger().Warn("txn client is in pause state, wait for it to be ready",
   458  			zap.String("txn ID", hex.EncodeToString(op.txnID)))
   459  		// Wait until the txn client's state changed to normal, and it will probably take
   460  		// no more than 5 seconds in theory.
   461  		client.mu.cond.Wait()
   462  		util.GetLogger().Warn("txn client is in ready state",
   463  			zap.String("txn ID", hex.EncodeToString(op.txnID)))
   464  	}
   465  
   466  	if !op.options.UserTxn() ||
   467  		client.mu.users < client.maxActiveTxn {
   468  		client.addActiveTxnLocked(op)
   469  		return nil
   470  	}
   471  	var cancelC chan struct{}
   472  	if client.timestampWaiter != nil {
   473  		cancelC = client.timestampWaiter.CancelC()
   474  		if cancelC == nil {
   475  			return moerr.NewWaiterPausedNoCtx()
   476  		}
   477  	}
   478  	op.waiter = newWaiter(timestamp.Timestamp{}, cancelC)
   479  	op.waiter.ref()
   480  	client.mu.waitActiveTxns = append(client.mu.waitActiveTxns, op)
   481  	return nil
   482  }
   483  
   484  func (client *txnClient) closeTxn(event TxnEvent) {
   485  	txn := event.Txn
   486  
   487  	client.mu.Lock()
   488  	defer func() {
   489  		v2.TxnActiveQueueSizeGauge.Set(float64(len(client.mu.activeTxns)))
   490  		v2.TxnWaitActiveQueueSizeGauge.Set(float64(len(client.mu.waitActiveTxns)))
   491  		client.mu.Unlock()
   492  	}()
   493  
   494  	key := string(txn.ID)
   495  	op, ok := client.mu.activeTxns[key]
   496  	if ok {
   497  		v2.TxnLifeCycleDurationHistogram.Observe(time.Since(op.createAt).Seconds())
   498  
   499  		delete(client.mu.activeTxns, key)
   500  		client.removeFromLeakCheck(txn.ID)
   501  		if !op.options.UserTxn() {
   502  			return
   503  		}
   504  		client.mu.users--
   505  		if client.mu.users < 0 {
   506  			panic("BUG: user txns < 0")
   507  		}
   508  		if len(client.mu.waitActiveTxns) > 0 {
   509  			newCanAdded := client.maxActiveTxn - client.mu.users
   510  			for i := 0; i < newCanAdded; i++ {
   511  				op := client.fetchWaitActiveOpLocked()
   512  				if op == nil {
   513  					return
   514  				}
   515  				client.addActiveTxnLocked(op)
   516  				op.notifyActive()
   517  			}
   518  		}
   519  	} else {
   520  		util.GetLogger().Warn("txn closed",
   521  			zap.String("txn ID", hex.EncodeToString(txn.ID)),
   522  			zap.String("stack", string(debug.Stack())))
   523  	}
   524  }
   525  
   526  func (client *txnClient) addActiveTxnLocked(op *txnOperator) {
   527  	if op.options.UserTxn() {
   528  		client.mu.users++
   529  	}
   530  	client.mu.activeTxns[string(op.txnID)] = op
   531  	client.addToLeakCheck(op)
   532  }
   533  
   534  func (client *txnClient) fetchWaitActiveOpLocked() *txnOperator {
   535  	if len(client.mu.waitActiveTxns) == 0 {
   536  		return nil
   537  	}
   538  	op := client.mu.waitActiveTxns[0]
   539  	client.mu.waitActiveTxns = append(client.mu.waitActiveTxns[:0], client.mu.waitActiveTxns[1:]...)
   540  	return op
   541  }
   542  
   543  func (client *txnClient) Pause() {
   544  	client.mu.Lock()
   545  	defer client.mu.Unlock()
   546  
   547  	util.GetLogger().Info("txn client status changed to paused")
   548  	client.mu.state = paused
   549  }
   550  
   551  func (client *txnClient) Resume() {
   552  	client.mu.Lock()
   553  	defer client.mu.Unlock()
   554  
   555  	util.GetLogger().Info("txn client status changed to normal")
   556  	client.mu.state = normal
   557  
   558  	// Notify all waiting transactions to goon with the opening operation.
   559  	if !client.normalStateNoWait {
   560  		client.mu.cond.Broadcast()
   561  	}
   562  }
   563  
   564  func (client *txnClient) AbortAllRunningTxn() {
   565  	client.mu.Lock()
   566  	ops := make([]*txnOperator, 0, len(client.mu.activeTxns))
   567  	for _, op := range client.mu.activeTxns {
   568  		ops = append(ops, op)
   569  	}
   570  	waitOps := append(([]*txnOperator)(nil), client.mu.waitActiveTxns...)
   571  	client.mu.waitActiveTxns = client.mu.waitActiveTxns[:0]
   572  	client.mu.Unlock()
   573  
   574  	if client.timestampWaiter != nil {
   575  		// Cancel all waiters, means that all waiters do not need to wait for
   576  		// the newer timestamp from logtail consumer.
   577  		client.timestampWaiter.Pause()
   578  	}
   579  
   580  	for _, op := range ops {
   581  		op.cannotCleanWorkspace = true
   582  		_ = op.Rollback(context.Background())
   583  		op.cannotCleanWorkspace = false
   584  	}
   585  	for _, op := range waitOps {
   586  		op.cannotCleanWorkspace = true
   587  		_ = op.Rollback(context.Background())
   588  		op.cannotCleanWorkspace = false
   589  		op.notifyActive()
   590  	}
   591  
   592  	if client.timestampWaiter != nil {
   593  		// After rollback all transactions, resume the timestamp waiter channel.
   594  		client.timestampWaiter.Resume()
   595  	}
   596  }
   597  
   598  func (client *txnClient) startLeakChecker() {
   599  	if client.leakChecker != nil {
   600  		client.leakChecker.start()
   601  	}
   602  }
   603  
   604  func (client *txnClient) addToLeakCheck(op *txnOperator) {
   605  	if client.leakChecker != nil {
   606  		client.leakChecker.txnOpened(op, op.txnID, op.options)
   607  	}
   608  }
   609  
   610  func (client *txnClient) removeFromLeakCheck(id []byte) {
   611  	if client.leakChecker != nil {
   612  		client.leakChecker.txnClosed(id)
   613  	}
   614  }
   615  
   616  func (client *txnClient) IterTxns(fn func(TxnOverview) bool) {
   617  	ops := client.getAllTxnOperators()
   618  
   619  	for _, op := range ops {
   620  		if !fn(op.GetOverview()) {
   621  			return
   622  		}
   623  	}
   624  }
   625  
   626  func (client *txnClient) getAllTxnOperators() []*txnOperator {
   627  	client.mu.RLock()
   628  	defer client.mu.RUnlock()
   629  
   630  	ops := make([]*txnOperator, 0, len(client.mu.activeTxns)+len(client.mu.waitActiveTxns))
   631  	for _, op := range client.mu.activeTxns {
   632  		ops = append(ops, op)
   633  	}
   634  	ops = append(ops, client.mu.waitActiveTxns...)
   635  	return ops
   636  }