From 3d3bf86ea69bad142eff33acb9c076debc4f710f Mon Sep 17 00:00:00 2001 From: fagongzi Date: Sat, 2 Nov 2024 21:52:10 +0800 Subject: [PATCH] abort all active txn on invalid service and make new txn can work to 2.0 (#19749) abort all active txn on invalid service and make new txn can work Approved by: @XuPeng-SH, @sukki37 --- pkg/common/moerr/error.go | 21 +- pkg/common/moerr/error_no_ctx.go | 4 + pkg/lockservice/lock_table_allocator.go | 65 ++- pkg/lockservice/lock_table_keeper_test.go | 8 +- pkg/lockservice/lock_table_local.go | 11 +- pkg/lockservice/lock_table_remote.go | 16 +- pkg/lockservice/lock_table_remote_test.go | 18 +- pkg/lockservice/log.go | 8 +- pkg/lockservice/rpc.go | 2 - pkg/lockservice/rpc_test.go | 20 +- pkg/lockservice/service.go | 28 +- pkg/lockservice/service_observability.go | 1 - pkg/lockservice/service_remote.go | 44 +- pkg/lockservice/service_remote_test.go | 2 +- pkg/lockservice/service_test.go | 20 +- pkg/lockservice/txn.go | 8 +- pkg/lockservice/txn_test.go | 1 - pkg/lockservice/types.go | 11 + pkg/pb/lock/lock.go | 2 +- pkg/pb/lock/lock.pb.go | 676 ++++++++++++++++++---- pkg/pb/txn/txn.pb.go | 227 +++++--- pkg/tests/issues/issue_test.go | 144 +++++ pkg/tnservice/store.go | 4 + pkg/tnservice/types.go | 3 + pkg/txn/client/client.go | 60 +- pkg/txn/client/client_test.go | 8 + pkg/txn/client/operator.go | 62 +- pkg/txn/client/testutil.go | 1 + pkg/txn/client/types.go | 6 + proto/lock.proto | 14 + proto/txn.proto | 1 + 31 files changed, 1145 insertions(+), 351 deletions(-) diff --git a/pkg/common/moerr/error.go b/pkg/common/moerr/error.go index 6657ae11f06e..d4680dd03678 100644 --- a/pkg/common/moerr/error.go +++ b/pkg/common/moerr/error.go @@ -240,6 +240,8 @@ const ( ErrLockConflict uint16 = 20706 // ErrLockNeedUpgrade row level lock is too large that need upgrade to table level lock ErrLockNeedUpgrade uint16 = 20707 + // ErrCannotCommitOnInvalidCN cannot commit transaction on invalid CN + ErrCannotCommitOnInvalidCN uint16 = 20708 // Group 8: partition ErrPartitionFunctionIsNotAllowed uint16 = 20801 @@ -466,13 +468,14 @@ var errorMsgRefer = map[uint16]moErrorMsgItem{ ErrCantDelGCChecker: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "can't delete gc checker"}, // Group 7: lock service - ErrDeadLockDetected: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "deadlock detected"}, - ErrLockTableBindChanged: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "lock table bind changed"}, - ErrLockTableNotFound: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "lock table not found on remote lock service"}, - ErrDeadlockCheckBusy: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "deadlock check is busy"}, - ErrCannotCommitOrphan: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "cannot commit a orphan transaction"}, - ErrLockConflict: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "lock options conflict, wait policy is fast fail"}, - ErrLockNeedUpgrade: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "row level lock is too large that need upgrade to table level lock"}, + ErrDeadLockDetected: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "deadlock detected"}, + ErrLockTableBindChanged: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "lock table bind changed"}, + ErrLockTableNotFound: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "lock table not found on remote lock service"}, + ErrDeadlockCheckBusy: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "deadlock check is busy"}, + ErrCannotCommitOrphan: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "cannot commit a orphan transaction"}, + ErrLockConflict: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "lock options conflict, wait policy is fast fail"}, + ErrLockNeedUpgrade: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "row level lock is too large that need upgrade to table level lock"}, + ErrCannotCommitOnInvalidCN: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "cannot commit a orphan transaction on invalid cn"}, // Group 8: partition ErrPartitionFunctionIsNotAllowed: {ER_PARTITION_FUNCTION_IS_NOT_ALLOWED, []string{MySQLDefaultSqlState}, "This partition function is not allowed"}, @@ -1266,6 +1269,10 @@ func NewCannotCommitOrphan(ctx context.Context) *Error { return newError(ctx, ErrCannotCommitOrphan) } +func NewCannotCommitOnInvalidCN(ctx context.Context) *Error { + return newError(ctx, ErrCannotCommitOnInvalidCN) +} + func NewLockTableBindChanged(ctx context.Context) *Error { return newError(ctx, ErrLockTableBindChanged) } diff --git a/pkg/common/moerr/error_no_ctx.go b/pkg/common/moerr/error_no_ctx.go index 24f8cf441682..06bf84f688c2 100644 --- a/pkg/common/moerr/error_no_ctx.go +++ b/pkg/common/moerr/error_no_ctx.go @@ -382,6 +382,10 @@ func NewCannotCommitOrphanNoCtx() *Error { return NewCannotCommitOrphan(Context()) } +func NewCannotCommitOnInvalidCNNoCtx() *Error { + return NewCannotCommitOnInvalidCN(Context()) +} + func NewLockTableBindChangedNoCtx() *Error { return newError(Context(), ErrLockTableBindChanged) } diff --git a/pkg/lockservice/lock_table_allocator.go b/pkg/lockservice/lock_table_allocator.go index 2feee59db31b..7f3ff5603ff5 100644 --- a/pkg/lockservice/lock_table_allocator.go +++ b/pkg/lockservice/lock_table_allocator.go @@ -20,14 +20,13 @@ import ( "sync" "time" - "go.uber.org/zap" - "github.com/matrixorigin/matrixone/pkg/common/log" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/morpc" "github.com/matrixorigin/matrixone/pkg/common/stopper" "github.com/matrixorigin/matrixone/pkg/common/util" pb "github.com/matrixorigin/matrixone/pkg/pb/lock" + "go.uber.org/zap" ) type lockTableAllocator struct { @@ -113,7 +112,7 @@ func (l *lockTableAllocator) Get( sharding pb.Sharding) pb.LockTable { binds := l.getServiceBinds(serviceID) if binds == nil { - binds = l.registerService(serviceID, tableID) + binds = l.registerService(serviceID) } return l.registerBind(binds, group, tableID, originTableID, sharding) } @@ -140,6 +139,15 @@ func (l *lockTableAllocator) AddCannotCommit(values []pb.OrphanTxn) [][]byte { return committing } +func (l *lockTableAllocator) AddInvalidService(serviceID string) { + l.inactiveService.Store(serviceID, time.Now()) +} + +func (l *lockTableAllocator) HasInvalidService(serviceID string) bool { + _, ok := l.inactiveService.Load(serviceID) + return ok +} + func (l *lockTableAllocator) Valid( serviceID string, txnID []byte, @@ -170,8 +178,9 @@ func (l *lockTableAllocator) Valid( if _, ok := l.inactiveService.Load(serviceID); ok { l.logger.Info("inactive service", - zap.String("serviceID", serviceID)) - return nil, moerr.NewCannotCommitOrphanNoCtx() + zap.String("serviceID", serviceID), + ) + return nil, moerr.NewCannotCommitOnInvalidCNNoCtx() } c := l.getCtl(serviceID) @@ -374,7 +383,6 @@ func (l *lockTableAllocator) getTimeoutBinds(now time.Time) []*serviceBinds { func (l *lockTableAllocator) registerService( serviceID string, - tableID uint64, ) *serviceBinds { l.mu.Lock() defer l.mu.Unlock() @@ -763,7 +771,8 @@ func (l *lockTableAllocator) initServer(cfg morpc.Config) { func (l *lockTableAllocator) initHandler() { l.server.RegisterMethodHandler( pb.Method_GetBind, - l.handleGetBind) + l.handleGetBind, + ) l.server.RegisterMethodHandler( pb.Method_KeepLockTableBind, @@ -787,11 +796,18 @@ func (l *lockTableAllocator) initHandler() { l.server.RegisterMethodHandler( pb.Method_CannotCommit, - l.handleCannotCommit) + l.handleCannotCommit, + ) l.server.RegisterMethodHandler( pb.Method_CheckOrphan, - l.handleCheckOrphan) + l.handleCheckOrphan, + ) + + l.server.RegisterMethodHandler( + pb.Method_ResumeInvalidCN, + l.handleResumeInvalidCN, + ) } func (l *lockTableAllocator) handleGetBind( @@ -801,7 +817,7 @@ func (l *lockTableAllocator) handleGetBind( resp *pb.Response, cs morpc.ClientSession) { if !l.canGetBind(req.GetBind.ServiceID) { - writeResponse(ctx, l.logger, cancel, resp, moerr.NewNewTxnInCNRollingRestart(), cs) + writeResponse(l.logger, cancel, resp, moerr.NewNewTxnInCNRollingRestart(), cs) return } resp.GetBind.LockTable = l.Get( @@ -810,7 +826,7 @@ func (l *lockTableAllocator) handleGetBind( req.GetBind.Table, req.GetBind.OriginTable, req.GetBind.Sharding) - writeResponse(ctx, l.logger, cancel, resp, nil, cs) + writeResponse(l.logger, cancel, resp, nil, cs) } func (l *lockTableAllocator) handleKeepLockTableBind( @@ -822,7 +838,7 @@ func (l *lockTableAllocator) handleKeepLockTableBind( resp.KeepLockTableBind.OK = l.KeepLockTableBind(req.KeepLockTableBind.ServiceID) if !resp.KeepLockTableBind.OK { // resp.KeepLockTableBind.Status = pb.Status_ServiceCanRestart - writeResponse(ctx, l.logger, cancel, resp, nil, cs) + writeResponse(l.logger, cancel, resp, nil, cs) return } b := l.getServiceBinds(req.KeepLockTableBind.ServiceID) @@ -832,7 +848,7 @@ func (l *lockTableAllocator) handleKeepLockTableBind( zap.String("serviceID", b.serviceID), zap.String("status", req.KeepLockTableBind.Status.String())) } else { - writeResponse(ctx, l.logger, cancel, resp, nil, cs) + writeResponse(l.logger, cancel, resp, nil, cs) return } } @@ -852,7 +868,7 @@ func (l *lockTableAllocator) handleKeepLockTableBind( resp.KeepLockTableBind.Status = req.KeepLockTableBind.Status } l.disableGroupTables(req.KeepLockTableBind.LockTables, b) - writeResponse(ctx, l.logger, cancel, resp, nil, cs) + writeResponse(l.logger, cancel, resp, nil, cs) } func (l *lockTableAllocator) handleSetRestartService( @@ -863,7 +879,7 @@ func (l *lockTableAllocator) handleSetRestartService( cs morpc.ClientSession) { l.setRestartService(req.SetRestartService.ServiceID) resp.SetRestartService.OK = true - writeResponse(ctx, l.logger, cancel, resp, nil, cs) + writeResponse(l.logger, cancel, resp, nil, cs) } func (l *lockTableAllocator) handleCanRestartService( @@ -873,7 +889,7 @@ func (l *lockTableAllocator) handleCanRestartService( resp *pb.Response, cs morpc.ClientSession) { resp.CanRestartService.OK = l.canRestartService(req.CanRestartService.ServiceID) - writeResponse(ctx, l.logger, cancel, resp, nil, cs) + writeResponse(l.logger, cancel, resp, nil, cs) } func (l *lockTableAllocator) handleRemainTxnInService( @@ -883,7 +899,7 @@ func (l *lockTableAllocator) handleRemainTxnInService( resp *pb.Response, cs morpc.ClientSession) { resp.RemainTxnInService.RemainTxn = l.remainTxnInService(req.RemainTxnInService.ServiceID) - writeResponse(ctx, l.logger, cancel, resp, nil, cs) + writeResponse(l.logger, cancel, resp, nil, cs) } func (l *lockTableAllocator) getLockTablesLocked(group uint32) map[uint64]pb.LockTable { @@ -904,7 +920,7 @@ func (l *lockTableAllocator) handleCannotCommit( cs morpc.ClientSession) { committingTxn := l.AddCannotCommit(req.CannotCommit.OrphanTxnList) resp.CannotCommit.CommittingTxn = committingTxn - writeResponse(ctx, l.logger, cancel, resp, nil, cs) + writeResponse(l.logger, cancel, resp, nil, cs) } func (l *lockTableAllocator) handleCheckOrphan( @@ -916,7 +932,18 @@ func (l *lockTableAllocator) handleCheckOrphan( c := l.getCtl(req.CheckOrphan.ServiceID) state, ok := c.getCtlState(util.UnsafeBytesToString(req.CheckOrphan.Txn)) resp.CheckOrphan.Orphan = ok && state == cannotCommitState - writeResponse(ctx, l.logger, cancel, resp, nil, cs) + writeResponse(l.logger, cancel, resp, nil, cs) +} + +func (l *lockTableAllocator) handleResumeInvalidCN( + ctx context.Context, + cancel context.CancelFunc, + req *pb.Request, + resp *pb.Response, + cs morpc.ClientSession, +) { + l.inactiveService.Delete(req.ResumeInvalidCN.ServiceID) + writeResponse(l.logger, cancel, resp, nil, cs) } func (l *lockTableAllocator) getCtl(serviceID string) *commitCtl { diff --git a/pkg/lockservice/lock_table_keeper_test.go b/pkg/lockservice/lock_table_keeper_test.go index 4208726bea20..243fa6a81bc3 100644 --- a/pkg/lockservice/lock_table_keeper_test.go +++ b/pkg/lockservice/lock_table_keeper_test.go @@ -46,7 +46,7 @@ func TestKeeper(t *testing.T) { if n1.Add(1) == 10 { close(c1) } - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) s.RegisterMethodHandler( pb.Method_KeepRemoteLock, @@ -60,7 +60,7 @@ func TestKeeper(t *testing.T) { if n2.Add(1) == 10 { close(c2) } - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) m := &lockTableHolders{service: "s1", holders: map[uint32]*lockTableHolder{}} m.set( @@ -119,7 +119,7 @@ func TestKeepBindFailedWillRemoveAllLocalLockTable(t *testing.T) { resp *pb.Response, cs morpc.ClientSession) { resp.KeepLockTableBind.OK = false - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) s.RegisterMethodHandler( @@ -130,7 +130,7 @@ func TestKeepBindFailedWillRemoveAllLocalLockTable(t *testing.T) { req *pb.Request, resp *pb.Response, cs morpc.ClientSession) { - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) m := &lockTableHolders{service: "s1", holders: map[uint32]*lockTableHolder{}} diff --git a/pkg/lockservice/lock_table_local.go b/pkg/lockservice/lock_table_local.go index 363c6a7357f0..2e7907b72491 100644 --- a/pkg/lockservice/lock_table_local.go +++ b/pkg/lockservice/lock_table_local.go @@ -224,7 +224,6 @@ func (l *localLockTable) unlock( logUnlockTableOnLocal( l.logger, - l.bind.ServiceID, txn, l.bind, ) @@ -586,7 +585,7 @@ func (l *localLockTable) addRangeLockLocked( prevStartKey, key, keyLock, mc, - c.txn) + ) prevStartKey = nil rangeStartEncountered = false return bytes.Compare(key, end) < 0 @@ -632,7 +631,7 @@ func (l *localLockTable) addRangeLockLocked( prevStartKey, key, keyLock, mc, - c.txn) + ) } break } @@ -662,7 +661,7 @@ func (l *localLockTable) mergeRangeLocked( seekKey []byte, seekLock Lock, mc *mergeContext, - txn *activeTxn) ([]byte, []byte) { +) ([]byte, []byte) { // range lock encountered a row lock if seekLock.isLockRow() { // 5 + [1, 4] => [1, 4] + [5] @@ -766,10 +765,10 @@ func (c *mergeContext) commit( s.Delete(util.UnsafeStringToBytes(k)) } txn.lockRemoved( - bind.ServiceID, bind.Group, bind.Table, - c.mergedLocks) + c.mergedLocks, + ) for _, q := range c.mergedWaiters { // release ref in merged waiters. The ref is moved to c.to. diff --git a/pkg/lockservice/lock_table_remote.go b/pkg/lockservice/lock_table_remote.go index 22e9750877fe..0b413f6b68d9 100644 --- a/pkg/lockservice/lock_table_remote.go +++ b/pkg/lockservice/lock_table_remote.go @@ -19,14 +19,13 @@ import ( "context" "time" - "go.uber.org/zap" - "github.com/matrixorigin/matrixone/pkg/common/log" "github.com/matrixorigin/matrixone/pkg/common/moerr" pb "github.com/matrixorigin/matrixone/pkg/pb/lock" "github.com/matrixorigin/matrixone/pkg/pb/timestamp" v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" "github.com/matrixorigin/matrixone/pkg/util/trace" + "go.uber.org/zap" ) // remoteLockTable the lock corresponding to the Table is managed by a remote LockTable. @@ -117,7 +116,7 @@ func (l *remoteLockTable) lock( // encounter any error, we need try to check bind is valid. // And use origin error to return, because once handlerError // swallows the error, the transaction will not be abort. - _ = l.handleError(txn.txnID, err, true) + _ = l.handleError(err, true) cb(pb.Result{}, err) } @@ -128,7 +127,6 @@ func (l *remoteLockTable) unlock( mutations ...pb.ExtraMutation) { logUnlockTableOnRemote( l.logger, - l.serviceID, txn, l.bind, ) @@ -140,7 +138,6 @@ func (l *remoteLockTable) unlock( logUnlockTableOnRemoteFailed( l.logger, - l.serviceID, txn, l.bind, err, @@ -151,7 +148,7 @@ func (l *remoteLockTable) unlock( // handleError returns nil meaning bind changed, then all locks // will be released. If handleError returns any error, it means // that the current bind is valid, retry unlock. - if err := l.handleError(txn.txnID, err, false); err == nil { + if err := l.handleError(err, false); err == nil { return } } @@ -172,7 +169,7 @@ func (l *remoteLockTable) getLock( } // why use loop is similar to unlock - if err = l.handleError(txn.TxnID, err, false); err == nil { + if err = l.handleError(err, false); err == nil { return } } @@ -247,7 +244,10 @@ func (l *remoteLockTable) close() { logLockTableClosed(l.logger, l.bind, true) } -func (l *remoteLockTable) handleError(txnID []byte, err error, mustHandleLockBindChangedErr bool) error { +func (l *remoteLockTable) handleError( + err error, + mustHandleLockBindChangedErr bool, +) error { oldError := err // ErrLockTableBindChanged error must already handled. Skip if !mustHandleLockBindChangedErr && moerr.IsMoErrCode(err, moerr.ErrLockTableBindChanged) { diff --git a/pkg/lockservice/lock_table_remote_test.go b/pkg/lockservice/lock_table_remote_test.go index f5cf8aa904e7..4c2fb8c6f44d 100644 --- a/pkg/lockservice/lock_table_remote_test.go +++ b/pkg/lockservice/lock_table_remote_test.go @@ -41,7 +41,7 @@ func TestLockRemote(t *testing.T) { req *pb.Request, resp *pb.Response, cs morpc.ClientSession) { - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }, ) }, @@ -74,7 +74,7 @@ func TestLockRemoteWithNeedUpgrade(t *testing.T) { req *pb.Request, resp *pb.Response, cs morpc.ClientSession) { - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }, ) }, @@ -109,7 +109,7 @@ func TestUnlockRemote(t *testing.T) { req *pb.Request, resp *pb.Response, cs morpc.ClientSession) { - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }, ) }, @@ -140,11 +140,11 @@ func TestUnlockRemoteWithRetry(t *testing.T) { cs morpc.ClientSession) { n++ if n == 1 { - writeResponse(ctx, getLogger(""), cancel, resp, moerr.NewRPCTimeout(ctx), cs) + writeResponse(getLogger(""), cancel, resp, moerr.NewRPCTimeout(ctx), cs) return } close(c) - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }, ) s.RegisterMethodHandler( @@ -159,7 +159,7 @@ func TestUnlockRemoteWithRetry(t *testing.T) { ServiceID: "s1", Valid: true, } - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }, ) }, @@ -196,7 +196,7 @@ func TestRemoteWithBindChanged(t *testing.T) { resp *pb.Response, cs morpc.ClientSession) { resp.NewBind = &newBind - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }, ) @@ -209,7 +209,7 @@ func TestRemoteWithBindChanged(t *testing.T) { resp *pb.Response, cs morpc.ClientSession) { resp.NewBind = &newBind - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }, ) @@ -222,7 +222,7 @@ func TestRemoteWithBindChanged(t *testing.T) { resp *pb.Response, cs morpc.ClientSession) { resp.NewBind = &newBind - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }, ) }, diff --git a/pkg/lockservice/log.go b/pkg/lockservice/log.go index 53853aa0d284..96533c0dcab4 100644 --- a/pkg/lockservice/log.go +++ b/pkg/lockservice/log.go @@ -18,6 +18,7 @@ import ( "bytes" "encoding/hex" "fmt" + "github.com/matrixorigin/matrixone/pkg/common/util" "github.com/matrixorigin/matrixone/pkg/common/log" @@ -558,7 +559,6 @@ func logLocalBindsInvalid( func logUnlockTxn( logger *log.MOLogger, - serviceID string, txn *activeTxn, ) func() { if logger == nil { @@ -577,7 +577,6 @@ func logUnlockTxn( func logTxnReadyToClose( logger *log.MOLogger, - serviceID string, txn *activeTxn, ) { if logger == nil { @@ -595,7 +594,6 @@ func logTxnReadyToClose( func logTxnUnlockTable( logger *log.MOLogger, - serviceID string, txn *activeTxn, table uint64, ) { @@ -615,7 +613,6 @@ func logTxnUnlockTable( func logTxnUnlockTableCompleted( logger *log.MOLogger, - serviceID string, txn *activeTxn, table uint64, cs *cowSlice, @@ -639,7 +636,6 @@ func logTxnUnlockTableCompleted( func logUnlockTableOnLocal( logger *log.MOLogger, - serviceID string, txn *activeTxn, bind pb.LockTable, ) { @@ -659,7 +655,6 @@ func logUnlockTableOnLocal( func logUnlockTableOnRemote( logger *log.MOLogger, - serviceID string, txn *activeTxn, bind pb.LockTable, ) { @@ -679,7 +674,6 @@ func logUnlockTableOnRemote( func logUnlockTableOnRemoteFailed( logger *log.MOLogger, - serviceID string, txn *activeTxn, bind pb.LockTable, err error, diff --git a/pkg/lockservice/rpc.go b/pkg/lockservice/rpc.go index 8404ebc12624..f8215fe4b330 100644 --- a/pkg/lockservice/rpc.go +++ b/pkg/lockservice/rpc.go @@ -345,7 +345,6 @@ func (s *server) onMessage( cs.RemoteAddress(), s.address) writeResponse( - ctx, s.logger, msg.Cancel, getResponse(req), @@ -410,7 +409,6 @@ func getResponse(req *pb.Request) *pb.Response { } func writeResponse( - ctx context.Context, logger *log.MOLogger, cancel context.CancelFunc, resp *pb.Response, diff --git a/pkg/lockservice/rpc_test.go b/pkg/lockservice/rpc_test.go index 286d98c60644..af7b9d517fc2 100644 --- a/pkg/lockservice/rpc_test.go +++ b/pkg/lockservice/rpc_test.go @@ -45,7 +45,7 @@ func TestRPCSend(t *testing.T) { req *lock.Request, resp *lock.Response, cs morpc.ClientSession) { - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100) @@ -74,7 +74,7 @@ func TestSetRestartServiceRPCSend(t *testing.T) { resp *lock.Response, cs morpc.ClientSession) { resp.SetRestartService.OK = true - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100) @@ -104,7 +104,7 @@ func TestCanRestartServiceRPCSend(t *testing.T) { resp *lock.Response, cs morpc.ClientSession) { resp.CanRestartService.OK = true - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100) @@ -134,7 +134,7 @@ func TestRemainTxnServiceRPCSend(t *testing.T) { resp *lock.Response, cs morpc.ClientSession) { resp.RemainTxnInService.RemainTxn = -1 - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100) @@ -163,7 +163,7 @@ func TestRPCSendErrBackendCannotConnect(t *testing.T) { req *lock.Request, resp *lock.Response, cs morpc.ClientSession) { - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100) @@ -207,7 +207,7 @@ func TestMOErrorCanHandled(t *testing.T) { req *lock.Request, resp *lock.Response, cs morpc.ClientSession) { - writeResponse(ctx, getLogger(""), cancel, resp, moerr.NewDeadLockDetectedNoCtx(), cs) + writeResponse(getLogger(""), cancel, resp, moerr.NewDeadLockDetectedNoCtx(), cs) }) ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) @@ -234,7 +234,7 @@ func TestRequestCanBeFilter(t *testing.T) { req *lock.Request, resp *lock.Response, cs morpc.ClientSession) { - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100) @@ -262,7 +262,7 @@ func TestRetryValidateService(t *testing.T) { req *lock.Request, resp *lock.Response, cs morpc.ClientSession) { - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) _, err := validateService(time.Millisecond*100, "s1", c, getLogger("")) @@ -285,7 +285,7 @@ func TestValidateService(t *testing.T) { resp *lock.Response, cs morpc.ClientSession) { resp.ValidateService.OK = true - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) valid, err := validateService(time.Millisecond*100, "UNKNOWN", c, getLogger("")) @@ -312,7 +312,7 @@ func TestLockTableBindChanged(t *testing.T) { resp *lock.Response, cs morpc.ClientSession) { resp.NewBind = &lock.LockTable{ServiceID: "s1"} - writeResponse(ctx, getLogger(""), cancel, resp, nil, cs) + writeResponse(getLogger(""), cancel, resp, nil, cs) }) ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) diff --git a/pkg/lockservice/service.go b/pkg/lockservice/service.go index 65b2bfa8ebe5..1991ada66e1f 100644 --- a/pkg/lockservice/service.go +++ b/pkg/lockservice/service.go @@ -225,8 +225,8 @@ func (s *service) Unlock( } } - defer logUnlockTxn(s.logger, s.serviceID, txn)() - txn.close(s.serviceID, txnID, commitTS, s.getLockTable, s.logger, mutations...) + defer logUnlockTxn(s.logger, txn)() + txn.close(txnID, commitTS, s.getLockTable, s.logger, mutations...) // The deadlock detector will hold the deadlocked transaction that is aborted // to avoid the situation where the deadlock detection is interfered with by // the abort transaction. When a transaction is unlocked, the deadlock detector @@ -253,6 +253,27 @@ func (s *service) IsOrphanTxn( return resp.CheckOrphan.Orphan, nil } +func (s *service) Resume() error { + ctx, cancel := context.WithTimeoutCause( + context.Background(), + defaultRPCTimeout, + moerr.NewInfoNoCtx("lockservice.resume"), + ) + defer cancel() + + req := acquireRequest() + req.Method = pb.Method_ResumeInvalidCN + req.ResumeInvalidCN.ServiceID = s.serviceID + + resp, err := s.remote.client.Send(ctx, req) + if err != nil { + return err + } + defer releaseResponse(resp) + + return err +} + func (s *service) reduceCanMoveGroupTables(txn *activeTxn) { s.mu.Lock() defer s.mu.Unlock() @@ -442,7 +463,6 @@ func (s *service) fetchTxnWaitingList(txn pb.WaitTxn, waiters *waiters) (bool, e return activeTxn.fetchWhoWaitingMe( s.serviceID, txnID, - s.activeTxnHolder, waiters.add, s.getLockTable), nil } @@ -468,7 +488,7 @@ func (s *service) abortDeadlockTxn(wait pb.WaitTxn, err error) { if activeTxn == nil { return } - activeTxn.abort(s.serviceID, wait, err, s.logger) + activeTxn.abort(wait, err, s.logger) } func (s *service) getLockTable( diff --git a/pkg/lockservice/service_observability.go b/pkg/lockservice/service_observability.go index f6f0480ee66f..a966845098bf 100644 --- a/pkg/lockservice/service_observability.go +++ b/pkg/lockservice/service_observability.go @@ -33,7 +33,6 @@ func (s *service) GetWaitingList( txn.fetchWhoWaitingMe( s.serviceID, txnID, - s.activeTxnHolder, func(w pb.WaitTxn) bool { values = append(values, w) return true diff --git a/pkg/lockservice/service_remote.go b/pkg/lockservice/service_remote.go index 6fa04d997c49..9335b0a03557 100644 --- a/pkg/lockservice/service_remote.go +++ b/pkg/lockservice/service_remote.go @@ -46,6 +46,7 @@ var methodVersions = map[pb.Method]int64{ pb.Method_CannotCommit: defines.MORPCVersion2, pb.Method_GetActiveTxn: defines.MORPCVersion2, pb.Method_CheckOrphan: defines.MORPCVersion2, + pb.Method_ResumeInvalidCN: defines.MORPCVersion2, } func (s *service) initRemote() { @@ -180,7 +181,7 @@ func (s *service) handleRemoteLock( resp *pb.Response, cs morpc.ClientSession) { if !s.canLockOnServiceStatus(req.Lock.TxnID, req.Lock.Options, req.LockTable.Table, req.Lock.Rows) { - writeResponse(ctx, s.logger, cancel, resp, moerr.NewRetryForCNRollingRestart(), cs) + writeResponse(s.logger, cancel, resp, moerr.NewRetryForCNRollingRestart(), cs) return } @@ -189,7 +190,7 @@ func (s *service) handleRemoteLock( l == nil { // means that the lockservice sending the lock request holds a stale // lock table binding. - writeResponse(ctx, s.logger, cancel, resp, err, cs) + writeResponse(s.logger, cancel, resp, err, cs) return } @@ -197,11 +198,11 @@ func (s *service) handleRemoteLock( txn.Lock() defer txn.Unlock() if !bytes.Equal(txn.txnID, req.Lock.TxnID) { - writeResponse(ctx, s.logger, cancel, resp, ErrTxnNotFound, cs) + writeResponse(s.logger, cancel, resp, ErrTxnNotFound, cs) return } if txn.deadlockFound { - writeResponse(ctx, s.logger, cancel, resp, ErrDeadLockDetected, cs) + writeResponse(s.logger, cancel, resp, ErrDeadLockDetected, cs) return } @@ -226,7 +227,7 @@ func (s *service) handleRemoteLock( func(result pb.Result, err error) { e = err resp.Lock.Result = result - writeResponse(ctx, s.logger, cancel, resp, err, cs) + writeResponse(s.logger, cancel, resp, err, cs) }) } @@ -237,7 +238,7 @@ func (s *service) handleForwardLock( resp *pb.Response, cs morpc.ClientSession) { if !s.canLockOnServiceStatus(req.Lock.TxnID, req.Lock.Options, req.LockTable.Table, req.Lock.Rows) { - writeResponse(ctx, s.logger, cancel, resp, moerr.NewRetryForCNRollingRestart(), cs) + writeResponse(s.logger, cancel, resp, moerr.NewRetryForCNRollingRestart(), cs) return } @@ -248,7 +249,7 @@ func (s *service) handleForwardLock( l == nil { // means that the lockservice sending the lock request holds a stale // lock table binding. - writeResponse(ctx, s.logger, cancel, resp, err, cs) + writeResponse(s.logger, cancel, resp, err, cs) return } @@ -256,12 +257,12 @@ func (s *service) handleForwardLock( txn.Lock() if !bytes.Equal(txn.txnID, req.Lock.TxnID) { txn.Unlock() - writeResponse(ctx, s.logger, cancel, resp, ErrTxnNotFound, cs) + writeResponse(s.logger, cancel, resp, ErrTxnNotFound, cs) return } if txn.deadlockFound { txn.Unlock() - writeResponse(ctx, s.logger, cancel, resp, ErrDeadLockDetected, cs) + writeResponse(s.logger, cancel, resp, ErrDeadLockDetected, cs) return } @@ -287,7 +288,7 @@ func (s *service) handleForwardLock( txn.Unlock() e = err resp.Lock.Result = result - writeResponse(ctx, s.logger, cancel, resp, err, cs) + writeResponse(s.logger, cancel, resp, err, cs) }) } @@ -302,11 +303,11 @@ func (s *service) handleRemoteUnlock( l == nil { // means that the lockservice sending the lock request holds a stale lock // table binding. - writeResponse(ctx, s.logger, cancel, resp, err, cs) + writeResponse(s.logger, cancel, resp, err, cs) return } err = s.Unlock(ctx, req.Unlock.TxnID, req.Unlock.CommitTS, req.Unlock.Mutations...) - writeResponse(ctx, s.logger, cancel, resp, err, cs) + writeResponse(s.logger, cancel, resp, err, cs) } func (s *service) handleValidateService( @@ -318,7 +319,7 @@ func (s *service) handleValidateService( resp.ValidateService = pb.ValidateServiceResponse{ OK: s.serviceID == req.ValidateService.ServiceID, } - writeResponse(ctx, s.logger, cancel, resp, nil, cs) + writeResponse(s.logger, cancel, resp, nil, cs) } func (s *service) handleGetActiveTxn( @@ -334,7 +335,7 @@ func (s *service) handleGetActiveTxn( return true }) } - writeResponse(ctx, s.logger, cancel, resp, nil, cs) + writeResponse(s.logger, cancel, resp, nil, cs) } func (s *service) handleRemoteGetLock( @@ -348,7 +349,7 @@ func (s *service) handleRemoteGetLock( l == nil { // means that the lockservice sending the lock request holds a stale lock // table binding. - writeResponse(ctx, s.logger, cancel, resp, err, cs) + writeResponse(s.logger, cancel, resp, err, cs) return } @@ -364,7 +365,7 @@ func (s *service) handleRemoteGetLock( }) resp.GetTxnLock.WaitingList = values }) - writeResponse(ctx, s.logger, cancel, resp, err, cs) + writeResponse(s.logger, cancel, resp, err, cs) } func (s *service) handleRemoteGetWaitingList( @@ -377,7 +378,7 @@ func (s *service) handleRemoteGetWaitingList( case s.fetchWhoWaitingListC <- who{ctx: ctx, cancel: cancel, cs: cs, resp: resp, txnID: req.GetWaitingList.Txn.TxnID}: return default: - writeResponse(ctx, s.logger, cancel, resp, ErrDeadLockDetected, cs) + writeResponse(s.logger, cancel, resp, ErrDeadLockDetected, cs) } } @@ -390,12 +391,12 @@ func (s *service) handleKeepRemoteLock( l, err := s.getLocalLockTable(req, resp) if err != nil || l == nil { - writeResponse(ctx, s.logger, cancel, resp, err, cs) + writeResponse(s.logger, cancel, resp, err, cs) return } s.activeTxnHolder.keepRemoteActiveTxn(req.KeepRemoteLock.ServiceID) - writeResponse(ctx, s.logger, cancel, resp, nil, cs) + writeResponse(s.logger, cancel, resp, nil, cs) } func (s *service) getLocalLockTable( @@ -556,19 +557,18 @@ func (s *service) handleFetchWhoWaitingMe(ctx context.Context) { false, "") if txn == nil { - writeResponse(w.ctx, s.logger, w.cancel, w.resp, nil, w.cs) + writeResponse(s.logger, w.cancel, w.resp, nil, w.cs) continue } txn.fetchWhoWaitingMe( s.serviceID, w.txnID, - s.activeTxnHolder, func(wt pb.WaitTxn) bool { w.resp.GetWaitingList.WaitingList = append(w.resp.GetWaitingList.WaitingList, wt) return true }, s.getLockTable) - writeResponse(w.ctx, s.logger, w.cancel, w.resp, nil, w.cs) + writeResponse(s.logger, w.cancel, w.resp, nil, w.cs) } } } diff --git a/pkg/lockservice/service_remote_test.go b/pkg/lockservice/service_remote_test.go index b1192ebc5d62..b562c7ede2ad 100644 --- a/pkg/lockservice/service_remote_test.go +++ b/pkg/lockservice/service_remote_test.go @@ -612,7 +612,7 @@ func runBindChangedTests( ) } -func waitBindDisabled(t *testing.T, alloc *lockTableAllocator, sid string) { +func waitBindDisabled(_ *testing.T, alloc *lockTableAllocator, sid string) { b := alloc.getServiceBinds(sid) if b == nil { return diff --git a/pkg/lockservice/service_test.go b/pkg/lockservice/service_test.go index f0a8b41b03b7..0a0ee48bd31e 100644 --- a/pkg/lockservice/service_test.go +++ b/pkg/lockservice/service_test.go @@ -1464,7 +1464,7 @@ func TestIssue3693(t *testing.T) { time.Second*10) defer cancel() - alloc.registerService(l.serviceID, 0) + alloc.registerService(l.serviceID) alloc.setRestartService("s1") for { @@ -3006,6 +3006,22 @@ func TestCannotCommit(t *testing.T) { ) } +func TestResumeInvalidService(t *testing.T) { + runLockServiceTests( + t, + []string{"s1"}, + func(alloc *lockTableAllocator, s []*service) { + alloc.inactiveService.Store(s[0].serviceID, time.Now()) + _, err := alloc.Valid(s[0].serviceID, []byte("testTxn"), nil) + require.True(t, moerr.IsMoErrCode(err, moerr.ErrCannotCommitOnInvalidCN)) + + require.NoError(t, s[0].Resume()) + _, err = alloc.Valid(s[0].serviceID, []byte("testTxn"), nil) + require.NoError(t, err) + }, + ) +} + func TestRetryLockSuccInRollingRestartCN(t *testing.T) { runLockServiceTests( t, @@ -3857,7 +3873,7 @@ func TestIssue14008(t *testing.T) { r1 *pb.Request, r2 *pb.Response, cs morpc.ClientSession) { - writeResponse(ctx, getLogger(s1.GetConfig().ServiceID), cf, r2, ErrTxnNotFound, cs) + writeResponse(getLogger(s1.GetConfig().ServiceID), cf, r2, ErrTxnNotFound, cs) }) var wg sync.WaitGroup for i := 0; i < 20; i++ { diff --git a/pkg/lockservice/txn.go b/pkg/lockservice/txn.go index 01a69799d385..0d0f75165140 100644 --- a/pkg/lockservice/txn.go +++ b/pkg/lockservice/txn.go @@ -69,7 +69,6 @@ func (txn activeTxn) TypeName() string { } func (txn *activeTxn) lockRemoved( - serviceID string, group uint32, table uint64, removedLocks map[string]struct{}) { @@ -131,14 +130,13 @@ func (txn *activeTxn) lockAdded( } func (txn *activeTxn) close( - serviceID string, txnID []byte, commitTS timestamp.Timestamp, lockTableFunc func(uint32, uint64) (lockTable, error), logger *log.MOLogger, mutations ...pb.ExtraMutation, ) error { - logTxnReadyToClose(logger, serviceID, txn) + logTxnReadyToClose(logger, txn) // cancel all blocked waiters txn.cancelBlocks(logger) @@ -175,14 +173,12 @@ func (txn *activeTxn) close( return func() { logTxnUnlockTable( logger, - serviceID, txn, table, ) l.unlock(txn, cs, commitTS, mutations...) logTxnUnlockTableCompleted( logger, - serviceID, txn, table, cs, @@ -228,7 +224,6 @@ func (txn *activeTxn) reset() { } func (txn *activeTxn) abort( - serviceID string, waitTxn pb.WaitTxn, err error, logger *log.MOLogger, @@ -322,7 +317,6 @@ func (txn *activeTxn) incLockTableRef(m map[uint32]map[uint64]uint64, serviceID func (txn *activeTxn) fetchWhoWaitingMe( serviceID string, txnID []byte, - holder activeTxnHolder, waiters func(pb.WaitTxn) bool, lockTableFunc func(uint32, uint64) (lockTable, error)) bool { txn.RLock() diff --git a/pkg/lockservice/txn_test.go b/pkg/lockservice/txn_test.go index 194f1db2216e..b17f5249064a 100644 --- a/pkg/lockservice/txn_test.go +++ b/pkg/lockservice/txn_test.go @@ -90,7 +90,6 @@ func TestClose(t *testing.T) { }) txn.close( - "s1", txn.txnID, timestamp.Timestamp{}, func(group uint32, table uint64) (lockTable, error) { diff --git a/pkg/lockservice/types.go b/pkg/lockservice/types.go index 25d9a9bcaaab..23ea0c8918a1 100644 --- a/pkg/lockservice/types.go +++ b/pkg/lockservice/types.go @@ -127,6 +127,13 @@ type LockService interface { CloseRemoteLockTable(group uint32, tableID, version uint64) (bool, error) } +type ResumeLockService interface { + LockService + + // Resume resume the current cn. + Resume() error +} + // lockTable is used to manage all locks of a Table. LockTable can be local or remote, as determined // by LockTableAllocator. // @@ -180,6 +187,10 @@ type LockTableAllocator interface { Valid(serviceID string, txnID []byte, binds []pb.LockTable) ([]uint64, error) // AddCannotCommit add cannot commit txn. AddCannotCommit(values []pb.OrphanTxn) [][]byte + // AddInvalidService add invalid service + AddInvalidService(serviceID string) + // HasInvalidService has invalid service + HasInvalidService(serviceID string) bool // Close close the lock table allocator Close() error diff --git a/pkg/pb/lock/lock.go b/pkg/pb/lock/lock.go index 4825b15cdcfe..b87a9fac4cd7 100644 --- a/pkg/pb/lock/lock.go +++ b/pkg/pb/lock/lock.go @@ -110,7 +110,7 @@ func (m LockTable) Equal(v LockTable) bool { // DebugString returns the debug string func (m LockTable) DebugString() string { - return fmt.Sprintf("%d-%s-%d", m.Table, m.ServiceID, m.Version) + return fmt.Sprintf("%d-%d(%d)-%s-%d", m.Group, m.Table, m.OriginTable, m.ServiceID, m.Version) } // WithGranularity set rows granularity, the default granularity is Row. diff --git a/pkg/pb/lock/lock.pb.go b/pkg/pb/lock/lock.pb.go index ac96d3824b53..8285cf62b3b0 100644 --- a/pkg/pb/lock/lock.pb.go +++ b/pkg/pb/lock/lock.pb.go @@ -166,6 +166,8 @@ const ( Method_CanRestartService Method = 14 // CheckOrphan check txn is orphan Method_CheckOrphan Method = 15 + // ResumeInvalidCN resume invalid cn + Method_ResumeInvalidCN Method = 16 ) var Method_name = map[int32]string{ @@ -185,6 +187,7 @@ var Method_name = map[int32]string{ 13: "RemainTxnInService", 14: "CanRestartService", 15: "CheckOrphan", + 16: "ResumeInvalidCN", } var Method_value = map[string]int32{ @@ -204,6 +207,7 @@ var Method_value = map[string]int32{ "RemainTxnInService": 13, "CanRestartService": 14, "CheckOrphan": 15, + "ResumeInvalidCN": 16, } func (x Method) String() string { @@ -479,6 +483,7 @@ type Request struct { CanRestartService CanRestartServiceRequest `protobuf:"bytes,15,opt,name=CanRestartService,proto3" json:"CanRestartService"` RemainTxnInService RemainTxnInServiceRequest `protobuf:"bytes,16,opt,name=RemainTxnInService,proto3" json:"RemainTxnInService"` CheckOrphan CheckOrphanRequest `protobuf:"bytes,17,opt,name=CheckOrphan,proto3" json:"CheckOrphan"` + ResumeInvalidCN ResumeInvalidCNRequest `protobuf:"bytes,18,opt,name=ResumeInvalidCN,proto3" json:"ResumeInvalidCN"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -636,6 +641,13 @@ func (m *Request) GetCheckOrphan() CheckOrphanRequest { return CheckOrphanRequest{} } +func (m *Request) GetResumeInvalidCN() ResumeInvalidCNRequest { + if m != nil { + return m.ResumeInvalidCN + } + return ResumeInvalidCNRequest{} +} + // Response response type Response struct { // RequestID corresponding request id @@ -659,6 +671,7 @@ type Response struct { CanRestartService CanRestartServiceResponse `protobuf:"bytes,16,opt,name=CanRestartService,proto3" json:"CanRestartService"` RemainTxnInService RemainTxnInServiceResponse `protobuf:"bytes,17,opt,name=RemainTxnInService,proto3" json:"RemainTxnInService"` CheckOrphan CheckOrphanResponse `protobuf:"bytes,18,opt,name=CheckOrphan,proto3" json:"CheckOrphan"` + ResumeInvalidCN ResumeInvalidCNResponse `protobuf:"bytes,19,opt,name=ResumeInvalidCN,proto3" json:"ResumeInvalidCN"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -823,6 +836,13 @@ func (m *Response) GetCheckOrphan() CheckOrphanResponse { return CheckOrphanResponse{} } +func (m *Response) GetResumeInvalidCN() ResumeInvalidCNResponse { + if m != nil { + return m.ResumeInvalidCN + } + return ResumeInvalidCNResponse{} +} + // LockRequest lock request type LockRequest struct { TxnID []byte `protobuf:"bytes,1,opt,name=TxnID,proto3" json:"TxnID,omitempty"` @@ -2599,6 +2619,92 @@ func (m *ExtraMutation) GetReplaceTo() []byte { return nil } +type ResumeInvalidCNRequest struct { + ServiceID string `protobuf:"bytes,1,opt,name=ServiceID,proto3" json:"ServiceID,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ResumeInvalidCNRequest) Reset() { *m = ResumeInvalidCNRequest{} } +func (m *ResumeInvalidCNRequest) String() string { return proto.CompactTextString(m) } +func (*ResumeInvalidCNRequest) ProtoMessage() {} +func (*ResumeInvalidCNRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_164ad2988c7acaf1, []int{36} +} +func (m *ResumeInvalidCNRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResumeInvalidCNRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ResumeInvalidCNRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ResumeInvalidCNRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResumeInvalidCNRequest.Merge(m, src) +} +func (m *ResumeInvalidCNRequest) XXX_Size() int { + return m.ProtoSize() +} +func (m *ResumeInvalidCNRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ResumeInvalidCNRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ResumeInvalidCNRequest proto.InternalMessageInfo + +func (m *ResumeInvalidCNRequest) GetServiceID() string { + if m != nil { + return m.ServiceID + } + return "" +} + +type ResumeInvalidCNResponse struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ResumeInvalidCNResponse) Reset() { *m = ResumeInvalidCNResponse{} } +func (m *ResumeInvalidCNResponse) String() string { return proto.CompactTextString(m) } +func (*ResumeInvalidCNResponse) ProtoMessage() {} +func (*ResumeInvalidCNResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_164ad2988c7acaf1, []int{37} +} +func (m *ResumeInvalidCNResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResumeInvalidCNResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ResumeInvalidCNResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ResumeInvalidCNResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResumeInvalidCNResponse.Merge(m, src) +} +func (m *ResumeInvalidCNResponse) XXX_Size() int { + return m.ProtoSize() +} +func (m *ResumeInvalidCNResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ResumeInvalidCNResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ResumeInvalidCNResponse proto.InternalMessageInfo + func init() { proto.RegisterEnum("lock.Granularity", Granularity_name, Granularity_value) proto.RegisterEnum("lock.LockMode", LockMode_name, LockMode_value) @@ -2642,135 +2748,140 @@ func init() { proto.RegisterType((*OrphanTxn)(nil), "lock.OrphanTxn") proto.RegisterType((*Result)(nil), "lock.Result") proto.RegisterType((*ExtraMutation)(nil), "lock.ExtraMutation") + proto.RegisterType((*ResumeInvalidCNRequest)(nil), "lock.ResumeInvalidCNRequest") + proto.RegisterType((*ResumeInvalidCNResponse)(nil), "lock.ResumeInvalidCNResponse") } func init() { proto.RegisterFile("lock.proto", fileDescriptor_164ad2988c7acaf1) } var fileDescriptor_164ad2988c7acaf1 = []byte{ - // 1959 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x59, 0x4b, 0x73, 0x1b, 0xc7, - 0x11, 0xe6, 0x02, 0x20, 0x1e, 0xbd, 0x78, 0x2c, 0x87, 0x14, 0xbd, 0x52, 0x1c, 0x0a, 0xd9, 0x92, - 0xab, 0x60, 0x3a, 0x16, 0x4b, 0x54, 0x64, 0x3b, 0x72, 0xac, 0x8a, 0x04, 0x4a, 0xb4, 0xac, 0x07, - 0x9d, 0x01, 0xa4, 0x54, 0xe5, 0xb6, 0x04, 0x46, 0xe4, 0x16, 0xc1, 0x5d, 0x64, 0x77, 0x20, 0x82, - 0x3f, 0x20, 0x55, 0xf9, 0x05, 0x39, 0xe7, 0x98, 0x4b, 0xf2, 0x3b, 0x7c, 0x74, 0x55, 0x6e, 0x39, - 0xa4, 0x12, 0xe5, 0x2f, 0x24, 0xf7, 0xd4, 0x3c, 0x76, 0x77, 0x66, 0x1f, 0x84, 0xe5, 0xdb, 0x4e, - 0x77, 0xcf, 0xd7, 0xd3, 0x8d, 0xd9, 0xaf, 0xbb, 0x17, 0x00, 0xb3, 0x60, 0x72, 0x76, 0x7b, 0x1e, - 0x06, 0x34, 0x40, 0x35, 0xf6, 0x7c, 0xe3, 0xd3, 0x13, 0x8f, 0x9e, 0x2e, 0x8e, 0x6f, 0x4f, 0x82, - 0xf3, 0xbd, 0x93, 0xe0, 0x24, 0xd8, 0xe3, 0xca, 0xe3, 0xc5, 0x1b, 0xbe, 0xe2, 0x0b, 0xfe, 0x24, - 0x36, 0xdd, 0xe8, 0x51, 0xef, 0x9c, 0x44, 0xd4, 0x3d, 0x9f, 0x0b, 0x81, 0xf3, 0xdf, 0x0a, 0x98, - 0xcf, 0x83, 0xc9, 0xd9, 0xd1, 0x9c, 0x7a, 0x81, 0x1f, 0xa1, 0xbb, 0x60, 0x1e, 0x86, 0xae, 0xbf, - 0x98, 0xb9, 0xa1, 0x47, 0x2f, 0x6d, 0xa3, 0x6f, 0x0c, 0xba, 0xfb, 0x1b, 0xb7, 0xb9, 0x5f, 0x45, - 0x81, 0x55, 0x2b, 0xe4, 0x40, 0xed, 0x45, 0x30, 0x25, 0x76, 0x85, 0x5b, 0x77, 0x85, 0x35, 0x43, - 0x65, 0x52, 0xcc, 0x75, 0x68, 0x00, 0xf5, 0x6f, 0x83, 0x99, 0x37, 0xb9, 0xb4, 0xab, 0xdc, 0xca, - 0x12, 0x56, 0xbf, 0x75, 0x3d, 0x2a, 0xe4, 0x58, 0xea, 0xd1, 0x87, 0xd0, 0x7a, 0x12, 0x84, 0x17, - 0x6e, 0x38, 0x1d, 0x07, 0x76, 0xad, 0x6f, 0x0c, 0x5a, 0x38, 0x15, 0xa0, 0x01, 0xf4, 0xc6, 0xee, - 0xf1, 0x8c, 0x1c, 0x90, 0x37, 0xc3, 0x53, 0xd7, 0x3f, 0x21, 0x53, 0x7b, 0xbd, 0x6f, 0x0c, 0x9a, - 0x38, 0x2b, 0x66, 0x38, 0x98, 0xd0, 0xf0, 0x92, 0xb9, 0xb0, 0xeb, 0x7d, 0x63, 0x50, 0xc5, 0xa9, - 0x00, 0x6d, 0xc1, 0xfa, 0x61, 0x18, 0x2c, 0xe6, 0x76, 0xa3, 0x6f, 0x0c, 0x3a, 0x58, 0x2c, 0xd0, - 0x2e, 0x34, 0x47, 0xa7, 0x6e, 0x38, 0xf5, 0xfc, 0x13, 0xbb, 0xa9, 0x46, 0x13, 0x4b, 0x71, 0xa2, - 0x47, 0xf7, 0x01, 0x46, 0xbe, 0x3b, 0x1f, 0x9d, 0x06, 0x74, 0x1c, 0xd9, 0xad, 0xbe, 0x31, 0x30, - 0xf7, 0xb7, 0x6e, 0xa7, 0x09, 0x1e, 0xc7, 0x4f, 0x8f, 0x6a, 0xdf, 0xfd, 0xf3, 0xe6, 0x1a, 0x56, - 0xac, 0x9d, 0xbf, 0x1b, 0xd0, 0x62, 0x09, 0xe2, 0x67, 0x66, 0x67, 0xe1, 0x0f, 0x3c, 0xdd, 0x35, - 0x2c, 0x16, 0xec, 0xfc, 0x23, 0x12, 0xbe, 0xf5, 0x26, 0xe4, 0xe9, 0x01, 0x4f, 0x6d, 0x0b, 0xa7, - 0x02, 0x64, 0x43, 0xe3, 0x35, 0x09, 0x23, 0x2f, 0xf0, 0x79, 0x42, 0x6b, 0x38, 0x5e, 0x32, 0xb4, - 0xd7, 0xee, 0xcc, 0x9b, 0xf2, 0xdc, 0x35, 0xb1, 0x58, 0xa4, 0xf1, 0xae, 0x97, 0xc5, 0x5b, 0x5f, - 0x11, 0x6f, 0x1f, 0xcc, 0xa3, 0xd0, 0x3b, 0xf1, 0x7c, 0x71, 0xd6, 0x06, 0xf7, 0xaa, 0x8a, 0x9c, - 0x7f, 0x34, 0xa1, 0x81, 0xc9, 0xef, 0x17, 0x24, 0xa2, 0x22, 0xfb, 0xfc, 0xf1, 0xe9, 0x81, 0x8c, - 0x2b, 0x15, 0xa0, 0xbb, 0x4a, 0xf8, 0x3c, 0x36, 0x73, 0xbf, 0x97, 0x5e, 0x1b, 0x2e, 0x96, 0x59, - 0x53, 0xd2, 0x74, 0x0b, 0xea, 0x2f, 0x08, 0x3d, 0x0d, 0xa6, 0xf2, 0x0a, 0xb5, 0xc5, 0x0e, 0x21, - 0xc3, 0x52, 0x87, 0x3e, 0x81, 0x1a, 0xdb, 0xc2, 0xa3, 0x37, 0xe3, 0xab, 0xcb, 0x24, 0xd2, 0xbb, - 0xc4, 0xe5, 0x46, 0xe8, 0x0e, 0xd4, 0x5f, 0xf9, 0xcc, 0x82, 0xa7, 0xc5, 0xdc, 0xdf, 0x14, 0xe6, - 0x42, 0xa6, 0x6f, 0x90, 0x86, 0xe8, 0x2b, 0x80, 0x43, 0x42, 0xc7, 0x4b, 0x9f, 0x7b, 0xa9, 0xf3, - 0x6d, 0x1f, 0xc8, 0x17, 0x24, 0x91, 0xeb, 0x5b, 0x95, 0x0d, 0xe8, 0x29, 0x74, 0x0f, 0x09, 0x65, - 0x57, 0xd0, 0xf3, 0x4f, 0x9e, 0x7b, 0x11, 0xe5, 0x89, 0x34, 0xf7, 0x7f, 0x92, 0x40, 0x28, 0x3a, - 0x1d, 0x26, 0xb3, 0x11, 0xfd, 0x02, 0x1a, 0x87, 0x84, 0x3e, 0xf2, 0xfc, 0x29, 0xbf, 0xab, 0xec, - 0xf6, 0xc5, 0x18, 0x4c, 0xa8, 0x6f, 0x8e, 0x4d, 0x11, 0x86, 0x8d, 0x67, 0x84, 0xcc, 0xd3, 0x3c, - 0xb3, 0xfd, 0xe2, 0xf6, 0xee, 0x88, 0xfd, 0x39, 0xb5, 0x8e, 0x94, 0xdf, 0xce, 0x82, 0x62, 0x42, - 0x4c, 0xce, 0x03, 0x4a, 0x78, 0x5e, 0x40, 0x0d, 0x4a, 0xd7, 0x65, 0x82, 0xd2, 0x95, 0xe8, 0x39, - 0xf4, 0xf8, 0x85, 0x75, 0x29, 0x91, 0x97, 0xdd, 0x36, 0x39, 0xd6, 0x87, 0x02, 0x2b, 0xa3, 0xd4, - 0xc1, 0xb2, 0x5b, 0xd1, 0x10, 0xda, 0x43, 0xd7, 0xf7, 0x03, 0x3a, 0x0c, 0xce, 0xcf, 0x3d, 0x6a, - 0xb7, 0x39, 0xd4, 0x75, 0x01, 0xa5, 0x6a, 0x74, 0x1c, 0x6d, 0x13, 0x03, 0x39, 0x24, 0xf4, 0xe1, - 0x84, 0x7a, 0x6f, 0xc9, 0x78, 0xe9, 0xdb, 0x1d, 0x15, 0x44, 0xd5, 0x64, 0x40, 0x54, 0x15, 0x4b, - 0xfb, 0x88, 0x50, 0xcc, 0x18, 0x21, 0xa4, 0x71, 0x64, 0x5d, 0x35, 0xed, 0x39, 0x75, 0x26, 0xed, - 0x39, 0x3d, 0xc3, 0x1c, 0xba, 0x7e, 0x06, 0xb3, 0xa7, 0x62, 0xe6, 0xd4, 0x19, 0xcc, 0x9c, 0x1e, - 0xbd, 0x02, 0x84, 0xc9, 0xb9, 0xeb, 0xf9, 0xe3, 0xa5, 0xff, 0xd4, 0x8f, 0x41, 0x2d, 0x0e, 0x7a, - 0x53, 0x80, 0xe6, 0xf5, 0x3a, 0x6a, 0x01, 0x00, 0xfa, 0x35, 0x98, 0xc3, 0x53, 0x32, 0x39, 0x3b, - 0x0a, 0xe7, 0xa7, 0xae, 0x6f, 0x6f, 0x70, 0x3c, 0x5b, 0x1e, 0x32, 0x55, 0xe8, 0x40, 0xea, 0x16, - 0xe7, 0x7f, 0x4d, 0x68, 0x62, 0x12, 0xcd, 0x03, 0x3f, 0x22, 0x2b, 0xd8, 0x25, 0x25, 0x8a, 0xca, - 0x15, 0x44, 0xb1, 0x05, 0xeb, 0x8f, 0xc3, 0x30, 0x08, 0x39, 0x9b, 0xb4, 0xb1, 0x58, 0xa0, 0x8f, - 0xa1, 0xf1, 0x92, 0x5c, 0xf0, 0x97, 0xa2, 0x56, 0xc8, 0x4b, 0x38, 0xd6, 0xa3, 0x9f, 0x4b, 0xa6, - 0x11, 0xd4, 0x81, 0x54, 0xa6, 0x11, 0xc7, 0xd4, 0xa8, 0x66, 0x3f, 0xa1, 0x9a, 0xba, 0xfa, 0xb2, - 0xc6, 0x54, 0xa3, 0xed, 0x88, 0xb9, 0xe6, 0x81, 0xc6, 0x35, 0x0d, 0x35, 0x69, 0x2a, 0xd7, 0x68, - 0x7b, 0x55, 0xb2, 0xf9, 0x26, 0x47, 0x36, 0x4d, 0xf5, 0x5d, 0xca, 0x92, 0x8d, 0x86, 0x93, 0x65, - 0x9b, 0x7b, 0x29, 0xdb, 0x08, 0xb6, 0xb8, 0x96, 0x61, 0x1b, 0x6d, 0x77, 0x42, 0x37, 0xa3, 0x22, - 0xba, 0x01, 0xf5, 0x3a, 0x15, 0xd0, 0x8d, 0x06, 0x55, 0xc0, 0x37, 0xdf, 0xe4, 0xf8, 0x46, 0xe3, - 0x88, 0x2c, 0xdf, 0xe8, 0x71, 0x65, 0x08, 0xe7, 0x45, 0x9e, 0x70, 0x04, 0x4b, 0xfc, 0xb4, 0x84, - 0x70, 0x34, 0xb4, 0x1c, 0xe3, 0x1c, 0x64, 0x18, 0x47, 0x90, 0xc5, 0x8d, 0x22, 0xc6, 0xd1, 0x80, - 0x74, 0xca, 0x39, 0xc8, 0x50, 0x4e, 0x57, 0x45, 0xd1, 0x29, 0x47, 0x47, 0xd1, 0x38, 0x67, 0x54, - 0xc4, 0x39, 0x3d, 0x35, 0xf7, 0x05, 0x9c, 0xa3, 0xe7, 0x3e, 0x4f, 0x3a, 0xa3, 0x22, 0xd2, 0xd1, - 0xf8, 0xa1, 0x80, 0x74, 0x74, 0xd0, 0x3c, 0xeb, 0xbc, 0x2e, 0x64, 0x1d, 0xc1, 0x12, 0xfd, 0x72, - 0xd6, 0xd1, 0x60, 0x8b, 0x68, 0xe7, 0xa1, 0x4e, 0x3b, 0x48, 0xa3, 0x7f, 0x95, 0x76, 0x34, 0x24, - 0x8d, 0x77, 0xfe, 0x68, 0x88, 0x0e, 0x39, 0x6e, 0x6c, 0x58, 0xb3, 0xb6, 0xf4, 0x25, 0xed, 0xb4, - 0xb1, 0x58, 0xac, 0x68, 0xd6, 0x10, 0xd4, 0x70, 0x70, 0x11, 0xd9, 0xd5, 0x7e, 0x75, 0xd0, 0xc6, - 0xfc, 0x19, 0xdd, 0x81, 0x86, 0x6c, 0xba, 0xf3, 0xad, 0x8a, 0x54, 0xc4, 0xef, 0x92, 0x5c, 0x3a, - 0xf7, 0xa1, 0xad, 0x5e, 0x68, 0xb4, 0x0b, 0x75, 0x4c, 0xa2, 0xc5, 0x8c, 0xf2, 0xb3, 0x98, 0x31, - 0xcf, 0x09, 0x59, 0x4c, 0x25, 0x62, 0xe5, 0x7c, 0x09, 0x1b, 0xb9, 0xf6, 0xa4, 0x24, 0x16, 0x0b, - 0xaa, 0x38, 0xb8, 0xe0, 0x51, 0xb4, 0x31, 0x7b, 0x74, 0x5c, 0x40, 0x79, 0xbe, 0x91, 0x8d, 0xe6, - 0x42, 0xb4, 0xad, 0xeb, 0x58, 0x2c, 0xd0, 0x3d, 0x30, 0x55, 0xc2, 0xa9, 0xf4, 0xab, 0x03, 0x73, - 0xbf, 0x93, 0x76, 0xfb, 0xe3, 0xa5, 0x1f, 0xa7, 0x59, 0xb1, 0x73, 0x1e, 0xc0, 0xb5, 0xc2, 0xde, - 0x07, 0x7d, 0x04, 0x55, 0xf6, 0x06, 0x88, 0x08, 0x0b, 0x71, 0x98, 0xde, 0x39, 0x82, 0xed, 0x62, - 0x3a, 0xcb, 0x1e, 0xc8, 0xf8, 0x81, 0x07, 0xfa, 0x0a, 0x1a, 0x52, 0x5b, 0xfe, 0x93, 0x0f, 0x43, - 0xe2, 0x52, 0x32, 0x3d, 0xf2, 0xe3, 0x9f, 0x3c, 0x11, 0x38, 0x7f, 0x32, 0xa0, 0xa3, 0xb5, 0x91, - 0x25, 0x28, 0x9f, 0x41, 0x53, 0xbc, 0xf3, 0xe3, 0x91, 0x6c, 0x84, 0xaf, 0x9a, 0x21, 0x12, 0x5b, - 0xf4, 0x39, 0xb4, 0x5e, 0x2c, 0xa8, 0x2b, 0x2e, 0x50, 0x95, 0xc7, 0x24, 0x9b, 0xd7, 0xc7, 0x4b, - 0x1a, 0xba, 0xb1, 0x2e, 0xee, 0xa2, 0x13, 0x5b, 0xc7, 0x82, 0xae, 0x5e, 0x73, 0x9c, 0xbf, 0x18, - 0xbc, 0x4c, 0x28, 0x9d, 0x9e, 0x7e, 0x9d, 0x8d, 0xec, 0x75, 0x4e, 0xe6, 0x95, 0x8a, 0x3a, 0xaf, - 0x24, 0x13, 0x46, 0xb5, 0x6c, 0xc2, 0xa8, 0xbd, 0xdf, 0x84, 0xb1, 0x9e, 0x9f, 0x30, 0x9e, 0x40, - 0x2f, 0x53, 0x6f, 0x7e, 0xd4, 0x28, 0xe1, 0xfc, 0xd5, 0x00, 0xbb, 0xac, 0xcd, 0x5d, 0x11, 0xfc, - 0x2d, 0xa8, 0x8f, 0xa8, 0x4b, 0x17, 0x91, 0xde, 0x5c, 0x08, 0x19, 0x96, 0x3a, 0xb4, 0x0d, 0x75, - 0xfe, 0xfb, 0xc6, 0xef, 0xbc, 0x5c, 0xa1, 0x7b, 0x00, 0x89, 0x4f, 0xf6, 0xe2, 0x57, 0xcb, 0x8f, - 0xab, 0x18, 0x3a, 0xbf, 0x81, 0xeb, 0xa5, 0x65, 0x12, 0x75, 0xa1, 0x72, 0xf4, 0x8c, 0x1f, 0xb4, - 0x89, 0x2b, 0x47, 0xcf, 0x7e, 0xd8, 0x09, 0x9d, 0x2f, 0xc0, 0x2e, 0xeb, 0x38, 0xaf, 0xce, 0x80, - 0xf3, 0x09, 0x5c, 0x2f, 0xad, 0x1b, 0xd9, 0xc3, 0x30, 0x37, 0x65, 0x4d, 0xe8, 0x6a, 0x37, 0xa5, - 0x95, 0x24, 0xe7, 0xe6, 0x97, 0x70, 0xbd, 0xb4, 0x2d, 0x5d, 0xe1, 0xe7, 0x3e, 0xdc, 0x28, 0xaf, - 0x2d, 0xa2, 0xd3, 0x94, 0x5a, 0x49, 0x74, 0xa9, 0xc0, 0xb9, 0x07, 0xd7, 0x0a, 0x87, 0x9b, 0x15, - 0x2e, 0x07, 0xb0, 0x5d, 0xdc, 0xa3, 0xe4, 0xe2, 0xfa, 0x0c, 0xb6, 0x8b, 0x27, 0x9e, 0x15, 0x1e, - 0x3e, 0x86, 0x0f, 0x4a, 0x1a, 0x97, 0x9c, 0x0b, 0x0c, 0x9b, 0x05, 0x93, 0x10, 0xfa, 0x12, 0x3a, - 0xa2, 0x02, 0x32, 0xda, 0x4f, 0x89, 0x53, 0x5e, 0xd6, 0x44, 0x25, 0x2f, 0xab, 0x6e, 0xeb, 0xfc, - 0x0a, 0xb6, 0x8a, 0x7a, 0x1d, 0x74, 0x0b, 0x3a, 0x42, 0xc2, 0x68, 0x56, 0x64, 0x94, 0xbd, 0x1d, - 0xba, 0xd0, 0xb9, 0x0b, 0x9b, 0x05, 0x63, 0xd5, 0x8a, 0x88, 0x1f, 0xc0, 0x56, 0x51, 0x63, 0x94, - 0x7e, 0x0e, 0x31, 0xd4, 0xcf, 0x21, 0x96, 0xa8, 0x2a, 0x15, 0xee, 0x9e, 0x17, 0x90, 0x03, 0x40, - 0xf9, 0x41, 0x64, 0x05, 0x17, 0x24, 0x28, 0x46, 0x8c, 0xf2, 0x29, 0x6c, 0x16, 0xf4, 0x15, 0x8c, - 0x0e, 0x64, 0x0b, 0x22, 0x4e, 0x21, 0x57, 0xce, 0xe7, 0xd0, 0x4a, 0x12, 0x87, 0x6c, 0x68, 0xc4, - 0x9d, 0x8f, 0xf0, 0x14, 0x2f, 0x0b, 0x4e, 0xfb, 0x87, 0x6a, 0x5c, 0xfb, 0xd1, 0x1d, 0x68, 0xb2, - 0x2b, 0xc4, 0xcb, 0x90, 0x71, 0x15, 0xff, 0x25, 0x66, 0x8c, 0x68, 0xbf, 0x76, 0xa3, 0x61, 0xe0, - 0xbf, 0x99, 0x79, 0x13, 0xca, 0xcf, 0xdf, 0xc4, 0xaa, 0x88, 0xfd, 0x50, 0x5f, 0xbb, 0xd1, 0xb7, - 0x21, 0x79, 0x2b, 0xfb, 0xd8, 0x2a, 0xb7, 0xd1, 0x85, 0xe8, 0x0b, 0x68, 0x25, 0x15, 0x4a, 0x76, - 0x31, 0x57, 0x55, 0xaf, 0xd4, 0xf8, 0x3d, 0x3e, 0xe3, 0xf5, 0xc1, 0x8c, 0x4f, 0xf5, 0x8c, 0x5c, - 0xf2, 0xe1, 0xa9, 0x8d, 0x55, 0x91, 0x6a, 0xc1, 0xb2, 0xd4, 0xd0, 0x2d, 0x58, 0x66, 0x77, 0x00, - 0xd8, 0xa9, 0x59, 0x3d, 0x27, 0x21, 0x9f, 0x81, 0xda, 0x58, 0x91, 0xb0, 0xcc, 0x8b, 0x27, 0xf1, - 0x1d, 0xaf, 0x83, 0xe3, 0x25, 0xdb, 0xf9, 0x92, 0x5c, 0xb0, 0xc4, 0x3d, 0x9c, 0x8a, 0xb9, 0xa5, - 0x89, 0x15, 0x89, 0x33, 0x82, 0x8e, 0x56, 0x6f, 0xd9, 0x4f, 0x75, 0x46, 0x2e, 0x65, 0x8d, 0x67, - 0x8f, 0xac, 0xf9, 0x8b, 0xce, 0xbc, 0xb9, 0xcc, 0x32, 0x7f, 0x66, 0xd7, 0x2a, 0x24, 0xf3, 0x99, - 0x3b, 0x21, 0xe3, 0x40, 0xce, 0x9f, 0xa9, 0x60, 0xf7, 0x67, 0xda, 0x47, 0x58, 0xd4, 0xe0, 0xfd, - 0x98, 0xb5, 0x86, 0x5a, 0xb0, 0x8e, 0x59, 0x52, 0x2c, 0x63, 0xf7, 0x23, 0xf1, 0xa3, 0xf3, 0x4f, - 0xab, 0x1d, 0x68, 0x3d, 0x5e, 0x4e, 0x66, 0x8b, 0xc8, 0x7b, 0x4b, 0xac, 0x35, 0x04, 0x50, 0x67, - 0x15, 0x95, 0x4c, 0x2d, 0x63, 0xf7, 0x16, 0x40, 0xfa, 0x85, 0x15, 0x35, 0xa1, 0xc6, 0x56, 0xd6, - 0x1a, 0x6a, 0x43, 0xf3, 0x89, 0x1b, 0xd1, 0x27, 0xae, 0x37, 0xb3, 0x8c, 0xdd, 0x9b, 0x69, 0x8d, - 0x66, 0x36, 0x2f, 0x03, 0x9f, 0x08, 0x6f, 0x8f, 0x2e, 0x99, 0x63, 0x63, 0xf7, 0x6f, 0x95, 0x78, - 0xa2, 0x66, 0x7a, 0xe6, 0x58, 0xf8, 0x11, 0x8d, 0x84, 0x65, 0xa0, 0xae, 0x3a, 0xa8, 0x5a, 0x15, - 0x84, 0xb2, 0x83, 0xa7, 0x55, 0x65, 0x32, 0x9d, 0xf4, 0xac, 0x1a, 0x32, 0x93, 0xa1, 0xd2, 0x5a, - 0x47, 0xd7, 0x0a, 0x46, 0x45, 0xab, 0x8e, 0x7a, 0x60, 0xca, 0xcf, 0xbf, 0x7c, 0x53, 0x03, 0x6d, - 0x40, 0x47, 0x0a, 0xa4, 0xff, 0x26, 0xda, 0xcc, 0x0d, 0x71, 0x56, 0x0b, 0x59, 0xfa, 0x28, 0x66, - 0x01, 0x93, 0xa8, 0x1c, 0x61, 0x99, 0xcc, 0x67, 0xae, 0x96, 0x59, 0x6d, 0xb4, 0x5d, 0x34, 0x8f, - 0x58, 0x1d, 0x66, 0x9e, 0xab, 0x49, 0x56, 0x97, 0x1d, 0x51, 0x79, 0xeb, 0xad, 0xde, 0x2e, 0x89, - 0x4b, 0xb0, 0x70, 0xc0, 0xed, 0xd8, 0xe9, 0x1f, 0xfb, 0x2c, 0x30, 0x6b, 0x8d, 0x39, 0x50, 0xc4, - 0x32, 0x51, 0x96, 0xa1, 0x98, 0xbf, 0xe2, 0xb9, 0x1c, 0x2d, 0x26, 0x13, 0xab, 0xa2, 0x88, 0x53, - 0xf7, 0x56, 0xf5, 0xd1, 0xf0, 0xfb, 0x7f, 0xef, 0x18, 0xdf, 0xbd, 0xdb, 0x31, 0xbe, 0x7f, 0xb7, - 0x63, 0xfc, 0xeb, 0xdd, 0xce, 0xda, 0x9f, 0xff, 0xb3, 0x63, 0xfc, 0x4e, 0xfd, 0x4f, 0xe0, 0xdc, - 0xa5, 0xa1, 0xb7, 0x0c, 0x78, 0x0b, 0x15, 0x2f, 0x7c, 0xb2, 0x37, 0x3f, 0x3b, 0xd9, 0x9b, 0x1f, - 0xef, 0xb1, 0xec, 0x1d, 0xd7, 0xf9, 0x3f, 0x01, 0x77, 0xff, 0x1f, 0x00, 0x00, 0xff, 0xff, 0xd7, - 0x5b, 0xeb, 0xde, 0x5d, 0x18, 0x00, 0x00, + // 2014 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x59, 0xcd, 0x72, 0x1b, 0xc7, + 0x11, 0xe6, 0x02, 0x20, 0x7e, 0x1a, 0x7f, 0xcb, 0x21, 0x45, 0x2d, 0x15, 0x87, 0x42, 0xb6, 0xe4, + 0x2a, 0x98, 0x8e, 0xc5, 0x12, 0x15, 0xd9, 0x8e, 0x1c, 0xab, 0x22, 0x81, 0x12, 0x4d, 0x4b, 0x14, + 0x9d, 0x01, 0xa4, 0x54, 0xe5, 0xb6, 0x04, 0x46, 0xe4, 0x16, 0x81, 0x5d, 0x64, 0xb1, 0xe0, 0xcf, + 0x03, 0xa4, 0x2a, 0x4f, 0x90, 0x73, 0x8e, 0xbe, 0xe4, 0x29, 0x92, 0x83, 0x8f, 0xae, 0xca, 0x3d, + 0x95, 0x28, 0xaf, 0x90, 0x07, 0x48, 0xf5, 0xcc, 0x2c, 0x76, 0x66, 0x7f, 0x08, 0xdb, 0xb7, 0x9d, + 0xee, 0x9e, 0xfe, 0xdb, 0xde, 0xaf, 0xbb, 0x01, 0x80, 0xb1, 0x3f, 0x3c, 0xbf, 0x3f, 0x0d, 0xfc, + 0xd0, 0x27, 0x25, 0x7c, 0xbe, 0xf3, 0xc9, 0xa9, 0x1b, 0x9e, 0xcd, 0x4f, 0xee, 0x0f, 0xfd, 0xc9, + 0xee, 0xa9, 0x7f, 0xea, 0xef, 0x72, 0xe6, 0xc9, 0xfc, 0x1d, 0x3f, 0xf1, 0x03, 0x7f, 0x12, 0x97, + 0xee, 0xb4, 0x43, 0x77, 0xc2, 0x66, 0xa1, 0x33, 0x99, 0x0a, 0x82, 0xfd, 0xbf, 0x02, 0xd4, 0x5f, + 0xf9, 0xc3, 0xf3, 0xe3, 0x69, 0xe8, 0xfa, 0xde, 0x8c, 0x3c, 0x84, 0xfa, 0x41, 0xe0, 0x78, 0xf3, + 0xb1, 0x13, 0xb8, 0xe1, 0xb5, 0x65, 0x74, 0x8c, 0x6e, 0x6b, 0x6f, 0xed, 0x3e, 0xb7, 0xab, 0x30, + 0xa8, 0x2a, 0x45, 0x6c, 0x28, 0x1d, 0xf9, 0x23, 0x66, 0x15, 0xb8, 0x74, 0x4b, 0x48, 0xa3, 0x56, + 0xa4, 0x52, 0xce, 0x23, 0x5d, 0x28, 0x7f, 0xe3, 0x8f, 0xdd, 0xe1, 0xb5, 0x55, 0xe4, 0x52, 0xa6, + 0x90, 0xfa, 0xbd, 0xe3, 0x86, 0x82, 0x4e, 0x25, 0x9f, 0x7c, 0x00, 0xb5, 0x17, 0x7e, 0x70, 0xe9, + 0x04, 0xa3, 0x81, 0x6f, 0x95, 0x3a, 0x46, 0xb7, 0x46, 0x63, 0x02, 0xe9, 0x42, 0x7b, 0xe0, 0x9c, + 0x8c, 0xd9, 0x3e, 0x7b, 0xd7, 0x3b, 0x73, 0xbc, 0x53, 0x36, 0xb2, 0x56, 0x3b, 0x46, 0xb7, 0x4a, + 0x93, 0x64, 0xd4, 0x43, 0x59, 0x18, 0x5c, 0xa3, 0x09, 0xab, 0xdc, 0x31, 0xba, 0x45, 0x1a, 0x13, + 0xc8, 0x06, 0xac, 0x1e, 0x04, 0xfe, 0x7c, 0x6a, 0x55, 0x3a, 0x46, 0xb7, 0x49, 0xc5, 0x81, 0xec, + 0x40, 0xb5, 0x7f, 0xe6, 0x04, 0x23, 0xd7, 0x3b, 0xb5, 0xaa, 0x6a, 0x34, 0x11, 0x95, 0x2e, 0xf8, + 0xe4, 0x31, 0x40, 0xdf, 0x73, 0xa6, 0xfd, 0x33, 0x3f, 0x1c, 0xcc, 0xac, 0x5a, 0xc7, 0xe8, 0xd6, + 0xf7, 0x36, 0xee, 0xc7, 0x09, 0x1e, 0x44, 0x4f, 0xcf, 0x4a, 0xdf, 0xfd, 0xeb, 0xee, 0x0a, 0x55, + 0xa4, 0xed, 0x7f, 0x1a, 0x50, 0xc3, 0x04, 0x71, 0x9f, 0xd1, 0x17, 0xfe, 0xc0, 0xd3, 0x5d, 0xa2, + 0xe2, 0x80, 0xfe, 0xf7, 0x59, 0x70, 0xe1, 0x0e, 0xd9, 0xe1, 0x3e, 0x4f, 0x6d, 0x8d, 0xc6, 0x04, + 0x62, 0x41, 0xe5, 0x2d, 0x0b, 0x66, 0xae, 0xef, 0xf1, 0x84, 0x96, 0x68, 0x74, 0x44, 0x6d, 0x6f, + 0x9d, 0xb1, 0x3b, 0xe2, 0xb9, 0xab, 0x52, 0x71, 0x88, 0xe3, 0x5d, 0xcd, 0x8b, 0xb7, 0xbc, 0x24, + 0xde, 0x0e, 0xd4, 0x8f, 0x03, 0xf7, 0xd4, 0xf5, 0x84, 0xaf, 0x15, 0x6e, 0x55, 0x25, 0xd9, 0xdf, + 0xd6, 0xa0, 0x42, 0xd9, 0x1f, 0xe7, 0x6c, 0x16, 0x8a, 0xec, 0xf3, 0xc7, 0xc3, 0x7d, 0x19, 0x57, + 0x4c, 0x20, 0x0f, 0x95, 0xf0, 0x79, 0x6c, 0xf5, 0xbd, 0x76, 0x5c, 0x36, 0x9c, 0x2c, 0xb3, 0xa6, + 0xa4, 0xe9, 0x1e, 0x94, 0x8f, 0x58, 0x78, 0xe6, 0x8f, 0x64, 0x09, 0x35, 0xc4, 0x0d, 0x41, 0xa3, + 0x92, 0x47, 0x3e, 0x86, 0x12, 0x5e, 0xe1, 0xd1, 0xd7, 0xa3, 0xd2, 0x45, 0x8a, 0xb4, 0x2e, 0xf5, + 0x72, 0x21, 0xf2, 0x00, 0xca, 0x6f, 0x3c, 0x94, 0xe0, 0x69, 0xa9, 0xef, 0xad, 0x0b, 0x71, 0x41, + 0xd3, 0x2f, 0x48, 0x41, 0xf2, 0x25, 0xc0, 0x01, 0x0b, 0x07, 0x57, 0x1e, 0xb7, 0x52, 0xe6, 0xd7, + 0x6e, 0xcb, 0x0f, 0x64, 0x41, 0xd7, 0xaf, 0x2a, 0x17, 0xc8, 0x21, 0xb4, 0x0e, 0x58, 0x88, 0x25, + 0xe8, 0x7a, 0xa7, 0xaf, 0xdc, 0x59, 0xc8, 0x13, 0x59, 0xdf, 0xfb, 0xd9, 0x42, 0x85, 0xc2, 0xd3, + 0xd5, 0x24, 0x2e, 0x92, 0x5f, 0x41, 0xe5, 0x80, 0x85, 0xcf, 0x5c, 0x6f, 0xc4, 0x6b, 0x15, 0xab, + 0x2f, 0xd2, 0x81, 0x44, 0xfd, 0x72, 0x24, 0x4a, 0x28, 0xac, 0xbd, 0x64, 0x6c, 0x1a, 0xe7, 0x19, + 0xef, 0x8b, 0xea, 0xdd, 0x16, 0xf7, 0x53, 0x6c, 0x5d, 0x53, 0xfa, 0x3a, 0x06, 0x85, 0x44, 0xca, + 0x26, 0x7e, 0xc8, 0x78, 0x5e, 0x40, 0x0d, 0x4a, 0xe7, 0x25, 0x82, 0xd2, 0x99, 0xe4, 0x15, 0xb4, + 0x79, 0xc1, 0x3a, 0x21, 0x93, 0xc5, 0x6e, 0xd5, 0xb9, 0xae, 0x0f, 0x84, 0xae, 0x04, 0x53, 0x57, + 0x96, 0xbc, 0x4a, 0x7a, 0xd0, 0xe8, 0x39, 0x9e, 0xe7, 0x87, 0x3d, 0x7f, 0x32, 0x71, 0x43, 0xab, + 0xc1, 0x55, 0x6d, 0x09, 0x55, 0x2a, 0x47, 0xd7, 0xa3, 0x5d, 0x42, 0x25, 0x07, 0x2c, 0x7c, 0x3a, + 0x0c, 0xdd, 0x0b, 0x36, 0xb8, 0xf2, 0xac, 0xa6, 0xaa, 0x44, 0xe5, 0x24, 0x94, 0xa8, 0x2c, 0x4c, + 0x7b, 0x9f, 0x85, 0x14, 0x11, 0x21, 0x08, 0xa3, 0xc8, 0x5a, 0x6a, 0xda, 0x53, 0xec, 0x44, 0xda, + 0x53, 0x7c, 0xd4, 0xd9, 0x73, 0xbc, 0x84, 0xce, 0xb6, 0xaa, 0x33, 0xc5, 0x4e, 0xe8, 0x4c, 0xf1, + 0xc9, 0x1b, 0x20, 0x94, 0x4d, 0x1c, 0xd7, 0x1b, 0x5c, 0x79, 0x87, 0x5e, 0xa4, 0xd4, 0xe4, 0x4a, + 0xef, 0x0a, 0xa5, 0x69, 0xbe, 0xae, 0x35, 0x43, 0x01, 0xf9, 0x2d, 0xd4, 0x7b, 0x67, 0x6c, 0x78, + 0x7e, 0x1c, 0x4c, 0xcf, 0x1c, 0xcf, 0x5a, 0xe3, 0xfa, 0x2c, 0xe9, 0x64, 0xcc, 0xd0, 0x15, 0xa9, + 0x57, 0xb0, 0x30, 0x28, 0x9b, 0xcd, 0x27, 0xec, 0xd0, 0xbb, 0xc0, 0xb7, 0xdc, 0x7b, 0x6d, 0x11, + 0xb5, 0x30, 0x12, 0xcc, 0x44, 0x61, 0x24, 0xb8, 0xf6, 0x3f, 0x6a, 0x50, 0xa5, 0x6c, 0x36, 0xf5, + 0xbd, 0x19, 0x5b, 0x82, 0x55, 0x31, 0xec, 0x14, 0x6e, 0x80, 0x9d, 0x0d, 0x58, 0x7d, 0x1e, 0x04, + 0x7e, 0xc0, 0xb1, 0xa9, 0x41, 0xc5, 0x81, 0x7c, 0x04, 0x95, 0xd7, 0xec, 0x92, 0x7f, 0x62, 0xa5, + 0x4c, 0x94, 0xa3, 0x11, 0x9f, 0xfc, 0x52, 0xe2, 0x96, 0x00, 0x22, 0xa2, 0xe2, 0x96, 0x70, 0x53, + 0x03, 0xae, 0xbd, 0x05, 0x70, 0x95, 0xd5, 0x4f, 0x3f, 0x02, 0x2e, 0xed, 0x46, 0x84, 0x5c, 0x4f, + 0x34, 0xe4, 0xaa, 0xa8, 0xaf, 0x40, 0x45, 0x2e, 0xed, 0xae, 0x0a, 0x5d, 0x5f, 0xa7, 0xa0, 0xab, + 0xaa, 0xbe, 0x80, 0x24, 0x74, 0x69, 0x7a, 0x92, 0xd8, 0xf5, 0x28, 0xc6, 0x2e, 0x81, 0x3d, 0xb7, + 0x12, 0xd8, 0xa5, 0xdd, 0x5e, 0x80, 0x57, 0x3f, 0x0b, 0xbc, 0x40, 0x2d, 0xce, 0x0c, 0xf0, 0xd2, + 0x54, 0x65, 0xa0, 0xd7, 0xd7, 0x29, 0xf4, 0xd2, 0x10, 0x27, 0x89, 0x5e, 0x7a, 0x5c, 0x09, 0xf8, + 0x3a, 0x4a, 0xc3, 0x97, 0xc0, 0x9c, 0x9f, 0xe7, 0xc0, 0x97, 0xa6, 0x2d, 0x85, 0x5f, 0xfb, 0x09, + 0xfc, 0x12, 0xd0, 0x73, 0x27, 0x0b, 0xbf, 0x34, 0x45, 0x3a, 0x80, 0xed, 0x27, 0x00, 0xac, 0xa5, + 0x6a, 0xd1, 0x01, 0x4c, 0xd7, 0xa2, 0x21, 0x58, 0x3f, 0x0b, 0xc1, 0xda, 0x6a, 0xee, 0x33, 0x10, + 0x4c, 0xcf, 0x7d, 0x1a, 0xc2, 0xfa, 0x59, 0x10, 0xa6, 0xa1, 0x4d, 0x06, 0x84, 0xe9, 0x4a, 0xd3, + 0x18, 0xf6, 0x36, 0x13, 0xc3, 0x04, 0xe6, 0x74, 0xf2, 0x31, 0x4c, 0x53, 0x9b, 0x05, 0x62, 0x4f, + 0x75, 0x10, 0x23, 0x5a, 0x33, 0x51, 0x41, 0x4c, 0xd3, 0xa4, 0xa1, 0xd8, 0x51, 0x1a, 0xc5, 0xd6, + 0xd5, 0xfa, 0x48, 0xa1, 0x98, 0x5e, 0x1f, 0x49, 0x18, 0xfb, 0xb3, 0x21, 0xc6, 0xf7, 0x68, 0xea, + 0xc2, 0x49, 0xf2, 0xca, 0x93, 0x28, 0xd6, 0xa0, 0xe2, 0xb0, 0x64, 0x92, 0x24, 0x50, 0xa2, 0xfe, + 0xe5, 0xcc, 0x2a, 0x76, 0x8a, 0xdd, 0x06, 0xe5, 0xcf, 0xe4, 0x01, 0x54, 0xe4, 0x46, 0x90, 0x9e, + 0xa3, 0x24, 0x23, 0xfa, 0x34, 0xe5, 0xd1, 0x7e, 0x0c, 0x0d, 0xf5, 0xfb, 0x20, 0x3b, 0x50, 0x46, + 0x6f, 0xc7, 0x21, 0xf7, 0xa5, 0x1e, 0xc1, 0xa6, 0xa0, 0x45, 0xc8, 0x24, 0x4e, 0xf6, 0x17, 0xb0, + 0x96, 0x9a, 0x9d, 0x72, 0x62, 0x31, 0xa1, 0x48, 0xfd, 0x4b, 0x1e, 0x45, 0x83, 0xe2, 0xa3, 0xed, + 0x00, 0x49, 0xc3, 0x97, 0x9c, 0x82, 0xe7, 0x62, 0xa6, 0x5e, 0xa5, 0xe2, 0x40, 0x1e, 0x41, 0x5d, + 0xc5, 0xaf, 0x42, 0xa7, 0xd8, 0xad, 0xef, 0x35, 0xe3, 0x55, 0x64, 0x70, 0xe5, 0x45, 0x6f, 0x4d, + 0x91, 0xb3, 0x9f, 0xc0, 0xad, 0xcc, 0xc1, 0x8c, 0x7c, 0x08, 0x45, 0xfc, 0xa0, 0x44, 0x84, 0x99, + 0x7a, 0x90, 0x6f, 0x1f, 0xc3, 0x66, 0x36, 0x3a, 0x26, 0x1d, 0x32, 0x7e, 0xa0, 0x43, 0x5f, 0x42, + 0x45, 0x72, 0xf3, 0x5f, 0x79, 0x2f, 0x60, 0x4e, 0xc8, 0x46, 0xc7, 0x5e, 0xf4, 0xca, 0x17, 0x04, + 0xfb, 0x2f, 0x06, 0x34, 0xb5, 0x19, 0x37, 0x47, 0xcb, 0xa7, 0x50, 0x15, 0x10, 0x32, 0xe8, 0xcb, + 0x29, 0xfd, 0xa6, 0x05, 0x67, 0x21, 0x4b, 0x3e, 0x83, 0xda, 0xd1, 0x3c, 0x74, 0x44, 0x01, 0x15, + 0x79, 0x4c, 0x72, 0xb2, 0x7e, 0x7e, 0x15, 0x06, 0x4e, 0xc4, 0x8b, 0x46, 0xfc, 0x85, 0xac, 0x6d, + 0x42, 0x4b, 0x6f, 0x61, 0xf6, 0xb7, 0x06, 0xef, 0x3a, 0xca, 0x18, 0xaa, 0x97, 0xb3, 0x91, 0x2c, + 0xe7, 0xc5, 0x32, 0x55, 0x50, 0x97, 0xa9, 0xc5, 0xfa, 0x53, 0xcc, 0x5b, 0x7f, 0x4a, 0x3f, 0x6e, + 0xfd, 0x59, 0x4d, 0xaf, 0x3f, 0x2f, 0xa0, 0x9d, 0x68, 0x5f, 0x3f, 0x69, 0xcf, 0xb1, 0xff, 0x66, + 0x80, 0x95, 0x37, 0x83, 0x2f, 0x09, 0xfe, 0x1e, 0x94, 0xfb, 0xa1, 0x13, 0xce, 0x67, 0xfa, 0xac, + 0x22, 0x68, 0x54, 0xf2, 0xc8, 0x26, 0x94, 0xf9, 0xfb, 0x8d, 0xbe, 0x79, 0x79, 0x22, 0x8f, 0x00, + 0x16, 0x36, 0xf1, 0xc3, 0x2f, 0xe6, 0xbb, 0xab, 0x08, 0xda, 0xbf, 0x83, 0xad, 0xdc, 0xae, 0x4b, + 0x5a, 0x50, 0x38, 0x7e, 0xc9, 0x1d, 0xad, 0xd2, 0xc2, 0xf1, 0xcb, 0x1f, 0xe6, 0xa1, 0xfd, 0x39, + 0x58, 0x79, 0xe3, 0xf0, 0xcd, 0x19, 0xb0, 0x3f, 0x86, 0xad, 0xdc, 0x36, 0x94, 0x74, 0x06, 0xcd, + 0xe4, 0x4d, 0xc8, 0xcb, 0xcd, 0xe4, 0x36, 0xa6, 0x94, 0x99, 0x5f, 0xc3, 0x56, 0xee, 0xcc, 0xbc, + 0xc4, 0xce, 0x63, 0xb8, 0x93, 0xdf, 0xaa, 0xc4, 0xe0, 0x2a, 0xb9, 0x12, 0xe8, 0x62, 0x82, 0xfd, + 0x08, 0x6e, 0x65, 0x6e, 0x5e, 0x4b, 0x4c, 0x76, 0x61, 0x33, 0x7b, 0xe4, 0x49, 0xc5, 0xf5, 0x29, + 0x6c, 0x66, 0xaf, 0x63, 0x4b, 0x2c, 0x7c, 0x04, 0xb7, 0x73, 0xe6, 0xa0, 0x94, 0x09, 0x0a, 0xeb, + 0x19, 0x6b, 0x1a, 0xf9, 0x02, 0x9a, 0xa2, 0xa1, 0x22, 0xec, 0xc7, 0xc0, 0x29, 0x8b, 0x75, 0xc1, + 0x92, 0xc5, 0xaa, 0xcb, 0xda, 0xbf, 0x81, 0x8d, 0xac, 0xd1, 0x89, 0xdc, 0x83, 0xa6, 0xa0, 0x20, + 0xcc, 0x8a, 0x8c, 0xe2, 0xd7, 0xa1, 0x13, 0xed, 0x87, 0xb0, 0x9e, 0xb1, 0xf3, 0x2d, 0x89, 0xf8, + 0x09, 0x6c, 0x64, 0xcd, 0x59, 0xf1, 0x6f, 0x35, 0x86, 0xfa, 0x5b, 0x8d, 0x29, 0xba, 0x4a, 0x81, + 0x9b, 0xe7, 0x0d, 0x64, 0x1f, 0x48, 0x7a, 0x4b, 0x5a, 0x82, 0x05, 0x0b, 0x2d, 0x46, 0xa4, 0xe5, + 0x13, 0x58, 0xcf, 0x18, 0x53, 0x10, 0x0e, 0xe4, 0x44, 0x23, 0xbc, 0x90, 0x27, 0xfb, 0x33, 0xa8, + 0x2d, 0x12, 0x47, 0x2c, 0xa8, 0x44, 0x83, 0x94, 0xb0, 0x14, 0x1d, 0x33, 0xbc, 0xfd, 0x53, 0x31, + 0xea, 0xfd, 0xe4, 0x01, 0x54, 0xb1, 0x84, 0x78, 0x1b, 0x32, 0x6e, 0xc2, 0xbf, 0x85, 0x18, 0x02, + 0xed, 0x57, 0xce, 0xac, 0xe7, 0x7b, 0xef, 0xc6, 0xee, 0x30, 0xe4, 0xfe, 0x57, 0xa9, 0x4a, 0xc2, + 0x17, 0xf5, 0x95, 0x33, 0xfb, 0x26, 0x60, 0x17, 0x72, 0x2c, 0x2e, 0x72, 0x19, 0x9d, 0x48, 0x3e, + 0x87, 0xda, 0xa2, 0x43, 0xc9, 0x29, 0xe6, 0xa6, 0xee, 0x15, 0x0b, 0xff, 0x88, 0xdf, 0x18, 0x3b, + 0x50, 0x8f, 0xbc, 0x7a, 0xc9, 0xae, 0xf9, 0x2e, 0xd6, 0xa0, 0x2a, 0x49, 0x95, 0xc0, 0x2c, 0x55, + 0x74, 0x09, 0xcc, 0xec, 0x36, 0x00, 0x7a, 0x8d, 0xfd, 0x9c, 0x05, 0x7c, 0xa5, 0x6a, 0x50, 0x85, + 0x82, 0x99, 0x17, 0x4f, 0xe2, 0x47, 0xc6, 0x26, 0x8d, 0x8e, 0x78, 0xf3, 0x35, 0xbb, 0xc4, 0xc4, + 0x3d, 0x1d, 0x89, 0x35, 0xa8, 0x4a, 0x15, 0x8a, 0xdd, 0x87, 0xa6, 0xd6, 0x6f, 0xf1, 0x55, 0x9d, + 0xb3, 0x6b, 0xd9, 0xe3, 0xf1, 0x11, 0x87, 0xbf, 0xd9, 0xb9, 0x3b, 0x95, 0x59, 0xe6, 0xcf, 0x58, + 0x56, 0x01, 0x9b, 0x8e, 0x9d, 0x21, 0x1b, 0xf8, 0x72, 0x9d, 0x8d, 0x09, 0xf8, 0xd1, 0x67, 0xaf, + 0xda, 0x4b, 0x3e, 0x81, 0x2d, 0xb8, 0x9d, 0x33, 0xdc, 0xee, 0xfc, 0x42, 0xfb, 0xd1, 0x99, 0x54, + 0xf8, 0x88, 0x67, 0xae, 0x90, 0x1a, 0xac, 0x52, 0xcc, 0xb3, 0x69, 0xec, 0x7c, 0x28, 0xea, 0x88, + 0xff, 0x94, 0xdc, 0x84, 0xda, 0xf3, 0xab, 0xe1, 0x78, 0x3e, 0x73, 0x2f, 0x98, 0xb9, 0x42, 0x00, + 0xca, 0xd8, 0xa4, 0xd9, 0xc8, 0x34, 0x76, 0xee, 0x01, 0xc4, 0xbf, 0x28, 0x93, 0x2a, 0x94, 0xf0, + 0x64, 0xae, 0x90, 0x06, 0x54, 0x5f, 0x38, 0xb3, 0xf0, 0x85, 0xe3, 0x8e, 0x4d, 0x63, 0xe7, 0x6e, + 0xdc, 0xf6, 0x51, 0xe6, 0xb5, 0xef, 0x31, 0x61, 0xed, 0xd9, 0x35, 0x1a, 0x36, 0x76, 0xfe, 0x5e, + 0x88, 0x76, 0x7e, 0xe4, 0xa3, 0x61, 0x61, 0x47, 0xcc, 0x26, 0xa6, 0x41, 0x5a, 0xea, 0x2a, 0x6d, + 0x16, 0x08, 0x49, 0xae, 0xc6, 0x66, 0x11, 0x69, 0x3a, 0x8e, 0x9a, 0x25, 0x52, 0x5f, 0xac, 0xbd, + 0xe6, 0x2a, 0xb9, 0x95, 0xb1, 0xcc, 0x9a, 0x65, 0xd2, 0x86, 0xba, 0xfc, 0xb9, 0x9b, 0x5f, 0xaa, + 0x90, 0x35, 0x68, 0x4a, 0x82, 0xb4, 0x5f, 0x25, 0xeb, 0xa9, 0x35, 0xd3, 0xac, 0x11, 0x53, 0x5f, + 0x16, 0x4d, 0x40, 0x8a, 0x0a, 0x3b, 0x66, 0x1d, 0x6d, 0xa6, 0xda, 0xa3, 0xd9, 0x20, 0x9b, 0x59, + 0x1b, 0x93, 0xd9, 0x44, 0xf1, 0x54, 0x9b, 0x33, 0x5b, 0xe8, 0xa2, 0x02, 0x24, 0x66, 0x1b, 0xfd, + 0x49, 0xbc, 0x5c, 0xd3, 0xdc, 0x61, 0x51, 0xab, 0x17, 0x56, 0xf9, 0x65, 0x0c, 0xe9, 0xb9, 0x87, + 0xd1, 0x9a, 0x2b, 0x68, 0x55, 0x21, 0xcb, 0xec, 0x99, 0x86, 0x22, 0xfe, 0x86, 0x27, 0xb8, 0x3f, + 0x1f, 0x0e, 0xcd, 0x82, 0x42, 0x8e, 0x7d, 0x32, 0x8b, 0xcf, 0x7a, 0xdf, 0xff, 0x67, 0xdb, 0xf8, + 0xee, 0xfd, 0xb6, 0xf1, 0xfd, 0xfb, 0x6d, 0xe3, 0xdf, 0xef, 0xb7, 0x57, 0xfe, 0xfa, 0xdf, 0x6d, + 0xe3, 0x0f, 0xea, 0x1f, 0x23, 0x13, 0x27, 0x0c, 0xdc, 0x2b, 0x9f, 0x8f, 0x6a, 0xd1, 0xc1, 0x63, + 0xbb, 0xd3, 0xf3, 0xd3, 0xdd, 0xe9, 0xc9, 0x2e, 0xa6, 0xf4, 0xa4, 0xcc, 0xff, 0x0e, 0x79, 0xf8, + 0xff, 0x00, 0x00, 0x00, 0xff, 0xff, 0xb2, 0x86, 0xcb, 0x66, 0x62, 0x19, 0x00, 0x00, } func (m *LockOptions) Marshal() (dAtA []byte, err error) { @@ -2950,6 +3061,18 @@ func (m *Request) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + { + size, err := m.ResumeInvalidCN.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintLock(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x92 { size, err := m.CheckOrphan.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -3141,6 +3264,18 @@ func (m *Response) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + { + size, err := m.ResumeInvalidCN.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintLock(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x9a { size, err := m.CheckOrphan.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -4694,6 +4829,67 @@ func (m *ExtraMutation) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *ResumeInvalidCNRequest) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ResumeInvalidCNRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ResumeInvalidCNRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.ServiceID) > 0 { + i -= len(m.ServiceID) + copy(dAtA[i:], m.ServiceID) + i = encodeVarintLock(dAtA, i, uint64(len(m.ServiceID))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ResumeInvalidCNResponse) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ResumeInvalidCNResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ResumeInvalidCNResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + return len(dAtA) - i, nil +} + func encodeVarintLock(dAtA []byte, offset int, v uint64) int { offset -= sovLock(v) base := offset @@ -4820,6 +5016,8 @@ func (m *Request) ProtoSize() (n int) { n += 2 + l + sovLock(uint64(l)) l = m.CheckOrphan.ProtoSize() n += 2 + l + sovLock(uint64(l)) + l = m.ResumeInvalidCN.ProtoSize() + n += 2 + l + sovLock(uint64(l)) if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -4874,6 +5072,8 @@ func (m *Response) ProtoSize() (n int) { n += 2 + l + sovLock(uint64(l)) l = m.CheckOrphan.ProtoSize() n += 2 + l + sovLock(uint64(l)) + l = m.ResumeInvalidCN.ProtoSize() + n += 2 + l + sovLock(uint64(l)) if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -5493,6 +5693,34 @@ func (m *ExtraMutation) ProtoSize() (n int) { return n } +func (m *ResumeInvalidCNRequest) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ServiceID) + if l > 0 { + n += 1 + l + sovLock(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *ResumeInvalidCNResponse) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + func sovLock(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -6509,6 +6737,39 @@ func (m *Request) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 18: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResumeInvalidCN", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLock + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLock + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLock + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ResumeInvalidCN.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLock(dAtA[iNdEx:]) @@ -7130,6 +7391,39 @@ func (m *Response) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 19: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResumeInvalidCN", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLock + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLock + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLock + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ResumeInvalidCN.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLock(dAtA[iNdEx:]) @@ -10514,6 +10808,140 @@ func (m *ExtraMutation) Unmarshal(dAtA []byte) error { } return nil } +func (m *ResumeInvalidCNRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLock + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ResumeInvalidCNRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ResumeInvalidCNRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ServiceID", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLock + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthLock + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthLock + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ServiceID = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipLock(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthLock + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ResumeInvalidCNResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLock + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ResumeInvalidCNResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ResumeInvalidCNResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipLock(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthLock + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipLock(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/pkg/pb/txn/txn.pb.go b/pkg/pb/txn/txn.pb.go index 5f7c85e296ef..32c6be6aab04 100644 --- a/pkg/pb/txn/txn.pb.go +++ b/pkg/pb/txn/txn.pb.go @@ -388,6 +388,7 @@ type CNTxnSnapshot struct { Disable1PCOpt bool `protobuf:"varint,4,opt,name=Disable1PCOpt,proto3" json:"Disable1PCOpt,omitempty"` LockTables []lock.LockTable `protobuf:"bytes,5,rep,name=LockTables,proto3" json:"LockTables"` Options TxnOptions `protobuf:"bytes,6,opt,name=Options,proto3" json:"Options"` + Flag uint32 `protobuf:"varint,7,opt,name=Flag,proto3" json:"Flag,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -468,6 +469,13 @@ func (m *CNTxnSnapshot) GetOptions() TxnOptions { return TxnOptions{} } +func (m *CNTxnSnapshot) GetFlag() uint32 { + if m != nil { + return m.Flag + } + return 0 +} + // CNOpRequest cn read/write request, CN -> TN. If data is written to more than one TN (>1) in a // single transaction, then the transaction becomes a 2pc transaction. type CNOpRequest struct { @@ -1883,106 +1891,106 @@ func init() { func init() { proto.RegisterFile("txn.proto", fileDescriptor_4f782e76b37adb9a) } var fileDescriptor_4f782e76b37adb9a = []byte{ - // 1572 bytes of a gzipped FileDescriptorProto + // 1577 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x58, 0xcb, 0x6e, 0xdb, 0x46, 0x17, 0x36, 0x75, 0x25, 0x8f, 0x2e, 0xa6, 0x26, 0x8e, 0xc3, 0xf8, 0xcf, 0xaf, 0x08, 0x44, 0x10, 0x28, 0x46, 0x7e, 0xeb, 0x4f, 0x82, 0x74, 0xd1, 0x02, 0x01, 0x6c, 0xd9, 0x4e, 0x05, 0x44, 0xb2, 0x31, 0x52, 0x5a, 0xa4, 0x9b, 0x82, 0x92, 0x26, 0x32, 0x61, 0x89, 0x54, 0xc8, 0x91, 0x21, 0x3f, - 0x44, 0x9f, 0xa0, 0x9b, 0x3e, 0x43, 0x9f, 0x22, 0xcb, 0x3c, 0x41, 0xd1, 0xa6, 0xe8, 0xa6, 0xdb, - 0xbe, 0x40, 0x31, 0xc3, 0x19, 0xde, 0x24, 0x25, 0x85, 0xbb, 0x12, 0xcf, 0xed, 0x3b, 0xc3, 0x33, - 0xe7, 0x3b, 0xc3, 0x11, 0x68, 0x74, 0xe9, 0x1c, 0xcc, 0x3d, 0x97, 0xba, 0x28, 0x4b, 0x97, 0xce, - 0xde, 0xff, 0x26, 0x36, 0xbd, 0x58, 0x0c, 0x0f, 0x46, 0xee, 0xac, 0x35, 0x71, 0x27, 0x6e, 0x8b, - 0xdb, 0x86, 0x8b, 0xb7, 0x5c, 0xe2, 0x02, 0x7f, 0x0a, 0x62, 0xf6, 0xb6, 0xa9, 0x3d, 0x23, 0x3e, - 0xb5, 0x66, 0x73, 0xa1, 0xa8, 0xce, 0x08, 0xb5, 0xc6, 0x16, 0xb5, 0x84, 0x0c, 0x53, 0x77, 0x74, - 0x19, 0x3c, 0x9b, 0x7f, 0x66, 0xa1, 0x38, 0x58, 0x3a, 0x5d, 0x42, 0x2d, 0x54, 0x85, 0x4c, 0xe7, - 0xd8, 0x50, 0x1a, 0x4a, 0xb3, 0x8c, 0x33, 0x9d, 0x63, 0xf4, 0x10, 0x0a, 0x7d, 0x6a, 0xd1, 0x85, - 0x6f, 0x64, 0x1a, 0x4a, 0xb3, 0xfa, 0xb4, 0x7a, 0xc0, 0x16, 0x36, 0x58, 0x3a, 0x81, 0x16, 0x0b, - 0x2b, 0xfa, 0x12, 0xa0, 0xef, 0x58, 0x73, 0xff, 0xc2, 0xa5, 0x83, 0xbe, 0x91, 0x6d, 0x28, 0xcd, - 0xd2, 0xd3, 0x9d, 0x83, 0x68, 0x15, 0x03, 0xf9, 0x74, 0x94, 0x7b, 0xff, 0xcb, 0xfd, 0x2d, 0x1c, - 0xf3, 0x66, 0xb1, 0xe7, 0x1e, 0x99, 0x5b, 0x1e, 0x19, 0x0f, 0xfa, 0x46, 0xee, 0xf3, 0xb1, 0x91, - 0x37, 0xfa, 0x02, 0xd4, 0xb6, 0x3b, 0x9b, 0xd9, 0x2c, 0x6b, 0xfe, 0xb3, 0x91, 0xa1, 0x2f, 0x7a, - 0x06, 0xea, 0xa0, 0xd7, 0xbf, 0xb0, 0xbc, 0xb1, 0x6f, 0x14, 0x1a, 0xd9, 0x66, 0xe9, 0x69, 0xed, - 0x20, 0x2c, 0x91, 0xb0, 0xc8, 0x20, 0xe9, 0x88, 0x9e, 0x03, 0xbc, 0x72, 0x47, 0x97, 0x03, 0x6b, - 0x38, 0x25, 0xbe, 0x51, 0xe4, 0x61, 0xdb, 0x07, 0xbc, 0x92, 0xa1, 0x5e, 0xae, 0x31, 0x72, 0x44, - 0x0d, 0xc8, 0x75, 0xdd, 0x31, 0x31, 0x54, 0x5e, 0xc1, 0xb2, 0xac, 0x20, 0xd3, 0x61, 0x6e, 0x41, - 0x2d, 0xd0, 0x3a, 0xbe, 0x3b, 0xb5, 0xa8, 0xed, 0x3a, 0x86, 0xc6, 0xdd, 0x6a, 0xd2, 0x2d, 0x34, - 0xe0, 0xc8, 0x07, 0xed, 0x42, 0xa1, 0x6b, 0x7b, 0x9e, 0xeb, 0x19, 0xd0, 0x50, 0x9a, 0x2a, 0x16, - 0x12, 0x6a, 0x40, 0x89, 0x25, 0xee, 0x13, 0xef, 0xca, 0x1e, 0x11, 0xa3, 0xd4, 0x50, 0x9a, 0x1a, - 0x8e, 0xab, 0xcc, 0x1f, 0x32, 0x50, 0x69, 0xf7, 0xd8, 0x06, 0x8a, 0x0d, 0x40, 0x0f, 0x20, 0x3b, - 0x58, 0x3a, 0x7c, 0xcf, 0x4b, 0xb1, 0xd5, 0x11, 0x6a, 0x89, 0x77, 0x61, 0x66, 0x74, 0x0f, 0x34, - 0x4c, 0xac, 0xf1, 0xf5, 0x99, 0x33, 0xbd, 0xe6, 0xbd, 0xa0, 0xe2, 0x48, 0x81, 0xf6, 0x41, 0x3f, - 0x71, 0xd8, 0xdb, 0xb6, 0xad, 0xd1, 0x05, 0xf9, 0xd6, 0xb3, 0x29, 0xe1, 0x4d, 0xa0, 0xe2, 0x15, - 0x3d, 0x7a, 0x00, 0x95, 0x63, 0xdb, 0x67, 0xca, 0x27, 0xe7, 0xed, 0xb3, 0x39, 0xe5, 0x3b, 0xae, - 0xe2, 0xa4, 0x32, 0x55, 0xeb, 0xfc, 0x3f, 0xad, 0x75, 0x0b, 0x8a, 0x67, 0x73, 0x56, 0x22, 0xb6, - 0xad, 0x0a, 0x8f, 0x11, 0x2f, 0x24, 0xd4, 0x22, 0x46, 0x7a, 0x99, 0x73, 0x28, 0xb5, 0x7b, 0x67, - 0x73, 0x4c, 0xde, 0x2d, 0x88, 0x4f, 0x59, 0x61, 0xcf, 0xe6, 0x6d, 0xb6, 0x5b, 0xac, 0x1e, 0x15, - 0x2c, 0x24, 0x64, 0x40, 0xf1, 0xdc, 0xba, 0x9e, 0xba, 0xd6, 0x98, 0xbf, 0x7c, 0x19, 0x4b, 0x11, - 0xb5, 0xa0, 0x30, 0xb0, 0xbc, 0x09, 0xa1, 0xa2, 0xeb, 0x37, 0xf6, 0x91, 0x70, 0x33, 0x9b, 0x50, - 0x0e, 0x32, 0xfa, 0x73, 0xd7, 0xf1, 0x13, 0xd0, 0x4a, 0x02, 0xda, 0xfc, 0x23, 0x0f, 0x30, 0x58, - 0x3a, 0x72, 0x6d, 0x7c, 0x0b, 0xf8, 0xa3, 0xa0, 0x68, 0x0e, 0x47, 0x0a, 0xb9, 0x8d, 0x99, 0x4f, - 0x6f, 0xe3, 0x43, 0x28, 0x74, 0x09, 0xbd, 0x70, 0xc7, 0x7c, 0xb5, 0x31, 0x3e, 0x07, 0x5a, 0x2c, - 0xac, 0x08, 0x41, 0xee, 0x74, 0x6a, 0x4d, 0xf8, 0xde, 0x54, 0x30, 0x7f, 0x46, 0x07, 0xa0, 0xb5, - 0x7b, 0x22, 0xa1, 0x20, 0x9b, 0xce, 0xc3, 0x63, 0x05, 0xc4, 0x91, 0x0b, 0xfa, 0x0a, 0x2a, 0x01, - 0xdf, 0x64, 0x4c, 0xb0, 0x23, 0xb7, 0x65, 0xca, 0x84, 0x11, 0x27, 0x7d, 0xd1, 0x21, 0x6c, 0x63, - 0x77, 0x3a, 0x1d, 0x5a, 0xa3, 0x4b, 0x19, 0x5e, 0xe4, 0xe1, 0x77, 0x64, 0x78, 0xca, 0x8c, 0xd3, - 0xfe, 0xe8, 0x05, 0x54, 0xc5, 0xa4, 0x90, 0x08, 0x2a, 0x47, 0xd8, 0x95, 0x08, 0x49, 0x2b, 0x4e, - 0x79, 0xa3, 0x63, 0xd0, 0x5f, 0x12, 0x2a, 0x06, 0x9d, 0x40, 0xd0, 0x38, 0x82, 0x21, 0x11, 0xd2, - 0x76, 0xbc, 0x12, 0x81, 0xce, 0x61, 0x47, 0x4c, 0x9d, 0xa0, 0x1b, 0x24, 0x12, 0x70, 0xa4, 0x7b, - 0xc9, 0x62, 0x24, 0x7d, 0xf0, 0xda, 0x48, 0xf4, 0x0d, 0xec, 0xca, 0x57, 0x4d, 0x61, 0x96, 0x38, - 0x66, 0x3d, 0x5d, 0xa1, 0x14, 0xea, 0x86, 0x68, 0x74, 0x02, 0x55, 0x4c, 0x66, 0xee, 0x15, 0xe9, - 0x8a, 0x06, 0x36, 0xca, 0x1c, 0xef, 0xbf, 0x21, 0x5e, 0xc2, 0x1a, 0x96, 0x2d, 0xa9, 0x46, 0xff, - 0x8f, 0x28, 0x58, 0x49, 0xd6, 0x5b, 0x44, 0x08, 0x6b, 0xc4, 0xc1, 0x37, 0x50, 0x5b, 0xb1, 0xa2, - 0x3a, 0x00, 0x26, 0xd4, 0xbb, 0x66, 0xf4, 0xf3, 0x0d, 0xa5, 0x91, 0x6d, 0xe6, 0x71, 0x4c, 0xc3, - 0xc6, 0x08, 0x97, 0x3a, 0x0e, 0x25, 0xde, 0x95, 0x35, 0xe5, 0x9d, 0x9f, 0xc5, 0x49, 0xa5, 0xf9, - 0x57, 0x1e, 0x4a, 0x1c, 0x5b, 0x90, 0xed, 0xd3, 0x1c, 0xaa, 0x6f, 0xe4, 0xd0, 0xbf, 0x67, 0xcf, - 0x23, 0x50, 0x07, 0x4b, 0xe7, 0x84, 0x0f, 0xed, 0x80, 0x3c, 0x15, 0x19, 0xcd, 0x95, 0x38, 0x34, - 0xa3, 0xe7, 0xc9, 0x09, 0x21, 0x78, 0x53, 0x8b, 0x71, 0x2d, 0x30, 0xe0, 0xe4, 0x20, 0x79, 0x01, - 0x55, 0xc9, 0x21, 0x11, 0x58, 0x4c, 0xd6, 0x3f, 0x69, 0xc5, 0x29, 0x6f, 0xd6, 0xef, 0x11, 0x85, - 0x04, 0x82, 0x9a, 0xec, 0xf7, 0xb4, 0x1d, 0xaf, 0x44, 0x30, 0xe2, 0x86, 0x3c, 0x12, 0x20, 0x5a, - 0x92, 0xb8, 0x29, 0x33, 0x4e, 0xfb, 0xa3, 0x97, 0x50, 0x8b, 0xd1, 0x48, 0x80, 0x04, 0x7c, 0xb9, - 0xbb, 0x86, 0x79, 0x02, 0x66, 0x35, 0x06, 0xf5, 0xe1, 0x76, 0x8a, 0x41, 0x02, 0xac, 0x94, 0x6c, - 0xec, 0xb5, 0x4e, 0x78, 0x7d, 0x2c, 0x7a, 0x03, 0x77, 0x56, 0x08, 0x24, 0x60, 0x03, 0xbe, 0xdc, - 0xdf, 0xc8, 0x3f, 0x01, 0xbc, 0x29, 0x1e, 0x9d, 0xae, 0x30, 0xb0, 0x92, 0x62, 0x74, 0x8a, 0x81, - 0x72, 0x27, 0x93, 0x7a, 0x73, 0x04, 0x7a, 0x7a, 0xbe, 0xa2, 0x47, 0xf1, 0x63, 0x26, 0x1b, 0x3f, - 0x19, 0x25, 0x91, 0xc3, 0x23, 0x6d, 0xe5, 0x84, 0xce, 0xac, 0x39, 0xa1, 0xcd, 0x43, 0xce, 0xda, - 0x54, 0x0f, 0x3d, 0x86, 0x5a, 0xc7, 0xb9, 0xb2, 0xa6, 0xf6, 0x38, 0x76, 0x7a, 0xb3, 0x7c, 0x39, - 0xbc, 0x6a, 0x30, 0x77, 0x00, 0xad, 0x0e, 0x72, 0xf3, 0x36, 0xdc, 0x5a, 0xd3, 0x6a, 0xe6, 0x29, - 0xcf, 0x97, 0x9a, 0xd1, 0x4f, 0xa0, 0x28, 0x8a, 0x28, 0x3e, 0x60, 0x36, 0x1e, 0xbf, 0xd2, 0x4f, - 0x24, 0x4d, 0xf5, 0x9c, 0xf9, 0x35, 0x4f, 0xba, 0x32, 0xbd, 0x6f, 0x80, 0xbf, 0x0b, 0x3b, 0xeb, - 0xfa, 0xd3, 0x7c, 0x05, 0x77, 0x36, 0xcc, 0xf9, 0x9b, 0x64, 0xd9, 0x03, 0x63, 0x53, 0xe3, 0x9a, - 0x3d, 0xb8, 0xbb, 0x71, 0xfa, 0xdf, 0x24, 0xd7, 0x3d, 0xd8, 0xdb, 0xdc, 0xcd, 0x66, 0x97, 0xaf, - 0x64, 0xed, 0xd9, 0x70, 0x93, 0x64, 0xff, 0x09, 0x16, 0xbf, 0xb6, 0xd1, 0xcd, 0x41, 0x34, 0x44, - 0xd9, 0x90, 0x8d, 0x7d, 0xa8, 0xf1, 0x67, 0xb4, 0x03, 0xf9, 0x60, 0xc2, 0x06, 0x1f, 0x69, 0x81, - 0xc0, 0x8e, 0x92, 0x20, 0x8a, 0xfb, 0x67, 0xb9, 0x7f, 0x4c, 0x63, 0xfe, 0x9c, 0xe3, 0x0e, 0xf2, - 0xe4, 0xd9, 0x03, 0xf5, 0x94, 0x58, 0x74, 0xe1, 0xf1, 0xd6, 0x65, 0xce, 0xa1, 0xcc, 0xee, 0x47, - 0xed, 0x1e, 0x47, 0xd7, 0x70, 0xa6, 0xdd, 0x63, 0xe7, 0x49, 0x9f, 0xf8, 0xbe, 0xed, 0x3a, 0x9d, - 0x63, 0x8e, 0xac, 0xe1, 0x48, 0xc1, 0xac, 0x87, 0xa3, 0x91, 0xbb, 0x70, 0xd8, 0x69, 0x13, 0x1c, - 0x06, 0x91, 0x02, 0x99, 0x50, 0x6e, 0xbb, 0x8e, 0x43, 0x46, 0x34, 0x08, 0xcf, 0x73, 0x87, 0x84, - 0x8e, 0xad, 0xe5, 0xb5, 0x4f, 0xbc, 0x9e, 0x35, 0x0b, 0x8e, 0x01, 0x0d, 0x87, 0x32, 0x7a, 0x08, - 0xd5, 0xfe, 0xa5, 0x3d, 0x4f, 0x5d, 0x49, 0x72, 0x38, 0xa5, 0x45, 0x2f, 0x00, 0x25, 0x34, 0x5d, - 0x7e, 0xa2, 0xaa, 0x8d, 0x2c, 0x3f, 0xc1, 0xc2, 0x4f, 0x6a, 0x7e, 0x1f, 0x59, 0xe3, 0xc9, 0x3e, - 0x50, 0xf9, 0x92, 0x89, 0xc7, 0x27, 0xb9, 0x86, 0xa5, 0xc8, 0xae, 0x1b, 0xbe, 0x78, 0x59, 0xe7, - 0xad, 0xcb, 0x47, 0xb4, 0x86, 0xe3, 0x2a, 0xb6, 0x7e, 0xdb, 0xc1, 0x0b, 0xa7, 0xff, 0x6e, 0xca, - 0x87, 0xae, 0x8a, 0x43, 0x39, 0xb0, 0x05, 0x1d, 0xcc, 0x27, 0x27, 0xb7, 0x05, 0x32, 0xdb, 0x32, - 0x3b, 0xec, 0x38, 0x3e, 0x05, 0x55, 0x1c, 0xd3, 0xb0, 0x35, 0x0d, 0xaf, 0x8f, 0xc8, 0xc4, 0x76, - 0x8c, 0x2a, 0x37, 0x4a, 0x91, 0x45, 0x5a, 0x0b, 0xea, 0x8e, 0x02, 0xdc, 0xed, 0x20, 0x32, 0xd2, - 0x04, 0xc8, 0x1d, 0x67, 0xe4, 0xf5, 0xe9, 0x8c, 0x1a, 0xba, 0x44, 0x96, 0x1a, 0x56, 0xd5, 0x28, - 0x0f, 0xf7, 0xa9, 0x71, 0x9f, 0x94, 0x76, 0xbf, 0x0e, 0xe5, 0xf8, 0xed, 0x0c, 0x15, 0x20, 0xd3, - 0xef, 0xe8, 0x5b, 0xec, 0x17, 0xb7, 0x75, 0x65, 0x7f, 0x3f, 0xb8, 0x54, 0xb3, 0xae, 0xac, 0x02, - 0xb0, 0xde, 0x9a, 0xd9, 0x3e, 0xb5, 0x47, 0xfa, 0x16, 0xda, 0x86, 0xd2, 0x39, 0xab, 0x91, 0x50, - 0x28, 0xfb, 0xdf, 0x83, 0x16, 0x5e, 0xa9, 0x11, 0x40, 0xe1, 0x70, 0x44, 0xed, 0x2b, 0xa2, 0x6f, - 0xa1, 0x32, 0xa8, 0xf2, 0xb2, 0xab, 0x2b, 0x0c, 0x27, 0x28, 0x0f, 0xb5, 0x9d, 0x89, 0x9e, 0x41, - 0x15, 0xd0, 0x84, 0x4c, 0xc6, 0x7a, 0x96, 0x39, 0x1f, 0x0e, 0x5d, 0x8f, 0x1b, 0x73, 0xa8, 0x04, - 0x45, 0x2e, 0x91, 0xb1, 0x9e, 0xdf, 0xff, 0x51, 0xe1, 0x19, 0xc4, 0xe7, 0x89, 0x0a, 0x39, 0x76, - 0x75, 0xd3, 0xb7, 0x90, 0x06, 0x79, 0x7e, 0x29, 0xd3, 0x15, 0x96, 0x36, 0x00, 0xd3, 0x33, 0x0c, - 0x49, 0xbe, 0xab, 0x9e, 0x65, 0x48, 0x62, 0x11, 0x7a, 0x8e, 0xe5, 0x0c, 0x47, 0x9b, 0x9e, 0x47, - 0x35, 0xf9, 0x8d, 0x2f, 0xe8, 0xab, 0x17, 0xd0, 0xad, 0xe8, 0xcb, 0x5d, 0x2a, 0x8b, 0x48, 0x87, - 0xb2, 0xa4, 0x34, 0x23, 0xb4, 0xae, 0xb2, 0xd4, 0xc7, 0x27, 0x47, 0xaf, 0x5f, 0xea, 0xda, 0xd1, - 0xd1, 0x87, 0xdf, 0xea, 0xca, 0xfb, 0x8f, 0x75, 0xe5, 0xc3, 0xc7, 0xba, 0xf2, 0xeb, 0xc7, 0xfa, - 0xd6, 0x4f, 0xbf, 0xd7, 0x95, 0xef, 0x1e, 0xc7, 0xfe, 0xf2, 0x98, 0x59, 0xd4, 0xb3, 0x97, 0xae, - 0x67, 0x4f, 0x6c, 0x47, 0x0a, 0x0e, 0x69, 0xcd, 0x2f, 0x27, 0xad, 0xf9, 0xb0, 0x45, 0x97, 0xce, - 0xb0, 0xc0, 0xff, 0xcb, 0x78, 0xf6, 0x77, 0x00, 0x00, 0x00, 0xff, 0xff, 0xa1, 0x3c, 0x02, 0xc4, - 0x39, 0x11, 0x00, 0x00, + 0x4b, 0x37, 0xed, 0x2b, 0xf4, 0x29, 0xb2, 0xcc, 0x13, 0x14, 0x6d, 0x8a, 0x6e, 0xba, 0xed, 0x0b, + 0x14, 0x33, 0x9c, 0xe1, 0x4d, 0x52, 0x52, 0xb8, 0x2b, 0xf1, 0xdc, 0xbe, 0x33, 0x3c, 0x73, 0xbe, + 0x33, 0x1c, 0x81, 0x46, 0x97, 0xce, 0xc1, 0xdc, 0x73, 0xa9, 0x8b, 0xb2, 0x74, 0xe9, 0xec, 0xfd, + 0x6f, 0x62, 0xd3, 0x8b, 0xc5, 0xf0, 0x60, 0xe4, 0xce, 0x5a, 0x13, 0x77, 0xe2, 0xb6, 0xb8, 0x6d, + 0xb8, 0x78, 0xcb, 0x25, 0x2e, 0xf0, 0xa7, 0x20, 0x66, 0x6f, 0x9b, 0xda, 0x33, 0xe2, 0x53, 0x6b, + 0x36, 0x17, 0x8a, 0xea, 0x8c, 0x50, 0x6b, 0x6c, 0x51, 0x4b, 0xc8, 0x30, 0x75, 0x47, 0x97, 0xc1, + 0xb3, 0xf9, 0x67, 0x16, 0x8a, 0x83, 0xa5, 0xd3, 0x25, 0xd4, 0x42, 0x55, 0xc8, 0x74, 0x8e, 0x0d, + 0xa5, 0xa1, 0x34, 0xcb, 0x38, 0xd3, 0x39, 0x46, 0x0f, 0xa1, 0xd0, 0xa7, 0x16, 0x5d, 0xf8, 0x46, + 0xa6, 0xa1, 0x34, 0xab, 0x4f, 0xab, 0x07, 0x6c, 0x61, 0x83, 0xa5, 0x13, 0x68, 0xb1, 0xb0, 0xa2, + 0x2f, 0x01, 0xfa, 0x8e, 0x35, 0xf7, 0x2f, 0x5c, 0x3a, 0xe8, 0x1b, 0xd9, 0x86, 0xd2, 0x2c, 0x3d, + 0xdd, 0x39, 0x88, 0x56, 0x31, 0x90, 0x4f, 0x47, 0xb9, 0xf7, 0xbf, 0xdc, 0xdf, 0xc2, 0x31, 0x6f, + 0x16, 0x7b, 0xee, 0x91, 0xb9, 0xe5, 0x91, 0xf1, 0xa0, 0x6f, 0xe4, 0x3e, 0x1f, 0x1b, 0x79, 0xa3, + 0x2f, 0x40, 0x6d, 0xbb, 0xb3, 0x99, 0xcd, 0xb2, 0xe6, 0x3f, 0x1b, 0x19, 0xfa, 0xa2, 0x67, 0xa0, + 0x0e, 0x7a, 0xfd, 0x0b, 0xcb, 0x1b, 0xfb, 0x46, 0xa1, 0x91, 0x6d, 0x96, 0x9e, 0xd6, 0x0e, 0xc2, + 0x12, 0x09, 0x8b, 0x0c, 0x92, 0x8e, 0xe8, 0x39, 0xc0, 0x2b, 0x77, 0x74, 0x39, 0xb0, 0x86, 0x53, + 0xe2, 0x1b, 0x45, 0x1e, 0xb6, 0x7d, 0xc0, 0x2b, 0x19, 0xea, 0xe5, 0x1a, 0x23, 0x47, 0xd4, 0x80, + 0x5c, 0xd7, 0x1d, 0x13, 0x43, 0xe5, 0x15, 0x2c, 0xcb, 0x0a, 0x32, 0x1d, 0xe6, 0x16, 0xd4, 0x02, + 0xad, 0xe3, 0xbb, 0x53, 0x8b, 0xda, 0xae, 0x63, 0x68, 0xdc, 0xad, 0x26, 0xdd, 0x42, 0x03, 0x8e, + 0x7c, 0xd0, 0x2e, 0x14, 0xba, 0xb6, 0xe7, 0xb9, 0x9e, 0x01, 0x0d, 0xa5, 0xa9, 0x62, 0x21, 0xa1, + 0x06, 0x94, 0x58, 0xe2, 0x3e, 0xf1, 0xae, 0xec, 0x11, 0x31, 0x4a, 0x0d, 0xa5, 0xa9, 0xe1, 0xb8, + 0xca, 0xfc, 0x29, 0x03, 0x95, 0x76, 0x8f, 0x6d, 0xa0, 0xd8, 0x00, 0xf4, 0x00, 0xb2, 0x83, 0xa5, + 0xc3, 0xf7, 0xbc, 0x14, 0x5b, 0x1d, 0xa1, 0x96, 0x78, 0x17, 0x66, 0x46, 0xf7, 0x40, 0xc3, 0xc4, + 0x1a, 0x5f, 0x9f, 0x39, 0xd3, 0x6b, 0xde, 0x0b, 0x2a, 0x8e, 0x14, 0x68, 0x1f, 0xf4, 0x13, 0x87, + 0xbd, 0x6d, 0xdb, 0x1a, 0x5d, 0x90, 0x6f, 0x3d, 0x9b, 0x12, 0xde, 0x04, 0x2a, 0x5e, 0xd1, 0xa3, + 0x07, 0x50, 0x39, 0xb6, 0x7d, 0xa6, 0x7c, 0x72, 0xde, 0x3e, 0x9b, 0x53, 0xbe, 0xe3, 0x2a, 0x4e, + 0x2a, 0x53, 0xb5, 0xce, 0xff, 0xd3, 0x5a, 0xb7, 0xa0, 0x78, 0x36, 0x67, 0x25, 0x62, 0xdb, 0xaa, + 0xf0, 0x18, 0xf1, 0x42, 0x42, 0x2d, 0x62, 0xa4, 0x17, 0x42, 0x90, 0x3b, 0x9d, 0x5a, 0x13, 0xa3, + 0xd8, 0x50, 0x9a, 0x15, 0xcc, 0x9f, 0xcd, 0x39, 0x94, 0xda, 0xbd, 0xb3, 0x39, 0x26, 0xef, 0x16, + 0xc4, 0xa7, 0xac, 0xd8, 0x67, 0xf3, 0x36, 0xdb, 0x41, 0x85, 0x3b, 0x09, 0x09, 0x19, 0x50, 0x3c, + 0xb7, 0xae, 0xa7, 0xae, 0x35, 0xe6, 0x05, 0x29, 0x63, 0x29, 0xa2, 0x16, 0x14, 0x06, 0x96, 0x37, + 0x21, 0x54, 0x30, 0x61, 0x63, 0x6f, 0x09, 0x37, 0xb3, 0x09, 0xe5, 0x20, 0xa3, 0x3f, 0x77, 0x1d, + 0x3f, 0x01, 0xad, 0x24, 0xa0, 0xcd, 0x3f, 0xf2, 0x00, 0x83, 0xa5, 0x23, 0xd7, 0xc6, 0xb7, 0x85, + 0x3f, 0x0a, 0xda, 0xe6, 0x70, 0xa4, 0x90, 0x5b, 0x9b, 0xf9, 0xf4, 0xd6, 0x3e, 0x84, 0x42, 0x97, + 0xd0, 0x0b, 0x77, 0xcc, 0x57, 0x1b, 0xe3, 0x78, 0xa0, 0xc5, 0xc2, 0x1a, 0x96, 0x2a, 0x17, 0x95, + 0x0a, 0x1d, 0x80, 0xd6, 0xee, 0x89, 0x84, 0x82, 0x80, 0x3a, 0x0f, 0x8f, 0x15, 0x10, 0x47, 0x2e, + 0xe8, 0x2b, 0xa8, 0x04, 0x1c, 0x94, 0x31, 0xc1, 0x2e, 0xdd, 0x96, 0x29, 0x13, 0x46, 0x9c, 0xf4, + 0x45, 0x87, 0xb0, 0x8d, 0xdd, 0xe9, 0x74, 0x68, 0x8d, 0x2e, 0x65, 0x78, 0x91, 0x87, 0xdf, 0x91, + 0xe1, 0x29, 0x33, 0x4e, 0xfb, 0xa3, 0x17, 0x50, 0x15, 0xd3, 0x43, 0x22, 0xa8, 0x1c, 0x61, 0x57, + 0x22, 0x24, 0xad, 0x38, 0xe5, 0x8d, 0x8e, 0x41, 0x7f, 0x49, 0xa8, 0x18, 0x7e, 0x02, 0x41, 0xe3, + 0x08, 0x86, 0x44, 0x48, 0xdb, 0xf1, 0x4a, 0x04, 0x3a, 0x87, 0x1d, 0x31, 0x89, 0x82, 0x6e, 0x90, + 0x48, 0xc0, 0x91, 0xee, 0x25, 0x8b, 0x91, 0xf4, 0xc1, 0x6b, 0x23, 0xd1, 0x37, 0xb0, 0x2b, 0x5f, + 0x35, 0x85, 0x59, 0xe2, 0x98, 0xf5, 0x74, 0x85, 0x52, 0xa8, 0x1b, 0xa2, 0xd1, 0x09, 0x54, 0x31, + 0x99, 0xb9, 0x57, 0xa4, 0x2b, 0x1a, 0xd8, 0x28, 0x73, 0xbc, 0xff, 0x86, 0x78, 0x09, 0x6b, 0x58, + 0xb6, 0xa4, 0x1a, 0xfd, 0x3f, 0xa2, 0x65, 0x25, 0x59, 0x6f, 0x11, 0x21, 0xac, 0x21, 0x2f, 0xcd, + 0x37, 0x50, 0x5b, 0xb1, 0xa2, 0x3a, 0x00, 0x26, 0xd4, 0xbb, 0x66, 0xf4, 0xf3, 0x0d, 0xa5, 0x91, + 0x6d, 0xe6, 0x71, 0x4c, 0xc3, 0x46, 0x0b, 0x97, 0x3a, 0x0e, 0x25, 0xde, 0x95, 0x35, 0xe5, 0x9d, + 0x9f, 0xc5, 0x49, 0xa5, 0xf9, 0x57, 0x1e, 0x4a, 0x1c, 0x5b, 0x90, 0xed, 0xd3, 0x1c, 0xaa, 0x6f, + 0xe4, 0xd0, 0xbf, 0x67, 0xcf, 0x23, 0x50, 0x07, 0x4b, 0xe7, 0x84, 0x0f, 0xf2, 0x80, 0x3c, 0x15, + 0x19, 0xcd, 0x95, 0x38, 0x34, 0xa3, 0xe7, 0xc9, 0x09, 0x21, 0x78, 0x53, 0x8b, 0x71, 0x2d, 0x30, + 0xe0, 0xe4, 0x20, 0x79, 0x01, 0x55, 0xc9, 0x21, 0x11, 0x58, 0x4c, 0xd6, 0x3f, 0x69, 0xc5, 0x29, + 0x6f, 0xd6, 0xef, 0x11, 0x85, 0x04, 0x82, 0x9a, 0xec, 0xf7, 0xb4, 0x1d, 0xaf, 0x44, 0x30, 0xe2, + 0x86, 0x3c, 0x12, 0x20, 0x5a, 0x92, 0xb8, 0x29, 0x33, 0x4e, 0xfb, 0xa3, 0x97, 0x50, 0x8b, 0xd1, + 0x48, 0x80, 0x04, 0x7c, 0xb9, 0xbb, 0x86, 0x79, 0x02, 0x66, 0x35, 0x06, 0xf5, 0xe1, 0x76, 0x8a, + 0x41, 0x02, 0xac, 0x94, 0x6c, 0xec, 0xb5, 0x4e, 0x78, 0x7d, 0x2c, 0x7a, 0x03, 0x77, 0x56, 0x08, + 0x24, 0x60, 0x03, 0xbe, 0xdc, 0xdf, 0xc8, 0x3f, 0x01, 0xbc, 0x29, 0x1e, 0x9d, 0xae, 0x30, 0xb0, + 0x92, 0x62, 0x74, 0x8a, 0x81, 0x72, 0x27, 0x93, 0x7a, 0x73, 0x04, 0x7a, 0x7a, 0xbe, 0xa2, 0x47, + 0xf1, 0x63, 0x26, 0x1b, 0x3f, 0x2d, 0x25, 0x91, 0xc3, 0x23, 0x6d, 0xe5, 0xd4, 0xce, 0xac, 0x39, + 0xb5, 0xcd, 0x43, 0xce, 0xda, 0x54, 0x0f, 0x3d, 0x86, 0x5a, 0xc7, 0xb9, 0xb2, 0xa6, 0xf6, 0x38, + 0x76, 0xa2, 0xb3, 0x7c, 0x39, 0xbc, 0x6a, 0x30, 0x77, 0x00, 0xad, 0x0e, 0x72, 0xf3, 0x36, 0xdc, + 0x5a, 0xd3, 0x6a, 0xe6, 0x29, 0xcf, 0x97, 0x9a, 0xd1, 0x4f, 0xa0, 0x28, 0x8a, 0x28, 0x3e, 0x6a, + 0x36, 0x1e, 0xbf, 0xd2, 0x4f, 0x24, 0x4d, 0xf5, 0x9c, 0xf9, 0x35, 0x4f, 0xba, 0x32, 0xbd, 0x6f, + 0x80, 0xbf, 0x0b, 0x3b, 0xeb, 0xfa, 0xd3, 0x7c, 0x05, 0x77, 0x36, 0xcc, 0xf9, 0x9b, 0x64, 0xd9, + 0x03, 0x63, 0x53, 0xe3, 0x9a, 0x3d, 0xb8, 0xbb, 0x71, 0xfa, 0xdf, 0x24, 0xd7, 0x3d, 0xd8, 0xdb, + 0xdc, 0xcd, 0x66, 0x97, 0xaf, 0x64, 0xed, 0xd9, 0x70, 0x93, 0x64, 0xff, 0x09, 0x16, 0xbf, 0xb6, + 0xd1, 0xcd, 0x41, 0x34, 0x44, 0xd9, 0x90, 0x8d, 0x7d, 0xa8, 0xf1, 0x67, 0xb4, 0x03, 0xf9, 0x60, + 0xc2, 0x06, 0x1f, 0x69, 0x81, 0xc0, 0x8e, 0x92, 0x20, 0x8a, 0xfb, 0x67, 0xb9, 0x7f, 0x4c, 0x63, + 0xfe, 0x9c, 0xe3, 0x0e, 0xf2, 0xe4, 0xd9, 0x03, 0xf5, 0x94, 0x58, 0x74, 0xe1, 0xf1, 0xd6, 0x65, + 0xce, 0xa1, 0xcc, 0xee, 0x4c, 0xed, 0x1e, 0x47, 0xd7, 0x70, 0xa6, 0xdd, 0x63, 0xe7, 0x49, 0x9f, + 0xf8, 0xbe, 0xed, 0x3a, 0x9d, 0x63, 0x8e, 0xac, 0xe1, 0x48, 0xc1, 0xac, 0x87, 0xa3, 0x91, 0xbb, + 0x70, 0xd8, 0x69, 0x13, 0x1c, 0x06, 0x91, 0x02, 0x99, 0x50, 0x6e, 0xbb, 0x8e, 0x43, 0x46, 0x34, + 0x08, 0xcf, 0x73, 0x87, 0x84, 0x8e, 0xad, 0xe5, 0xb5, 0x4f, 0xbc, 0x9e, 0x35, 0x0b, 0x8e, 0x01, + 0x0d, 0x87, 0x32, 0x7a, 0x08, 0xd5, 0xfe, 0xa5, 0x3d, 0x4f, 0x5d, 0x53, 0x72, 0x38, 0xa5, 0x45, + 0x2f, 0x00, 0x25, 0x34, 0x5d, 0x7e, 0xa2, 0xaa, 0x8d, 0x2c, 0x3f, 0xc1, 0xc2, 0xcf, 0x6c, 0x7e, + 0x47, 0x59, 0xe3, 0xc9, 0x3e, 0x50, 0xf9, 0x92, 0x89, 0xc7, 0x27, 0xb9, 0x86, 0xa5, 0xc8, 0xae, + 0x20, 0xbe, 0x78, 0x59, 0xe7, 0xad, 0xcb, 0x47, 0xb4, 0x86, 0xe3, 0x2a, 0xb6, 0x7e, 0xdb, 0xc1, + 0x0b, 0xa7, 0xff, 0x6e, 0xca, 0x87, 0xae, 0x8a, 0x43, 0x39, 0xb0, 0x05, 0x1d, 0xcc, 0x27, 0x27, + 0xb7, 0x05, 0x32, 0xdb, 0x32, 0x3b, 0xec, 0x38, 0x3e, 0x05, 0x55, 0x1c, 0xd3, 0xb0, 0x35, 0x0d, + 0xaf, 0x8f, 0xc8, 0xc4, 0x76, 0x8c, 0x2a, 0x37, 0x4a, 0x91, 0x45, 0x5a, 0x0b, 0xea, 0x8e, 0x02, + 0xdc, 0xed, 0x20, 0x32, 0xd2, 0x04, 0xc8, 0x1d, 0x67, 0xe4, 0xf5, 0xe9, 0x8c, 0x1a, 0xba, 0x44, + 0x96, 0x1a, 0x56, 0xd5, 0x28, 0x0f, 0xf7, 0xa9, 0x71, 0x9f, 0x94, 0x76, 0xbf, 0x0e, 0xe5, 0xf8, + 0x8d, 0x0d, 0x15, 0x20, 0xd3, 0xef, 0xe8, 0x5b, 0xec, 0x17, 0xb7, 0x75, 0x65, 0x7f, 0x3f, 0xb8, + 0x68, 0xb3, 0xae, 0xac, 0x02, 0xb0, 0xde, 0x9a, 0xd9, 0x3e, 0xb5, 0x47, 0xfa, 0x16, 0xda, 0x86, + 0xd2, 0x39, 0xab, 0x91, 0x50, 0x28, 0xfb, 0xdf, 0x83, 0x16, 0x5e, 0xb3, 0x11, 0x40, 0xe1, 0x70, + 0x44, 0xed, 0x2b, 0xa2, 0x6f, 0xa1, 0x32, 0xa8, 0xf2, 0x02, 0xac, 0x2b, 0x0c, 0x27, 0x28, 0x0f, + 0xb5, 0x9d, 0x89, 0x9e, 0x41, 0x15, 0xd0, 0x84, 0x4c, 0xc6, 0x7a, 0x96, 0x39, 0x1f, 0x0e, 0x5d, + 0x8f, 0x1b, 0x73, 0xa8, 0x04, 0x45, 0x2e, 0x91, 0xb1, 0x9e, 0xdf, 0xff, 0x41, 0xe1, 0x19, 0xc4, + 0xe7, 0x89, 0x0a, 0x39, 0x76, 0x9d, 0xd3, 0xb7, 0x90, 0x06, 0x79, 0x7e, 0x51, 0xd3, 0x15, 0x96, + 0x36, 0x00, 0xd3, 0x33, 0x0c, 0x49, 0xbe, 0xab, 0x9e, 0x65, 0x48, 0x62, 0x11, 0x7a, 0x8e, 0xe5, + 0x0c, 0x47, 0x9b, 0x9e, 0x47, 0x35, 0xf9, 0x8d, 0x2f, 0xe8, 0xab, 0x17, 0xd0, 0xad, 0xe8, 0xcb, + 0x5d, 0x2a, 0x8b, 0x48, 0x87, 0xb2, 0xa4, 0x34, 0x23, 0xb4, 0xae, 0xb2, 0xd4, 0xc7, 0x27, 0x47, + 0xaf, 0x5f, 0xea, 0xda, 0xd1, 0xd1, 0x87, 0xdf, 0xea, 0xca, 0xfb, 0x8f, 0x75, 0xe5, 0xc3, 0xc7, + 0xba, 0xf2, 0xeb, 0xc7, 0xfa, 0xd6, 0x8f, 0xbf, 0xd7, 0x95, 0xef, 0x1e, 0xc7, 0xfe, 0x06, 0x99, + 0x59, 0xd4, 0xb3, 0x97, 0xae, 0x67, 0x4f, 0x6c, 0x47, 0x0a, 0x0e, 0x69, 0xcd, 0x2f, 0x27, 0xad, + 0xf9, 0xb0, 0x45, 0x97, 0xce, 0xb0, 0xc0, 0xff, 0xdf, 0x78, 0xf6, 0x77, 0x00, 0x00, 0x00, 0xff, + 0xff, 0x28, 0x6b, 0x1e, 0x0e, 0x4d, 0x11, 0x00, 0x00, } func (m *TxnMeta) Marshal() (dAtA []byte, err error) { @@ -2133,6 +2141,11 @@ func (m *CNTxnSnapshot) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.Flag != 0 { + i = encodeVarintTxn(dAtA, i, uint64(m.Flag)) + i-- + dAtA[i] = 0x38 + } { size, err := m.Options.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -3443,6 +3456,9 @@ func (m *CNTxnSnapshot) ProtoSize() (n int) { } l = m.Options.ProtoSize() n += 1 + l + sovTxn(uint64(l)) + if m.Flag != 0 { + n += 1 + sovTxn(uint64(m.Flag)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -4476,6 +4492,25 @@ func (m *CNTxnSnapshot) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Flag", wireType) + } + m.Flag = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTxn + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Flag |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipTxn(dAtA[iNdEx:]) diff --git a/pkg/tests/issues/issue_test.go b/pkg/tests/issues/issue_test.go index 1a58549195d8..6c69211df7cf 100644 --- a/pkg/tests/issues/issue_test.go +++ b/pkg/tests/issues/issue_test.go @@ -26,14 +26,18 @@ import ( "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/cnservice" + "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/runtime" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/embed" + "github.com/matrixorigin/matrixone/pkg/lockservice" "github.com/matrixorigin/matrixone/pkg/pb/lock" + "github.com/matrixorigin/matrixone/pkg/pb/metadata" "github.com/matrixorigin/matrixone/pkg/sql/colexec/deletion" "github.com/matrixorigin/matrixone/pkg/tests/testutils" + "github.com/matrixorigin/matrixone/pkg/tnservice" "github.com/matrixorigin/matrixone/pkg/util/executor" "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" "github.com/stretchr/testify/require" @@ -600,3 +604,143 @@ func TestLockNeedUpgrade(t *testing.T) { }, ) } + +func TestIssue19551(t *testing.T) { + embed.RunBaseClusterTests( + func(c embed.Cluster) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*1000) + defer cancel() + + var allocator lockservice.LockTableAllocator + c.ForeachServices( + func(s embed.ServiceOperator) bool { + if s.ServiceType() != metadata.ServiceType_TN { + return true + } + + tn := s.RawService().(tnservice.Service) + allocator = tn.GetLockTableAllocator() + return false + }, + ) + + cn1, err := c.GetCNService(0) + require.NoError(t, err) + lockSID := lockservice.GetLockServiceByServiceID(cn1.ServiceID()).GetServiceID() + + db := testutils.GetDatabaseName(t) + table := "t" + + testutils.CreateTableAndWaitCNApplied( + t, + db, + table, + "create table "+table+" (id int primary key, v int)", + cn1, + ) + + // workflow: + // start txn1, txn2 on cn1 + // mark cn1 invalid + // commit txn1 + // wait abort active txn completed + // commit txn2 + // start txn3 and commit will success + + var wg sync.WaitGroup + wg.Add(2) + + txn1StartedC := make(chan struct{}) + txn2StartedC := make(chan struct{}) + invalidMarkedC := make(chan struct{}) + + // txn1 + exec := testutils.GetSQLExecutor(cn1) + go func() { + defer wg.Done() + + err := exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + close(txn1StartedC) + <-invalidMarkedC + + res, err := txn.Exec( + "insert into "+table+" values (1, 1)", + executor.StatementOption{}, + ) + if err != nil { + return err + } + res.Close() + return nil + }, + executor.Options{}.WithDatabase(db), + ) + require.Error(t, err) + require.True(t, moerr.IsMoErrCode(err, moerr.ErrCannotCommitOnInvalidCN)) + }() + + go func() { + defer wg.Done() + + err := exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + close(txn2StartedC) + + res, err := txn.Exec( + "insert into "+table+" values (2, 2)", + executor.StatementOption{}, + ) + if err != nil { + return err + } + res.Close() + + // wait valid service resume, means all active in txn client is marked aborted + for { + if !allocator.HasInvalidService(lockSID) { + break + } + time.Sleep(time.Millisecond * 100) + } + return nil + }, + executor.Options{}.WithDatabase(db), + ) + require.Error(t, err) + require.True(t, moerr.IsMoErrCode(err, moerr.ErrTxnClosed)) + + }() + + <-txn1StartedC + <-txn2StartedC + + // mark cn1 invalid + allocator.AddInvalidService(lockSID) + require.True(t, allocator.HasInvalidService(lockSID)) + close(invalidMarkedC) + wg.Wait() + + // service is resume, txn3 can commit ok + err = exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + res, err := txn.Exec( + "insert into "+table+" values (3, 3)", + executor.StatementOption{}, + ) + if err != nil { + return err + } + res.Close() + + return nil + }, + executor.Options{}.WithDatabase(db), + ) + require.NoError(t, err) + }, + ) +} diff --git a/pkg/tnservice/store.go b/pkg/tnservice/store.go index d170a7926440..f06250b02580 100644 --- a/pkg/tnservice/store.go +++ b/pkg/tnservice/store.go @@ -510,3 +510,7 @@ func (s *store) setupStatusServer() { } } + +func (s *store) GetLockTableAllocator() lockservice.LockTableAllocator { + return s.lockTableAllocator +} diff --git a/pkg/tnservice/types.go b/pkg/tnservice/types.go index f6537260fd0b..dc9999e4b6d8 100644 --- a/pkg/tnservice/types.go +++ b/pkg/tnservice/types.go @@ -15,6 +15,7 @@ package tnservice import ( + "github.com/matrixorigin/matrixone/pkg/lockservice" "github.com/matrixorigin/matrixone/pkg/pb/metadata" "github.com/matrixorigin/matrixone/pkg/taskservice" ) @@ -49,4 +50,6 @@ type Service interface { // GetTaskService returns taskservice GetTaskService() (taskservice.TaskService, bool) + // GetLockTableAllocator returns lock table allocator + GetLockTableAllocator() lockservice.LockTableAllocator } diff --git a/pkg/txn/client/client.go b/pkg/txn/client/client.go index 06ac7b94eab7..3b8c32ca6775 100644 --- a/pkg/txn/client/client.go +++ b/pkg/txn/client/client.go @@ -23,12 +23,10 @@ import ( "sync/atomic" "time" - "go.uber.org/ratelimit" - "go.uber.org/zap" - "github.com/matrixorigin/matrixone/pkg/common/log" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/runtime" + "github.com/matrixorigin/matrixone/pkg/common/stopper" "github.com/matrixorigin/matrixone/pkg/lockservice" "github.com/matrixorigin/matrixone/pkg/pb/timestamp" "github.com/matrixorigin/matrixone/pkg/pb/txn" @@ -36,6 +34,8 @@ import ( "github.com/matrixorigin/matrixone/pkg/txn/rpc" "github.com/matrixorigin/matrixone/pkg/txn/util" v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" + "go.uber.org/ratelimit" + "go.uber.org/zap" ) // WithTxnIDGenerator setup txn id generator @@ -157,6 +157,7 @@ const ( type txnClient struct { sid string + stopper *stopper.Stopper logger *log.MOLogger clock clock.Clock sender rpc.TxnSender @@ -203,6 +204,8 @@ type txnClient struct { // FIFO queue for ready to active txn waitActiveTxns []*txnOperator } + + abortC chan time.Time } func (client *txnClient) GetState() TxnState { @@ -236,7 +239,9 @@ func NewTxnClient( logger: util.GetLogger(sid), clock: runtime.ServiceRuntime(sid).Clock(), sender: sender, + abortC: make(chan time.Time, 1), } + c.stopper = stopper.NewStopper("txn-client", stopper.WithLogger(c.logger.RawLogger())) c.mu.state = paused c.mu.cond = sync.NewCond(&c.mu) c.mu.activeTxns = make(map[string]*txnOperator, 100000) @@ -245,6 +250,9 @@ func NewTxnClient( } c.adjust() c.startLeakChecker() + if err := c.stopper.RunTask(c.handleMarkActiveTxnAborted); err != nil { + panic(err) + } return c } @@ -314,9 +322,11 @@ func (client *txnClient) doCreateTxn( client.limiter.Take() op.timestampWaiter = client.timestampWaiter - op.AppendEventCallback(ClosedEvent, + op.AppendEventCallback( + ClosedEvent, client.updateLastCommitTS, - client.closeTxn) + client.closeTxn, + ) if err := client.openTxn(op); err != nil { return nil, err @@ -364,6 +374,7 @@ func (client *txnClient) NewWithSnapshot( } func (client *txnClient) Close() error { + client.stopper.Stop() if client.leakChecker != nil { client.leakChecker.close() } @@ -545,6 +556,10 @@ func (client *txnClient) closeTxn(event TxnEvent) { client.mu.Unlock() }() + if moerr.IsMoErrCode(event.Err, moerr.ErrCannotCommitOnInvalidCN) { + client.markAllActiveTxnAborted() + } + key := string(txn.ID) op, ok := client.mu.activeTxns[key] if ok { @@ -729,3 +744,38 @@ func (client *txnClient) getTxnOptions( } return options } + +func (client *txnClient) markAllActiveTxnAborted() { + select { + case client.abortC <- time.Now(): + default: + } +} + +func (client *txnClient) handleMarkActiveTxnAborted( + ctx context.Context, +) { + select { + case <-ctx.Done(): + return + case from := <-client.abortC: + fn := func() { + client.mu.Lock() + defer client.mu.Unlock() + + for _, op := range client.mu.activeTxns { + if op.reset.createAt.Before(from) { + op.addFlag(AbortedFlag) + } + } + } + fn() + + if err := client.lockService.(lockservice.ResumeLockService).Resume(); err != nil { + client.logger.Error( + "resume lock service failed", + zap.Error(err), + ) + } + } +} diff --git a/pkg/txn/client/client_test.go b/pkg/txn/client/client_test.go index 7a3c6ea66b81..1453331bc295 100644 --- a/pkg/txn/client/client_test.go +++ b/pkg/txn/client/client_test.go @@ -78,6 +78,10 @@ func TestNewTxnWithNormalStateWait(t *testing.T) { }, 0))) runtime.SetupServiceBasedRuntime("", rt) c := NewTxnClient("", newTestTxnSender()) + defer func() { + require.NoError(t, c.Close()) + }() + // Do not resume the txn client for now. // c.Resume() var wg sync.WaitGroup @@ -109,6 +113,10 @@ func TestNewTxnWithNormalStateNoWait(t *testing.T) { }, 0))) runtime.SetupServiceBasedRuntime("", rt) c := NewTxnClient("", newTestTxnSender(), WithNormalStateNoWait(true)) + defer func() { + require.NoError(t, c.Close()) + }() + // Do not resume the txn client. // c.Resume() var wg sync.WaitGroup diff --git a/pkg/txn/client/operator.go b/pkg/txn/client/operator.go index 986ccc11e376..fe44fff9b99e 100644 --- a/pkg/txn/client/operator.go +++ b/pkg/txn/client/operator.go @@ -24,8 +24,6 @@ import ( "sync/atomic" "time" - "go.uber.org/zap" - "github.com/matrixorigin/matrixone/pkg/common/log" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/runtime" @@ -38,6 +36,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/txn/rpc" "github.com/matrixorigin/matrixone/pkg/txn/util" v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" + "go.uber.org/zap" ) var ( @@ -55,14 +54,15 @@ var ( moerr.ErrTxnNotActive: {}, } commitTxnErrors = map[uint16]struct{}{ - moerr.ErrTAECommit: {}, - moerr.ErrTAERollback: {}, - moerr.ErrTAEPrepare: {}, - moerr.ErrRpcError: {}, - moerr.ErrTxnNotFound: {}, - moerr.ErrTxnNotActive: {}, - moerr.ErrLockTableBindChanged: {}, - moerr.ErrCannotCommitOrphan: {}, + moerr.ErrTAECommit: {}, + moerr.ErrTAERollback: {}, + moerr.ErrTAEPrepare: {}, + moerr.ErrRpcError: {}, + moerr.ErrTxnNotFound: {}, + moerr.ErrTxnNotActive: {}, + moerr.ErrLockTableBindChanged: {}, + moerr.ErrCannotCommitOrphan: {}, + moerr.ErrCannotCommitOnInvalidCN: {}, } rollbackTxnErrors = map[uint16]struct{}{ moerr.ErrTAERollback: {}, @@ -234,6 +234,7 @@ type txnOperator struct { waitLocks map[uint64]Lock //read-only txn operators for supporting snapshot read feature. children []*txnOperator + flag uint32 } reset struct { @@ -255,6 +256,7 @@ type txnOperator struct { rollbackStmtCounter counter fprints footPrints runningSQL atomic.Bool + commitErr error } opts struct { @@ -325,6 +327,7 @@ func (tc *txnOperator) initReset() { tc.reset.rollbackStmtCounter = counter{} tc.reset.fprints = footPrints{} tc.reset.runningSQL.Store(false) + tc.reset.commitErr = nil } func (tc *txnOperator) initProtectedFields() { @@ -390,6 +393,7 @@ func newTxnOperatorWithSnapshot( tc.mu.txn = snapshot.Txn tc.mu.txn.Mirror = true tc.mu.lockTables = snapshot.LockTables + tc.mu.flag = snapshot.Flag tc.adjust() util.LogTxnCreated(tc.logger, tc.mu.txn) @@ -494,6 +498,7 @@ func (tc *txnOperator) Snapshot() (txn.CNTxnSnapshot, error) { Txn: tc.mu.txn, LockTables: tc.mu.lockTables, Options: tc.opts.options, + Flag: tc.mu.flag, }, nil } @@ -801,7 +806,8 @@ func (tc *txnOperator) doWrite(ctx context.Context, requests []txn.TxnRequest, c tc.mu.Unlock() }() if tc.mu.closed { - return nil, moerr.NewTxnClosedNoCtx(tc.reset.txnID) + tc.reset.commitErr = moerr.NewTxnClosedNoCtx(tc.reset.txnID) + return nil, tc.reset.commitErr } if tc.needUnlockLocked() { @@ -846,7 +852,16 @@ func (tc *txnOperator) doWrite(ctx context.Context, requests []txn.TxnRequest, c Disable1PCOpt: tc.opts.options.Is1PCDisabled(), }}) } - return tc.trimResponses(tc.handleError(tc.doSend(ctx, requests, commit))) + if commit && tc.markAbortedLocked() { + tc.reset.commitErr = moerr.NewTxnClosedNoCtx(tc.reset.txnID) + return nil, tc.reset.commitErr + } + + resp, err := tc.trimResponses(tc.handleError(tc.doSend(ctx, requests, commit))) + if err != nil && commit { + tc.reset.commitErr = err + } + return resp, err } func (tc *txnOperator) updateWritePartitions(requests []txn.TxnRequest, locked bool) { @@ -971,8 +986,12 @@ func (tc *txnOperator) getTxnMeta(locked bool) txn.TxnMeta { return tc.mu.txn } -func (tc *txnOperator) doSend(ctx context.Context, requests []txn.TxnRequest, locked bool) (*rpc.SendResult, error) { - txnMeta := tc.getTxnMeta(locked) +func (tc *txnOperator) doSend( + ctx context.Context, + requests []txn.TxnRequest, + commit bool, +) (*rpc.SendResult, error) { + txnMeta := tc.getTxnMeta(commit) for idx := range requests { requests[idx].Txn = txnMeta } @@ -994,7 +1013,7 @@ func (tc *txnOperator) doSend(ctx context.Context, requests []txn.TxnRequest, lo if resp.Txn == nil { return result, nil } - if !locked { + if !commit { tc.mu.Lock() defer tc.mu.Unlock() } @@ -1242,6 +1261,7 @@ func (tc *txnOperator) closeLocked() { TxnEvent{ Event: ClosedEvent, Txn: tc.mu.txn, + Err: tc.reset.commitErr, }) } } @@ -1419,3 +1439,15 @@ func (tc *txnOperator) counter() string { func (tc *txnOperator) SetFootPrints(id int, enter bool) { tc.reset.fprints.add(id, enter) } + +func (tc *txnOperator) addFlag(flags ...uint32) { + tc.mu.Lock() + defer tc.mu.Unlock() + for _, flag := range flags { + tc.mu.flag |= flag + } +} + +func (tc *txnOperator) markAbortedLocked() bool { + return tc.mu.flag&AbortedFlag != 0 +} diff --git a/pkg/txn/client/testutil.go b/pkg/txn/client/testutil.go index a6810707be3d..46edbd33652b 100644 --- a/pkg/txn/client/testutil.go +++ b/pkg/txn/client/testutil.go @@ -32,6 +32,7 @@ func RunTxnTests(fn func(TxnClient, rpc.TxnSender), opts ...TxnClientCreateOptio runtime.SetupServiceBasedRuntime("", runtime.DefaultRuntime()) ts := newTestTxnSender() c := NewTxnClient("", ts, opts...) + defer c.Close() c.Resume() fn(c, ts) } diff --git a/pkg/txn/client/types.go b/pkg/txn/client/types.go index 9ac0becc5355..e05207c28c33 100644 --- a/pkg/txn/client/types.go +++ b/pkg/txn/client/types.go @@ -25,6 +25,11 @@ import ( "github.com/matrixorigin/matrixone/pkg/txn/rpc" ) +const ( + DebugFlag = uint32(1 << 0) + AbortedFlag = uint32(1 << 1) +) + // TxnOption options for setup transaction // FIXME(fagongzi): refactor TxnOption to avoid mem alloc type TxnOption func(*txnOperator) @@ -123,6 +128,7 @@ type TxnOperator interface { Txn() txn.TxnMeta // TxnOptions returns the current txn options TxnOptions() txn.TxnOptions + // TxnRef returns pointer of current txn metadata. In RC mode, txn's snapshot ts // will updated before statement executed. TxnRef() *txn.TxnMeta diff --git a/proto/lock.proto b/proto/lock.proto index 25e829365ca5..88e10bbbba30 100644 --- a/proto/lock.proto +++ b/proto/lock.proto @@ -113,6 +113,8 @@ enum Method { CanRestartService = 14; // CheckOrphan check txn is orphan CheckOrphan = 15; + // ResumeInvalidCN resume invalid cn + ResumeInvalidCN = 16; } enum Status { @@ -144,8 +146,11 @@ message Request { CanRestartServiceRequest CanRestartService = 15 [(gogoproto.nullable) = false]; RemainTxnInServiceRequest RemainTxnInService = 16 [(gogoproto.nullable) = false]; CheckOrphanRequest CheckOrphan = 17 [(gogoproto.nullable) = false]; + ResumeInvalidCNRequest ResumeInvalidCN = 18 [(gogoproto.nullable) = false]; } + + // Response response message Response { // RequestID corresponding request id @@ -169,6 +174,7 @@ message Response { CanRestartServiceResponse CanRestartService = 16 [(gogoproto.nullable) = false]; RemainTxnInServiceResponse RemainTxnInService = 17 [(gogoproto.nullable) = false]; CheckOrphanResponse CheckOrphan = 18 [(gogoproto.nullable) = false]; + ResumeInvalidCNResponse ResumeInvalidCN = 19 [(gogoproto.nullable) = false]; } // LockRequest lock request @@ -369,4 +375,12 @@ message ExtraMutation { bytes key = 1; bool skip = 2; bytes replaceTo = 3; +} + +message ResumeInvalidCNRequest { + string ServiceID = 1; +} + +message ResumeInvalidCNResponse { + } \ No newline at end of file diff --git a/proto/txn.proto b/proto/txn.proto index 390144b318b3..e359ad6ce857 100644 --- a/proto/txn.proto +++ b/proto/txn.proto @@ -132,6 +132,7 @@ message CNTxnSnapshot { bool Disable1PCOpt = 4; repeated lock.LockTable LockTables = 5 [(gogoproto.nullable) = false]; TxnOptions Options = 6 [(gogoproto.nullable) = false]; + uint32 Flag = 7; } // CNOpRequest cn read/write request, CN -> TN. If data is written to more than one TN (>1) in a