github.com/matrixorigin/matrixone@v0.7.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 "encoding/gob" 23 "fmt" 24 "io" 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 StateQuery struct{} 62 type ScheduleCommandQuery struct{ UUID string } 63 type ClusterDetailsQuery struct{ Cfg Config } 64 65 type stateMachine struct { 66 replicaID uint64 67 state pb.HAKeeperRSMState 68 } 69 70 func parseCmdTag(cmd []byte) pb.HAKeeperUpdateType { 71 return pb.HAKeeperUpdateType(binaryEnc.Uint32(cmd)) 72 } 73 74 func GetInitialClusterRequestCmd(numOfLogShards uint64, 75 numOfDNShards uint64, numOfLogReplicas uint64) []byte { 76 req := pb.InitialClusterRequest{ 77 NumOfLogShards: numOfLogShards, 78 NumOfDNShards: numOfDNShards, 79 NumOfLogReplicas: numOfLogReplicas, 80 } 81 payload, err := req.Marshal() 82 if err != nil { 83 panic(err) 84 } 85 cmd := make([]byte, headerSize+len(payload)) 86 binaryEnc.PutUint32(cmd, uint32(pb.InitialClusterUpdate)) 87 copy(cmd[headerSize:], payload) 88 return cmd 89 } 90 91 func parseInitialClusterRequestCmd(cmd []byte) pb.InitialClusterRequest { 92 if parseCmdTag(cmd) != pb.InitialClusterUpdate { 93 panic("not a initial cluster update") 94 } 95 payload := cmd[headerSize:] 96 var result pb.InitialClusterRequest 97 if err := result.Unmarshal(payload); err != nil { 98 panic(err) 99 } 100 return result 101 } 102 103 func parseTaskTableUserCmd(cmd []byte) pb.TaskTableUser { 104 if parseCmdTag(cmd) != pb.SetTaskTableUserUpdate { 105 panic("not a task table user update") 106 } 107 payload := cmd[headerSize:] 108 var result pb.TaskTableUser 109 if err := result.Unmarshal(payload); err != nil { 110 panic(err) 111 } 112 return result 113 } 114 115 func GetUpdateCommandsCmd(term uint64, cmds []pb.ScheduleCommand) []byte { 116 b := pb.CommandBatch{ 117 Term: term, 118 Commands: cmds, 119 } 120 data := make([]byte, headerSize+b.Size()) 121 binaryEnc.PutUint32(data, uint32(pb.ScheduleCommandUpdate)) 122 if _, err := b.MarshalTo(data[headerSize:]); err != nil { 123 panic(err) 124 } 125 return data 126 } 127 128 func GetGetIDCmd(count uint64) []byte { 129 cmd := make([]byte, headerSize+8) 130 binaryEnc.PutUint32(cmd, uint32(pb.GetIDUpdate)) 131 binaryEnc.PutUint64(cmd[headerSize:], count) 132 return cmd 133 } 134 135 func parseHeartbeatCmd(cmd []byte) []byte { 136 return cmd[headerSize:] 137 } 138 139 func parseGetIDCmd(cmd []byte) uint64 { 140 return binaryEnc.Uint64(cmd[headerSize:]) 141 } 142 143 func parseSetStateCmd(cmd []byte) pb.HAKeeperState { 144 return pb.HAKeeperState(binaryEnc.Uint32(cmd[headerSize:])) 145 } 146 147 func parseSetInitTaskStateCmd(cmd []byte) pb.TaskSchedulerState { 148 return pb.TaskSchedulerState(binaryEnc.Uint32(cmd[headerSize:])) 149 } 150 151 func GetSetStateCmd(state pb.HAKeeperState) []byte { 152 cmd := make([]byte, headerSize+4) 153 binaryEnc.PutUint32(cmd, uint32(pb.SetStateUpdate)) 154 binaryEnc.PutUint32(cmd[headerSize:], uint32(state)) 155 return cmd 156 } 157 158 func GetSetTaskSchedulerStateCmd(state pb.TaskSchedulerState) []byte { 159 cmd := make([]byte, headerSize+4) 160 binaryEnc.PutUint32(cmd, uint32(pb.SetTaskSchedulerStateUpdate)) 161 binaryEnc.PutUint32(cmd[headerSize:], uint32(state)) 162 return cmd 163 } 164 165 func GetTaskTableUserCmd(user pb.TaskTableUser) []byte { 166 cmd := make([]byte, headerSize+user.Size()) 167 binaryEnc.PutUint32(cmd, uint32(pb.SetTaskTableUserUpdate)) 168 if _, err := user.MarshalTo(cmd[headerSize:]); err != nil { 169 panic(err) 170 } 171 return cmd 172 } 173 174 func GetTickCmd() []byte { 175 cmd := make([]byte, headerSize) 176 binaryEnc.PutUint32(cmd, uint32(pb.TickUpdate)) 177 return cmd 178 } 179 180 func GetLogStoreHeartbeatCmd(data []byte) []byte { 181 return getHeartbeatCmd(data, pb.LogHeartbeatUpdate) 182 } 183 184 func GetCNStoreHeartbeatCmd(data []byte) []byte { 185 return getHeartbeatCmd(data, pb.CNHeartbeatUpdate) 186 } 187 188 func GetDNStoreHeartbeatCmd(data []byte) []byte { 189 return getHeartbeatCmd(data, pb.DNHeartbeatUpdate) 190 } 191 192 func getHeartbeatCmd(data []byte, tag pb.HAKeeperUpdateType) []byte { 193 cmd := make([]byte, headerSize+len(data)) 194 binaryEnc.PutUint32(cmd, uint32(tag)) 195 copy(cmd[headerSize:], data) 196 return cmd 197 } 198 199 func NewStateMachine(shardID uint64, replicaID uint64) sm.IStateMachine { 200 if shardID != DefaultHAKeeperShardID { 201 panic(moerr.NewInvalidInputNoCtx("HAKeeper shard ID %d does not match DefaultHAKeeperShardID %d", shardID, DefaultHAKeeperShardID)) 202 } 203 return &stateMachine{ 204 replicaID: replicaID, 205 state: pb.NewRSMState(), 206 } 207 } 208 209 func (s *stateMachine) Close() error { 210 return nil 211 } 212 213 func (s *stateMachine) assignID() uint64 { 214 s.state.NextID++ 215 return s.state.NextID 216 } 217 218 func (s *stateMachine) handleUpdateCommandsCmd(cmd []byte) sm.Result { 219 data := cmd[headerSize:] 220 var b pb.CommandBatch 221 if err := b.Unmarshal(data); err != nil { 222 panic(err) 223 } 224 if s.state.Term > b.Term { 225 return sm.Result{} 226 } 227 228 for _, c := range b.Commands { 229 if c.Bootstrapping { 230 if s.state.State != pb.HAKeeperBootstrapping { 231 plog.Errorf("ignored bootstrapping cmd: %s", c.LogString()) 232 return sm.Result{} 233 } 234 } 235 } 236 237 s.state.Term = b.Term 238 s.state.ScheduleCommands = make(map[string]pb.CommandBatch) 239 for _, c := range b.Commands { 240 if c.Bootstrapping { 241 s.handleSetStateCmd(GetSetStateCmd(pb.HAKeeperBootstrapCommandsReceived)) 242 } 243 if c.DeleteCNStore != nil { 244 s.handleDeleteCNCmd(c.UUID) 245 continue 246 } 247 l, ok := s.state.ScheduleCommands[c.UUID] 248 if !ok { 249 l = pb.CommandBatch{ 250 Commands: make([]pb.ScheduleCommand, 0), 251 } 252 } 253 plog.Infof("adding schedule command to hakeeper rsm: %s", c.LogString()) 254 l.Commands = append(l.Commands, c) 255 s.state.ScheduleCommands[c.UUID] = l 256 } 257 258 return sm.Result{} 259 } 260 261 func (s *stateMachine) getCommandBatch(uuid string) sm.Result { 262 if batch, ok := s.state.ScheduleCommands[uuid]; ok { 263 delete(s.state.ScheduleCommands, uuid) 264 data, err := batch.Marshal() 265 if err != nil { 266 panic(err) 267 } 268 return sm.Result{Data: data} 269 } 270 return sm.Result{} 271 272 } 273 274 func (s *stateMachine) handleCNHeartbeat(cmd []byte) sm.Result { 275 data := parseHeartbeatCmd(cmd) 276 var hb pb.CNStoreHeartbeat 277 if err := hb.Unmarshal(data); err != nil { 278 panic(err) 279 } 280 s.state.CNState.Update(hb, s.state.Tick) 281 return s.getCommandBatch(hb.UUID) 282 } 283 284 func (s *stateMachine) handleDNHeartbeat(cmd []byte) sm.Result { 285 data := parseHeartbeatCmd(cmd) 286 var hb pb.DNStoreHeartbeat 287 if err := hb.Unmarshal(data); err != nil { 288 panic(err) 289 } 290 s.state.DNState.Update(hb, s.state.Tick) 291 return s.getCommandBatch(hb.UUID) 292 } 293 294 func (s *stateMachine) handleLogHeartbeat(cmd []byte) sm.Result { 295 data := parseHeartbeatCmd(cmd) 296 var hb pb.LogStoreHeartbeat 297 if err := hb.Unmarshal(data); err != nil { 298 panic(err) 299 } 300 s.state.LogState.Update(hb, s.state.Tick) 301 return s.getCommandBatch(hb.UUID) 302 } 303 304 func (s *stateMachine) handleTick(cmd []byte) sm.Result { 305 s.state.Tick++ 306 return sm.Result{} 307 } 308 309 func (s *stateMachine) handleGetIDCmd(cmd []byte) sm.Result { 310 count := parseGetIDCmd(cmd) 311 s.state.NextID++ 312 v := s.state.NextID 313 s.state.NextID += count - 1 314 return sm.Result{Value: v} 315 } 316 317 func (s *stateMachine) handleSetStateCmd(cmd []byte) sm.Result { 318 re := func() sm.Result { 319 data := make([]byte, 4) 320 binaryEnc.PutUint32(data, uint32(s.state.State)) 321 return sm.Result{Data: data} 322 } 323 defer func() { 324 plog.Infof("HAKeeper is in %s state", s.state.State) 325 }() 326 state := parseSetStateCmd(cmd) 327 switch s.state.State { 328 case pb.HAKeeperCreated: 329 return re() 330 case pb.HAKeeperBootstrapping: 331 if state == pb.HAKeeperBootstrapCommandsReceived { 332 s.state.State = state 333 return sm.Result{} 334 } 335 return re() 336 case pb.HAKeeperBootstrapCommandsReceived: 337 if state == pb.HAKeeperBootstrapFailed || state == pb.HAKeeperRunning { 338 s.state.State = state 339 return sm.Result{} 340 } 341 return re() 342 case pb.HAKeeperBootstrapFailed: 343 return re() 344 case pb.HAKeeperRunning: 345 return re() 346 default: 347 panic("unknown HAKeeper state") 348 } 349 } 350 351 func (s *stateMachine) handleSetTaskSchedulerStateUpdateCmd(cmd []byte) sm.Result { 352 re := func() sm.Result { 353 data := make([]byte, 4) 354 binaryEnc.PutUint32(data, uint32(s.state.TaskSchedulerState)) 355 return sm.Result{Data: data} 356 } 357 defer func() { 358 plog.Infof("Task scheduler is in %s state", s.state.TaskSchedulerState) 359 }() 360 state := parseSetInitTaskStateCmd(cmd) 361 switch s.state.TaskSchedulerState { 362 case pb.TaskSchedulerCreated: 363 return re() 364 case pb.TaskSchedulerRunning: 365 if state == pb.TaskSchedulerStopped { 366 s.state.TaskSchedulerState = state 367 return sm.Result{} 368 } 369 return re() 370 case pb.TaskSchedulerStopped: 371 if state == pb.TaskSchedulerRunning { 372 s.state.TaskSchedulerState = state 373 return sm.Result{} 374 } 375 return re() 376 default: 377 panic("unknown task table init state") 378 } 379 } 380 381 func (s *stateMachine) handleTaskTableUserCmd(cmd []byte) sm.Result { 382 result := sm.Result{Value: uint64(s.state.TaskSchedulerState)} 383 if s.state.TaskSchedulerState != pb.TaskSchedulerCreated { 384 return result 385 } 386 req := parseTaskTableUserCmd(cmd) 387 if req.Username == "" || req.Password == "" { 388 panic("task table username and password cannot be null") 389 } 390 391 s.state.TaskTableUser = req 392 plog.Infof("task table user set, TaskSchedulerState in TaskSchedulerRunning state") 393 394 s.state.TaskSchedulerState = pb.TaskSchedulerRunning 395 return result 396 } 397 398 func (s *stateMachine) handleDeleteCNCmd(uuid string) sm.Result { 399 delete(s.state.CNState.Stores, uuid) 400 return sm.Result{} 401 } 402 403 // FIXME: NextID should be set to K8SIDRangeEnd once HAKeeper state is 404 // set to HAKeeperBootstrapping. 405 func (s *stateMachine) handleInitialClusterRequestCmd(cmd []byte) sm.Result { 406 result := sm.Result{Value: uint64(s.state.State)} 407 if s.state.State != pb.HAKeeperCreated { 408 return result 409 } 410 req := parseInitialClusterRequestCmd(cmd) 411 if req.NumOfLogShards != req.NumOfDNShards { 412 panic("DN:Log 1:1 mode is the only supported mode") 413 } 414 415 dnShards := make([]metadata.DNShardRecord, 0) 416 logShards := make([]metadata.LogShardRecord, 0) 417 // HAKeeper shard is assigned ShardID 0 418 rec := metadata.LogShardRecord{ 419 ShardID: 0, 420 NumberOfReplicas: req.NumOfLogReplicas, 421 } 422 logShards = append(logShards, rec) 423 424 s.state.NextID++ 425 for i := uint64(0); i < req.NumOfLogShards; i++ { 426 rec := metadata.LogShardRecord{ 427 ShardID: s.state.NextID, 428 NumberOfReplicas: req.NumOfLogReplicas, 429 } 430 s.state.NextID++ 431 logShards = append(logShards, rec) 432 433 drec := metadata.DNShardRecord{ 434 ShardID: s.state.NextID, 435 LogShardID: rec.ShardID, 436 } 437 s.state.NextID++ 438 dnShards = append(dnShards, drec) 439 } 440 s.state.ClusterInfo = pb.ClusterInfo{ 441 DNShards: dnShards, 442 LogShards: logShards, 443 } 444 445 // make sure we are not using the ID range assigned to k8s 446 if s.state.NextID > K8SIDRangeStart { 447 panic("too many IDs assigned during initial cluster request") 448 } 449 s.state.NextID = K8SIDRangeEnd 450 451 plog.Infof("initial cluster set, HAKeeper is in BOOTSTRAPPING state") 452 s.state.State = pb.HAKeeperBootstrapping 453 return result 454 } 455 456 func (s *stateMachine) assertState() { 457 if s.state.State != pb.HAKeeperRunning && s.state.State != pb.HAKeeperBootstrapping { 458 panic(fmt.Sprintf("HAKeeper not in the running state, in %s", s.state.State.String())) 459 } 460 } 461 462 func (s *stateMachine) Update(e sm.Entry) (sm.Result, error) { 463 // TODO: we need to make sure InitialClusterRequestCmd is the 464 // first user cmd added to the Raft log 465 cmd := e.Cmd 466 switch parseCmdTag(cmd) { 467 case pb.DNHeartbeatUpdate: 468 return s.handleDNHeartbeat(cmd), nil 469 case pb.CNHeartbeatUpdate: 470 return s.handleCNHeartbeat(cmd), nil 471 case pb.LogHeartbeatUpdate: 472 return s.handleLogHeartbeat(cmd), nil 473 case pb.TickUpdate: 474 return s.handleTick(cmd), nil 475 case pb.GetIDUpdate: 476 s.assertState() 477 return s.handleGetIDCmd(cmd), nil 478 case pb.ScheduleCommandUpdate: 479 return s.handleUpdateCommandsCmd(cmd), nil 480 case pb.SetStateUpdate: 481 return s.handleSetStateCmd(cmd), nil 482 case pb.SetTaskSchedulerStateUpdate: 483 s.assertState() 484 return s.handleSetTaskSchedulerStateUpdateCmd(cmd), nil 485 case pb.InitialClusterUpdate: 486 return s.handleInitialClusterRequestCmd(cmd), nil 487 case pb.SetTaskTableUserUpdate: 488 s.assertState() 489 return s.handleTaskTableUserCmd(cmd), nil 490 default: 491 panic(moerr.NewInvalidInputNoCtx("unknown haKeeper cmd '%v'", cmd)) 492 } 493 } 494 495 func (s *stateMachine) handleStateQuery() interface{} { 496 internal := &pb.CheckerState{ 497 Tick: s.state.Tick, 498 ClusterInfo: s.state.ClusterInfo, 499 DNState: s.state.DNState, 500 LogState: s.state.LogState, 501 CNState: s.state.CNState, 502 State: s.state.State, 503 TaskSchedulerState: s.state.TaskSchedulerState, 504 TaskTableUser: s.state.TaskTableUser, 505 } 506 copied := deepcopy.Copy(internal) 507 result, ok := copied.(*pb.CheckerState) 508 if !ok { 509 panic("deep copy failed") 510 } 511 return result 512 } 513 514 func (s *stateMachine) handleScheduleCommandQuery(uuid string) *pb.CommandBatch { 515 if batch, ok := s.state.ScheduleCommands[uuid]; ok { 516 return &batch 517 } 518 return &pb.CommandBatch{} 519 } 520 521 func (s *stateMachine) handleClusterDetailsQuery(cfg Config) *pb.ClusterDetails { 522 cfg.Fill() 523 cd := &pb.ClusterDetails{ 524 CNStores: make([]pb.CNStore, 0, len(s.state.CNState.Stores)), 525 DNStores: make([]pb.DNStore, 0, len(s.state.DNState.Stores)), 526 LogStores: make([]pb.LogStore, 0, len(s.state.LogState.Stores)), 527 } 528 for uuid, info := range s.state.CNState.Stores { 529 state := pb.NormalState 530 if cfg.CNStoreExpired(info.Tick, s.state.Tick) { 531 state = pb.TimeoutState 532 } 533 n := pb.CNStore{ 534 UUID: uuid, 535 Tick: info.Tick, 536 ServiceAddress: info.ServiceAddress, 537 SQLAddress: info.SQLAddress, 538 State: state, 539 } 540 cd.CNStores = append(cd.CNStores, n) 541 } 542 for uuid, info := range s.state.DNState.Stores { 543 state := pb.NormalState 544 if cfg.DNStoreExpired(info.Tick, s.state.Tick) { 545 state = pb.TimeoutState 546 } 547 n := pb.DNStore{ 548 UUID: uuid, 549 Tick: info.Tick, 550 State: state, 551 ServiceAddress: info.ServiceAddress, 552 Shards: info.Shards, 553 LogtailServerAddress: info.LogtailServerAddress, 554 } 555 cd.DNStores = append(cd.DNStores, n) 556 } 557 for uuid, info := range s.state.LogState.Stores { 558 state := pb.NormalState 559 if cfg.LogStoreExpired(info.Tick, s.state.Tick) { 560 state = pb.TimeoutState 561 } 562 n := pb.LogStore{ 563 UUID: uuid, 564 Tick: info.Tick, 565 State: state, 566 ServiceAddress: info.ServiceAddress, 567 Replicas: info.Replicas, 568 } 569 cd.LogStores = append(cd.LogStores, n) 570 } 571 return cd 572 } 573 574 func (s *stateMachine) Lookup(query interface{}) (interface{}, error) { 575 if _, ok := query.(*StateQuery); ok { 576 return s.handleStateQuery(), nil 577 } else if q, ok := query.(*ScheduleCommandQuery); ok { 578 return s.handleScheduleCommandQuery(q.UUID), nil 579 } else if q, ok := query.(*ClusterDetailsQuery); ok { 580 return s.handleClusterDetailsQuery(q.Cfg), nil 581 } 582 panic("unknown query type") 583 } 584 585 func (s *stateMachine) SaveSnapshot(w io.Writer, 586 _ sm.ISnapshotFileCollection, _ <-chan struct{}) error { 587 // FIXME: ready to use gogoproto to marshal the state, just need to figure 588 // out how to write to the writer. 589 enc := gob.NewEncoder(w) 590 return enc.Encode(s.state) 591 } 592 593 func (s *stateMachine) RecoverFromSnapshot(r io.Reader, 594 _ []sm.SnapshotFile, _ <-chan struct{}) error { 595 dec := gob.NewDecoder(r) 596 return dec.Decode(&s.state) 597 }