github.com/matrixorigin/matrixone@v1.2.0/pkg/hakeeper/rsm.go (about)

     1  // Copyright 2021 - 2022 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //      http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  /*
    16  Package hakeeper implements MO's hakeeper component.
    17  */
    18  package hakeeper
    19  
    20  import (
    21  	"encoding/binary"
    22  	"fmt"
    23  	"io"
    24  	"strings"
    25  	"time"
    26  
    27  	"github.com/lni/dragonboat/v4/logger"
    28  	sm "github.com/lni/dragonboat/v4/statemachine"
    29  	"github.com/mohae/deepcopy"
    30  
    31  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    32  	pb "github.com/matrixorigin/matrixone/pkg/pb/logservice"
    33  	"github.com/matrixorigin/matrixone/pkg/pb/metadata"
    34  )
    35  
    36  var (
    37  	plog = logger.GetLogger("hakeeper")
    38  )
    39  
    40  var (
    41  	binaryEnc = binary.BigEndian
    42  )
    43  
    44  const (
    45  	// When bootstrapping, k8s will first bootstrap the HAKeeper by starting some
    46  	// Log stores with command line options specifying that those stores will be hosting
    47  	// a HAKeeper replicas. It will be k8s's responsibility to assign Replica IDs to those
    48  	// HAKeeper replicas, and those IDs will have to be assigned from the range
    49  	// [K8SIDRangeStart, K8SIDRangeEnd)
    50  
    51  	K8SIDRangeStart uint64 = 131072
    52  	K8SIDRangeEnd   uint64 = 262144
    53  	// CheckDuration defines how often HAKeeper checks the health state of the cluster
    54  	CheckDuration = 3 * time.Second
    55  	// DefaultHAKeeperShardID is the shard ID assigned to the special HAKeeper
    56  	// shard.
    57  	DefaultHAKeeperShardID uint64 = 0
    58  	headerSize                    = pb.HeaderSize
    59  )
    60  
    61  type IndexQuery struct{}
    62  type StateQuery struct{}
    63  type ScheduleCommandQuery struct{ UUID string }
    64  type ClusterDetailsQuery struct{ Cfg Config }
    65  
    66  type stateMachine struct {
    67  	replicaID uint64
    68  	state     pb.HAKeeperRSMState
    69  }
    70  
    71  func parseCmdTag(cmd []byte) pb.HAKeeperUpdateType {
    72  	return pb.HAKeeperUpdateType(binaryEnc.Uint32(cmd))
    73  }
    74  
    75  func GetInitialClusterRequestCmd(numOfLogShards uint64,
    76  	numOfTNShards uint64, numOfLogReplicas uint64, nextID uint64, nextIDByKey map[string]uint64) []byte {
    77  	req := pb.InitialClusterRequest{
    78  		NumOfLogShards:   numOfLogShards,
    79  		NumOfTNShards:    numOfTNShards,
    80  		NumOfLogReplicas: numOfLogReplicas,
    81  		NextID:           nextID,
    82  		NextIDByKey:      nextIDByKey,
    83  	}
    84  	payload, err := req.Marshal()
    85  	if err != nil {
    86  		panic(err)
    87  	}
    88  	cmd := make([]byte, headerSize+len(payload))
    89  	binaryEnc.PutUint32(cmd, uint32(pb.InitialClusterUpdate))
    90  	copy(cmd[headerSize:], payload)
    91  	return cmd
    92  }
    93  
    94  func parseInitialClusterRequestCmd(cmd []byte) pb.InitialClusterRequest {
    95  	if parseCmdTag(cmd) != pb.InitialClusterUpdate {
    96  		panic("not a initial cluster update")
    97  	}
    98  	payload := cmd[headerSize:]
    99  	var result pb.InitialClusterRequest
   100  	if err := result.Unmarshal(payload); err != nil {
   101  		panic(err)
   102  	}
   103  	return result
   104  }
   105  
   106  func parseTaskTableUserCmd(cmd []byte) pb.TaskTableUser {
   107  	if parseCmdTag(cmd) != pb.SetTaskTableUserUpdate {
   108  		panic("not a task table user update")
   109  	}
   110  	payload := cmd[headerSize:]
   111  	var result pb.TaskTableUser
   112  	if err := result.Unmarshal(payload); err != nil {
   113  		panic(err)
   114  	}
   115  	return result
   116  }
   117  
   118  func GetUpdateCommandsCmd(term uint64, cmds []pb.ScheduleCommand) []byte {
   119  	b := pb.CommandBatch{
   120  		Term:     term,
   121  		Commands: cmds,
   122  	}
   123  	data := make([]byte, headerSize+b.Size())
   124  	binaryEnc.PutUint32(data, uint32(pb.ScheduleCommandUpdate))
   125  	if _, err := b.MarshalTo(data[headerSize:]); err != nil {
   126  		panic(err)
   127  	}
   128  	return data
   129  }
   130  
   131  func parseHeartbeatCmd(cmd []byte) []byte {
   132  	return cmd[headerSize:]
   133  }
   134  
   135  func parseSetStateCmd(cmd []byte) pb.HAKeeperState {
   136  	return pb.HAKeeperState(binaryEnc.Uint32(cmd[headerSize:]))
   137  }
   138  
   139  func parseSetInitTaskStateCmd(cmd []byte) pb.TaskSchedulerState {
   140  	return pb.TaskSchedulerState(binaryEnc.Uint32(cmd[headerSize:]))
   141  }
   142  
   143  func parseAllocateIDCmd(cmd []byte) pb.CNAllocateID {
   144  	if parseCmdTag(cmd) != pb.GetIDUpdate {
   145  		panic("not a allocate ID cmd")
   146  	}
   147  	payload := cmd[headerSize:]
   148  	var result pb.CNAllocateID
   149  	if err := result.Unmarshal(payload); err != nil {
   150  		panic(err)
   151  	}
   152  	return result
   153  }
   154  
   155  func parseUpdateCNLabelCmd(cmd []byte) pb.CNStoreLabel {
   156  	if parseCmdTag(cmd) != pb.UpdateCNLabel {
   157  		panic("not a SetCNLabel cmd")
   158  	}
   159  	payload := cmd[headerSize:]
   160  	var result pb.CNStoreLabel
   161  	if err := result.Unmarshal(payload); err != nil {
   162  		panic(err)
   163  	}
   164  	return result
   165  }
   166  
   167  func parseUpdateCNWorkStateCmd(cmd []byte) pb.CNWorkState {
   168  	if parseCmdTag(cmd) != pb.UpdateCNWorkState {
   169  		panic("not a SetCNWorkState cmd")
   170  	}
   171  	payload := cmd[headerSize:]
   172  	var result pb.CNWorkState
   173  	if err := result.Unmarshal(payload); err != nil {
   174  		panic(err)
   175  	}
   176  	return result
   177  }
   178  
   179  func parsePatchCNStoreCmd(cmd []byte) pb.CNStateLabel {
   180  	if parseCmdTag(cmd) != pb.PatchCNStore {
   181  		panic("not a PatchCNStore cmd")
   182  	}
   183  	payload := cmd[headerSize:]
   184  	var result pb.CNStateLabel
   185  	if err := result.Unmarshal(payload); err != nil {
   186  		panic(err)
   187  	}
   188  	return result
   189  }
   190  
   191  func parseDeleteCNStoreCmd(cmd []byte) pb.DeleteCNStore {
   192  	if parseCmdTag(cmd) != pb.RemoveCNStore {
   193  		panic("not a RemoveCNStore cmd")
   194  	}
   195  	payload := cmd[headerSize:]
   196  	var result pb.DeleteCNStore
   197  	if err := result.Unmarshal(payload); err != nil {
   198  		panic(err)
   199  	}
   200  	return result
   201  }
   202  
   203  func GetSetStateCmd(state pb.HAKeeperState) []byte {
   204  	cmd := make([]byte, headerSize+4)
   205  	binaryEnc.PutUint32(cmd, uint32(pb.SetStateUpdate))
   206  	binaryEnc.PutUint32(cmd[headerSize:], uint32(state))
   207  	return cmd
   208  }
   209  
   210  func GetSetTaskSchedulerStateCmd(state pb.TaskSchedulerState) []byte {
   211  	cmd := make([]byte, headerSize+4)
   212  	binaryEnc.PutUint32(cmd, uint32(pb.SetTaskSchedulerStateUpdate))
   213  	binaryEnc.PutUint32(cmd[headerSize:], uint32(state))
   214  	return cmd
   215  }
   216  
   217  func GetTaskTableUserCmd(user pb.TaskTableUser) []byte {
   218  	cmd := make([]byte, headerSize+user.Size())
   219  	binaryEnc.PutUint32(cmd, uint32(pb.SetTaskTableUserUpdate))
   220  	if _, err := user.MarshalTo(cmd[headerSize:]); err != nil {
   221  		panic(err)
   222  	}
   223  	return cmd
   224  }
   225  
   226  func GetTickCmd() []byte {
   227  	cmd := make([]byte, headerSize)
   228  	binaryEnc.PutUint32(cmd, uint32(pb.TickUpdate))
   229  	return cmd
   230  }
   231  
   232  func GetLogStoreHeartbeatCmd(data []byte) []byte {
   233  	return getHeartbeatCmd(data, pb.LogHeartbeatUpdate)
   234  }
   235  
   236  func GetCNStoreHeartbeatCmd(data []byte) []byte {
   237  	return getHeartbeatCmd(data, pb.CNHeartbeatUpdate)
   238  }
   239  
   240  func GetTNStoreHeartbeatCmd(data []byte) []byte {
   241  	return getHeartbeatCmd(data, pb.TNHeartbeatUpdate)
   242  }
   243  
   244  func GetProxyHeartbeatCmd(data []byte) []byte {
   245  	return getHeartbeatCmd(data, pb.ProxyHeartbeatUpdate)
   246  }
   247  
   248  func getHeartbeatCmd(data []byte, tag pb.HAKeeperUpdateType) []byte {
   249  	cmd := make([]byte, headerSize+len(data))
   250  	binaryEnc.PutUint32(cmd, uint32(tag))
   251  	copy(cmd[headerSize:], data)
   252  	return cmd
   253  }
   254  
   255  func GetAllocateIDCmd(allocID pb.CNAllocateID) []byte {
   256  	cmd := make([]byte, headerSize+allocID.Size())
   257  	binaryEnc.PutUint32(cmd, uint32(pb.GetIDUpdate))
   258  	if _, err := allocID.MarshalTo(cmd[headerSize:]); err != nil {
   259  		panic(err)
   260  	}
   261  	return cmd
   262  }
   263  
   264  func GetUpdateCNLabelCmd(label pb.CNStoreLabel) []byte {
   265  	cmd := make([]byte, headerSize+label.Size())
   266  	binaryEnc.PutUint32(cmd, uint32(pb.UpdateCNLabel))
   267  	if _, err := label.MarshalTo(cmd[headerSize:]); err != nil {
   268  		panic(err)
   269  	}
   270  	return cmd
   271  }
   272  
   273  func GetUpdateCNWorkStateCmd(state pb.CNWorkState) []byte {
   274  	cmd := make([]byte, headerSize+state.Size())
   275  	binaryEnc.PutUint32(cmd, uint32(pb.UpdateCNWorkState))
   276  	if _, err := state.MarshalTo(cmd[headerSize:]); err != nil {
   277  		panic(err)
   278  	}
   279  	return cmd
   280  }
   281  
   282  func GetPatchCNStoreCmd(stateLabel pb.CNStateLabel) []byte {
   283  	cmd := make([]byte, headerSize+stateLabel.Size())
   284  	binaryEnc.PutUint32(cmd, uint32(pb.PatchCNStore))
   285  	if _, err := stateLabel.MarshalTo(cmd[headerSize:]); err != nil {
   286  		panic(err)
   287  	}
   288  	return cmd
   289  }
   290  
   291  func GetDeleteCNStoreCmd(cnStore pb.DeleteCNStore) []byte {
   292  	cmd := make([]byte, headerSize+cnStore.Size())
   293  	binaryEnc.PutUint32(cmd, uint32(pb.RemoveCNStore))
   294  	if _, err := cnStore.MarshalTo(cmd[headerSize:]); err != nil {
   295  		panic(err)
   296  	}
   297  	return cmd
   298  }
   299  
   300  func NewStateMachine(shardID uint64, replicaID uint64) sm.IStateMachine {
   301  	if shardID != DefaultHAKeeperShardID {
   302  		panic(moerr.NewInvalidInputNoCtx("HAKeeper shard ID %d does not match DefaultHAKeeperShardID %d", shardID, DefaultHAKeeperShardID))
   303  	}
   304  	return &stateMachine{
   305  		replicaID: replicaID,
   306  		state:     pb.NewRSMState(),
   307  	}
   308  }
   309  
   310  func (s *stateMachine) Close() error {
   311  	return nil
   312  }
   313  
   314  func (s *stateMachine) assignID() uint64 {
   315  	s.state.NextID++
   316  	return s.state.NextID
   317  }
   318  
   319  func (s *stateMachine) assignIDByKey(key string) uint64 {
   320  	if _, ok := s.state.NextIDByKey[key]; !ok {
   321  		s.state.NextIDByKey[key] = 0
   322  	}
   323  	s.state.NextIDByKey[key]++
   324  	return s.state.NextIDByKey[key]
   325  }
   326  
   327  func (s *stateMachine) handleUpdateCommandsCmd(cmd []byte) sm.Result {
   328  	data := cmd[headerSize:]
   329  	var b pb.CommandBatch
   330  	if err := b.Unmarshal(data); err != nil {
   331  		panic(err)
   332  	}
   333  	if s.state.Term > b.Term {
   334  		return sm.Result{}
   335  	}
   336  
   337  	for _, c := range b.Commands {
   338  		if c.Bootstrapping {
   339  			if s.state.State != pb.HAKeeperBootstrapping {
   340  				plog.Errorf("ignored bootstrapping cmd: %s", c.LogString())
   341  				return sm.Result{}
   342  			}
   343  		}
   344  	}
   345  
   346  	s.state.Term = b.Term
   347  	s.state.ScheduleCommands = make(map[string]pb.CommandBatch)
   348  	for _, c := range b.Commands {
   349  		if c.Bootstrapping {
   350  			s.handleSetStateCmd(GetSetStateCmd(pb.HAKeeperBootstrapCommandsReceived))
   351  		}
   352  		if c.DeleteCNStore != nil {
   353  			s.handleDeleteCNCmd(c.UUID)
   354  			continue
   355  		}
   356  		if c.DeleteProxyStore != nil {
   357  			s.handleDeleteProxyCmd(c.UUID)
   358  			continue
   359  		}
   360  		l, ok := s.state.ScheduleCommands[c.UUID]
   361  		if !ok {
   362  			l = pb.CommandBatch{
   363  				Commands: make([]pb.ScheduleCommand, 0),
   364  			}
   365  		}
   366  		plog.Infof("adding schedule command to hakeeper rsm: %s", c.LogString())
   367  		l.Commands = append(l.Commands, c)
   368  		s.state.ScheduleCommands[c.UUID] = l
   369  	}
   370  
   371  	return sm.Result{}
   372  }
   373  
   374  func (s *stateMachine) getCommandBatch(uuid string) sm.Result {
   375  	if batch, ok := s.state.ScheduleCommands[uuid]; ok {
   376  		delete(s.state.ScheduleCommands, uuid)
   377  		data, err := batch.Marshal()
   378  		if err != nil {
   379  			panic(err)
   380  		}
   381  		return sm.Result{Data: data}
   382  	}
   383  	return sm.Result{}
   384  
   385  }
   386  
   387  func (s *stateMachine) handleCNHeartbeat(cmd []byte) sm.Result {
   388  	data := parseHeartbeatCmd(cmd)
   389  	var hb pb.CNStoreHeartbeat
   390  	if err := hb.Unmarshal(data); err != nil {
   391  		panic(err)
   392  	}
   393  	s.state.CNState.Update(hb, s.state.Tick)
   394  	return s.getCommandBatch(hb.UUID)
   395  }
   396  
   397  func (s *stateMachine) handleTNHeartbeat(cmd []byte) sm.Result {
   398  	data := parseHeartbeatCmd(cmd)
   399  	var hb pb.TNStoreHeartbeat
   400  	if err := hb.Unmarshal(data); err != nil {
   401  		panic(err)
   402  	}
   403  	s.state.TNState.Update(hb, s.state.Tick)
   404  	return s.getCommandBatch(hb.UUID)
   405  }
   406  
   407  func (s *stateMachine) handleLogHeartbeat(cmd []byte) sm.Result {
   408  	data := parseHeartbeatCmd(cmd)
   409  	var hb pb.LogStoreHeartbeat
   410  	if err := hb.Unmarshal(data); err != nil {
   411  		panic(err)
   412  	}
   413  	s.state.LogState.Update(hb, s.state.Tick)
   414  	return s.getCommandBatch(hb.UUID)
   415  }
   416  
   417  func (s *stateMachine) handleTick(cmd []byte) sm.Result {
   418  	s.state.Tick++
   419  	return sm.Result{}
   420  }
   421  
   422  func (s *stateMachine) handleGetIDCmd(cmd []byte) sm.Result {
   423  	allocIDCmd := parseAllocateIDCmd(cmd)
   424  	// Empty key means it is a shared ID.
   425  	if len(allocIDCmd.Key) == 0 {
   426  		s.state.NextID++
   427  		v := s.state.NextID
   428  		s.state.NextID += allocIDCmd.Batch - 1
   429  		return sm.Result{Value: v}
   430  	}
   431  
   432  	_, ok := s.state.NextIDByKey[allocIDCmd.Key]
   433  	if !ok {
   434  		s.state.NextIDByKey[allocIDCmd.Key] = 0
   435  	}
   436  	s.state.NextIDByKey[allocIDCmd.Key]++
   437  	v := s.state.NextIDByKey[allocIDCmd.Key]
   438  	s.state.NextIDByKey[allocIDCmd.Key] += allocIDCmd.Batch - 1
   439  	return sm.Result{Value: v}
   440  }
   441  
   442  func (s *stateMachine) handleSetStateCmd(cmd []byte) sm.Result {
   443  	re := func() sm.Result {
   444  		data := make([]byte, 4)
   445  		binaryEnc.PutUint32(data, uint32(s.state.State))
   446  		return sm.Result{Data: data}
   447  	}
   448  	defer func() {
   449  		plog.Infof("HAKeeper is in %s state", s.state.State)
   450  	}()
   451  	state := parseSetStateCmd(cmd)
   452  	switch s.state.State {
   453  	case pb.HAKeeperCreated:
   454  		return re()
   455  	case pb.HAKeeperBootstrapping:
   456  		if state == pb.HAKeeperBootstrapCommandsReceived {
   457  			s.state.State = state
   458  			return sm.Result{}
   459  		}
   460  		return re()
   461  	case pb.HAKeeperBootstrapCommandsReceived:
   462  		if state == pb.HAKeeperBootstrapFailed || state == pb.HAKeeperRunning {
   463  			s.state.State = state
   464  			return sm.Result{}
   465  		}
   466  		return re()
   467  	case pb.HAKeeperBootstrapFailed:
   468  		return re()
   469  	case pb.HAKeeperRunning:
   470  		return re()
   471  	default:
   472  		panic("unknown HAKeeper state")
   473  	}
   474  }
   475  
   476  func (s *stateMachine) handleSetTaskSchedulerStateUpdateCmd(cmd []byte) sm.Result {
   477  	re := func() sm.Result {
   478  		data := make([]byte, 4)
   479  		binaryEnc.PutUint32(data, uint32(s.state.TaskSchedulerState))
   480  		return sm.Result{Data: data}
   481  	}
   482  	defer func() {
   483  		plog.Infof("Task scheduler is in %s state", s.state.TaskSchedulerState)
   484  	}()
   485  	state := parseSetInitTaskStateCmd(cmd)
   486  	switch s.state.TaskSchedulerState {
   487  	case pb.TaskSchedulerCreated:
   488  		return re()
   489  	case pb.TaskSchedulerRunning:
   490  		if state == pb.TaskSchedulerStopped {
   491  			s.state.TaskSchedulerState = state
   492  			return sm.Result{}
   493  		}
   494  		return re()
   495  	case pb.TaskSchedulerStopped:
   496  		if state == pb.TaskSchedulerRunning {
   497  			s.state.TaskSchedulerState = state
   498  			return sm.Result{}
   499  		}
   500  		return re()
   501  	default:
   502  		panic("unknown task table init state")
   503  	}
   504  }
   505  
   506  func (s *stateMachine) handleTaskTableUserCmd(cmd []byte) sm.Result {
   507  	result := sm.Result{Value: uint64(s.state.TaskSchedulerState)}
   508  	if s.state.TaskSchedulerState != pb.TaskSchedulerCreated {
   509  		return result
   510  	}
   511  	req := parseTaskTableUserCmd(cmd)
   512  	if req.Username == "" || req.Password == "" {
   513  		panic("task table username and password cannot be null")
   514  	}
   515  
   516  	s.state.TaskTableUser = req
   517  	plog.Infof("task table user set, TaskSchedulerState in TaskSchedulerRunning state")
   518  
   519  	s.state.TaskSchedulerState = pb.TaskSchedulerRunning
   520  	return result
   521  }
   522  
   523  func (s *stateMachine) handleDeleteCNCmd(uuid string) sm.Result {
   524  	deletedTimeout := time.Hour * 24 * 7
   525  	var pos int
   526  	for _, store := range s.state.DeletedStores {
   527  		if time.Now().UnixNano()-store.DownTime > int64(deletedTimeout) {
   528  			pos++
   529  		}
   530  	}
   531  	s.state.DeletedStores = s.state.DeletedStores[pos:]
   532  	if store, ok := s.state.CNState.Stores[uuid]; ok {
   533  		delete(s.state.CNState.Stores, uuid)
   534  		var addr string
   535  		addrItems := strings.Split(store.SQLAddress, ":")
   536  		if len(addrItems) > 1 {
   537  			addr = addrItems[0]
   538  		}
   539  		s.state.DeletedStores = append(s.state.DeletedStores, pb.DeletedStore{
   540  			UUID:      uuid,
   541  			StoreType: "CN",
   542  			Address:   addr,
   543  			UpTime:    store.UpTime,
   544  			DownTime:  time.Now().UnixNano(),
   545  		})
   546  	}
   547  	return sm.Result{}
   548  }
   549  
   550  func (s *stateMachine) handleDeleteProxyCmd(uuid string) sm.Result {
   551  	delete(s.state.ProxyState.Stores, uuid)
   552  	return sm.Result{}
   553  }
   554  
   555  func (s *stateMachine) handleProxyHeartbeat(cmd []byte) sm.Result {
   556  	data := parseHeartbeatCmd(cmd)
   557  	var hb pb.ProxyHeartbeat
   558  	if err := hb.Unmarshal(data); err != nil {
   559  		panic(err)
   560  	}
   561  	s.state.ProxyState.Update(hb, s.state.Tick)
   562  	return s.getCommandBatch(hb.UUID)
   563  }
   564  
   565  // FIXME: NextID should be set to K8SIDRangeEnd once HAKeeper state is
   566  // set to HAKeeperBootstrapping.
   567  func (s *stateMachine) handleInitialClusterRequestCmd(cmd []byte) sm.Result {
   568  	result := sm.Result{Value: uint64(s.state.State)}
   569  	if s.state.State != pb.HAKeeperCreated {
   570  		return result
   571  	}
   572  	req := parseInitialClusterRequestCmd(cmd)
   573  	if req.NumOfLogShards != req.NumOfTNShards {
   574  		panic("DN:Log 1:1 mode is the only supported mode")
   575  	}
   576  
   577  	tnShards := make([]metadata.TNShardRecord, 0)
   578  	logShards := make([]metadata.LogShardRecord, 0)
   579  	// HAKeeper shard is assigned ShardID 0
   580  	rec := metadata.LogShardRecord{
   581  		ShardID:          0,
   582  		NumberOfReplicas: req.NumOfLogReplicas,
   583  	}
   584  	logShards = append(logShards, rec)
   585  
   586  	s.state.NextID++
   587  	for i := uint64(0); i < req.NumOfLogShards; i++ {
   588  		rec := metadata.LogShardRecord{
   589  			ShardID:          s.state.NextID,
   590  			NumberOfReplicas: req.NumOfLogReplicas,
   591  		}
   592  		s.state.NextID++
   593  		logShards = append(logShards, rec)
   594  
   595  		drec := metadata.TNShardRecord{
   596  			ShardID:    s.state.NextID,
   597  			LogShardID: rec.ShardID,
   598  		}
   599  		s.state.NextID++
   600  		tnShards = append(tnShards, drec)
   601  	}
   602  	s.state.ClusterInfo = pb.ClusterInfo{
   603  		TNShards:  tnShards,
   604  		LogShards: logShards,
   605  	}
   606  
   607  	// make sure we are not using the ID range assigned to k8s
   608  	if s.state.NextID > K8SIDRangeStart {
   609  		panic("too many IDs assigned during initial cluster request")
   610  	}
   611  	if req.NextID > K8SIDRangeEnd {
   612  		s.state.NextID = req.NextID
   613  	} else {
   614  		s.state.NextID = K8SIDRangeEnd
   615  	}
   616  	if req.NextIDByKey != nil && len(req.NextIDByKey) > 0 {
   617  		s.state.NextIDByKey = req.NextIDByKey
   618  	}
   619  
   620  	plog.Infof("initial cluster set, HAKeeper is in BOOTSTRAPPING state")
   621  	s.state.State = pb.HAKeeperBootstrapping
   622  	return result
   623  }
   624  
   625  func (s *stateMachine) assertState() {
   626  	if s.state.State != pb.HAKeeperRunning && s.state.State != pb.HAKeeperBootstrapping {
   627  		panic(fmt.Sprintf("HAKeeper not in the running state, in %s", s.state.State.String()))
   628  	}
   629  }
   630  
   631  func (s *stateMachine) Update(e sm.Entry) (sm.Result, error) {
   632  	// TODO: we need to make sure InitialClusterRequestCmd is the
   633  	// first user cmd added to the Raft log
   634  	cmd := e.Cmd
   635  	s.state.Index = e.Index
   636  	switch parseCmdTag(cmd) {
   637  	case pb.TNHeartbeatUpdate:
   638  		return s.handleTNHeartbeat(cmd), nil
   639  	case pb.CNHeartbeatUpdate:
   640  		return s.handleCNHeartbeat(cmd), nil
   641  	case pb.LogHeartbeatUpdate:
   642  		return s.handleLogHeartbeat(cmd), nil
   643  	case pb.TickUpdate:
   644  		return s.handleTick(cmd), nil
   645  	case pb.GetIDUpdate:
   646  		s.assertState()
   647  		return s.handleGetIDCmd(cmd), nil
   648  	case pb.ScheduleCommandUpdate:
   649  		return s.handleUpdateCommandsCmd(cmd), nil
   650  	case pb.SetStateUpdate:
   651  		return s.handleSetStateCmd(cmd), nil
   652  	case pb.SetTaskSchedulerStateUpdate:
   653  		s.assertState()
   654  		return s.handleSetTaskSchedulerStateUpdateCmd(cmd), nil
   655  	case pb.InitialClusterUpdate:
   656  		return s.handleInitialClusterRequestCmd(cmd), nil
   657  	case pb.SetTaskTableUserUpdate:
   658  		s.assertState()
   659  		return s.handleTaskTableUserCmd(cmd), nil
   660  	case pb.UpdateCNLabel:
   661  		return s.handleUpdateCNLabel(cmd), nil
   662  	case pb.UpdateCNWorkState:
   663  		return s.handleUpdateCNWorkState(cmd), nil
   664  	case pb.PatchCNStore:
   665  		return s.handlePatchCNStore(cmd), nil
   666  	case pb.RemoveCNStore:
   667  		return s.handleDeleteCNCmd(parseDeleteCNStoreCmd(cmd).StoreID), nil
   668  	case pb.ProxyHeartbeatUpdate:
   669  		return s.handleProxyHeartbeat(cmd), nil
   670  	default:
   671  		panic(moerr.NewInvalidInputNoCtx("unknown haKeeper cmd '%v'", cmd))
   672  	}
   673  }
   674  
   675  func (s *stateMachine) handleStateQuery() interface{} {
   676  	internal := &pb.CheckerState{
   677  		Tick:               s.state.Tick,
   678  		ClusterInfo:        s.state.ClusterInfo,
   679  		TNState:            s.state.TNState,
   680  		LogState:           s.state.LogState,
   681  		CNState:            s.state.CNState,
   682  		ProxyState:         s.state.ProxyState,
   683  		State:              s.state.State,
   684  		TaskSchedulerState: s.state.TaskSchedulerState,
   685  		TaskTableUser:      s.state.TaskTableUser,
   686  		NextId:             s.state.NextID,
   687  		NextIDByKey:        s.state.NextIDByKey,
   688  	}
   689  	copied := deepcopy.Copy(internal)
   690  	result, ok := copied.(*pb.CheckerState)
   691  	if !ok {
   692  		panic("deep copy failed")
   693  	}
   694  	return result
   695  }
   696  
   697  func (s *stateMachine) handleScheduleCommandQuery(uuid string) *pb.CommandBatch {
   698  	if batch, ok := s.state.ScheduleCommands[uuid]; ok {
   699  		return &batch
   700  	}
   701  	return &pb.CommandBatch{}
   702  }
   703  
   704  func (s *stateMachine) handleClusterDetailsQuery(cfg Config) *pb.ClusterDetails {
   705  	cfg.Fill()
   706  	cd := &pb.ClusterDetails{
   707  		CNStores:    make([]pb.CNStore, 0, len(s.state.CNState.Stores)),
   708  		TNStores:    make([]pb.TNStore, 0, len(s.state.TNState.Stores)),
   709  		LogStores:   make([]pb.LogStore, 0, len(s.state.LogState.Stores)),
   710  		ProxyStores: make([]pb.ProxyStore, 0, len(s.state.ProxyState.Stores)),
   711  	}
   712  	for uuid, info := range s.state.CNState.Stores {
   713  		state := pb.NormalState
   714  		if cfg.CNStoreExpired(info.Tick, s.state.Tick) {
   715  			state = pb.TimeoutState
   716  		}
   717  		n := pb.CNStore{
   718  			UUID:               uuid,
   719  			Tick:               info.Tick,
   720  			ServiceAddress:     info.ServiceAddress,
   721  			SQLAddress:         info.SQLAddress,
   722  			LockServiceAddress: info.LockServiceAddress,
   723  			State:              state,
   724  			WorkState:          info.WorkState,
   725  			Labels:             info.Labels,
   726  			QueryAddress:       info.QueryAddress,
   727  			ConfigData:         info.ConfigData,
   728  			Resource:           info.Resource,
   729  			UpTime:             info.UpTime,
   730  		}
   731  		cd.CNStores = append(cd.CNStores, n)
   732  	}
   733  	for uuid, info := range s.state.TNState.Stores {
   734  		state := pb.NormalState
   735  		if cfg.TNStoreExpired(info.Tick, s.state.Tick) {
   736  			state = pb.TimeoutState
   737  		}
   738  		n := pb.TNStore{
   739  			UUID:                 uuid,
   740  			Tick:                 info.Tick,
   741  			State:                state,
   742  			ServiceAddress:       info.ServiceAddress,
   743  			Shards:               info.Shards,
   744  			LogtailServerAddress: info.LogtailServerAddress,
   745  			LockServiceAddress:   info.LockServiceAddress,
   746  			ConfigData:           info.ConfigData,
   747  			QueryAddress:         info.QueryAddress,
   748  		}
   749  		cd.TNStores = append(cd.TNStores, n)
   750  	}
   751  	for uuid, info := range s.state.LogState.Stores {
   752  		state := pb.NormalState
   753  		if cfg.LogStoreExpired(info.Tick, s.state.Tick) {
   754  			state = pb.TimeoutState
   755  		}
   756  		n := pb.LogStore{
   757  			UUID:           uuid,
   758  			Tick:           info.Tick,
   759  			State:          state,
   760  			ServiceAddress: info.ServiceAddress,
   761  			Replicas:       info.Replicas,
   762  			ConfigData:     info.ConfigData,
   763  		}
   764  		cd.LogStores = append(cd.LogStores, n)
   765  	}
   766  	for uuid, info := range s.state.ProxyState.Stores {
   767  		cd.ProxyStores = append(cd.ProxyStores, pb.ProxyStore{
   768  			UUID:          uuid,
   769  			Tick:          info.Tick,
   770  			ListenAddress: info.ListenAddress,
   771  			ConfigData:    info.ConfigData,
   772  		})
   773  	}
   774  	for _, store := range s.state.DeletedStores {
   775  		cd.DeletedStores = append(cd.DeletedStores, pb.DeletedStore{
   776  			UUID:      store.UUID,
   777  			StoreType: store.StoreType,
   778  			Address:   store.Address,
   779  			UpTime:    store.UpTime,
   780  			DownTime:  store.DownTime,
   781  		})
   782  	}
   783  	return cd
   784  }
   785  
   786  func (s *stateMachine) handleUpdateCNLabel(cmd []byte) sm.Result {
   787  	s.state.CNState.UpdateLabel(parseUpdateCNLabelCmd(cmd))
   788  	return sm.Result{}
   789  }
   790  
   791  func (s *stateMachine) handleUpdateCNWorkState(cmd []byte) sm.Result {
   792  	s.state.CNState.UpdateWorkState(parseUpdateCNWorkStateCmd(cmd))
   793  	return sm.Result{}
   794  }
   795  
   796  func (s *stateMachine) handlePatchCNStore(cmd []byte) sm.Result {
   797  	s.state.CNState.PatchCNStore(parsePatchCNStoreCmd(cmd))
   798  	return sm.Result{}
   799  }
   800  
   801  func (s *stateMachine) Lookup(query interface{}) (interface{}, error) {
   802  	if _, ok := query.(*StateQuery); ok {
   803  		return s.handleStateQuery(), nil
   804  	} else if q, ok := query.(*ScheduleCommandQuery); ok {
   805  		return s.handleScheduleCommandQuery(q.UUID), nil
   806  	} else if q, ok := query.(*ClusterDetailsQuery); ok {
   807  		return s.handleClusterDetailsQuery(q.Cfg), nil
   808  	} else if _, ok := query.(*IndexQuery); ok {
   809  		return s.state.Index, nil
   810  	}
   811  	panic("unknown query type")
   812  }
   813  
   814  func (s *stateMachine) SaveSnapshot(w io.Writer,
   815  	_ sm.ISnapshotFileCollection, _ <-chan struct{}) error {
   816  	// FIXME: memory recycling when necessary
   817  	data := make([]byte, s.state.Size())
   818  	n, err := s.state.MarshalToSizedBuffer(data)
   819  	if err != nil {
   820  		return err
   821  	}
   822  	_, err = w.Write(data[:n])
   823  	return err
   824  }
   825  
   826  func (s *stateMachine) RecoverFromSnapshot(r io.Reader,
   827  	_ []sm.SnapshotFile, _ <-chan struct{}) error {
   828  	data, err := io.ReadAll(r)
   829  	if err != nil {
   830  		return err
   831  	}
   832  	return s.state.Unmarshal(data)
   833  }