| @@ -1,41 +1,54 @@ | |||
| package cluster | |||
| import ( | |||
| "context" | |||
| "crypto/tls" | |||
| "fmt" | |||
| "os" | |||
| "path/filepath" | |||
| "sync" | |||
| "time" | |||
| "github.com/hashicorp/raft" | |||
| raftboltdb "github.com/hashicorp/raft-boltdb" | |||
| "github.com/samber/lo" | |||
| "gitlink.org.cn/cloudream/common/pkgs/async" | |||
| "gitlink.org.cn/cloudream/common/pkgs/logger" | |||
| "gitlink.org.cn/cloudream/jcs-pub/common/ecode" | |||
| "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" | |||
| clirpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/client" | |||
| "google.golang.org/grpc" | |||
| "google.golang.org/grpc/credentials" | |||
| ) | |||
| type Cluster struct { | |||
| cfg Config | |||
| cliPool *clirpc.Pool | |||
| cfg Config | |||
| poolCfg clirpc.PoolConfig | |||
| masterCli MasterClient | |||
| raft *raft.Raft | |||
| transport *Transport | |||
| doneCh chan any | |||
| } | |||
| func New(cfg *Config) *Cluster { | |||
| c := Config{} | |||
| if cfg != nil { | |||
| c = *cfg | |||
| } else { | |||
| c.IsMaster = true | |||
| } | |||
| func New(cfg Config) *Cluster { | |||
| return &Cluster{ | |||
| cfg: c, | |||
| cfg: cfg, | |||
| masterCli: MasterClient{ | |||
| Client: clirpc.NewFusedClient(rpc.Failed(ecode.OperationFailed, "master unknown")), | |||
| }, | |||
| doneCh: make(chan any, 1), | |||
| } | |||
| } | |||
| func (c *Cluster) Start() error { | |||
| func (c *Cluster) Start() (*ClusterEventChan, error) { | |||
| log := logger.WithField("Mod", "Cluster") | |||
| if c.cfg.IsMaster { | |||
| log.Infof("cluster start as master") | |||
| return nil | |||
| ch := async.NewUnboundChannel[ClusterEvent]() | |||
| if !c.cfg.Enabled { | |||
| log.Infof("cluster disabled") | |||
| return ch, nil | |||
| } | |||
| poolCfgJSON := clirpc.PoolConfigJSON{ | |||
| Address: c.cfg.MasterAddress, | |||
| RootCA: c.cfg.RootCA, | |||
| ClientCert: c.cfg.ClientCert, | |||
| ClientKey: c.cfg.ClientKey, | |||
| @@ -43,40 +56,238 @@ func (c *Cluster) Start() error { | |||
| poolCfg, err := poolCfgJSON.Build() | |||
| if err != nil { | |||
| return err | |||
| return nil, fmt.Errorf("build pool config: %w", err) | |||
| } | |||
| c.poolCfg = *poolCfg | |||
| raftCfg := raft.DefaultConfig() | |||
| raftCfg.LocalID = raft.ServerID(c.cfg.Announce) | |||
| err = os.MkdirAll(c.cfg.StoreBase, 0755) | |||
| if err != nil { | |||
| return nil, fmt.Errorf("create store base dir: %w", err) | |||
| } | |||
| logDB, err := raftboltdb.NewBoltStore(filepath.Join(c.cfg.StoreBase, "log.db")) | |||
| if err != nil { | |||
| return nil, fmt.Errorf("create raft log store: %w", err) | |||
| } | |||
| stableDB, err := raftboltdb.NewBoltStore(filepath.Join(c.cfg.StoreBase, "stable.db")) | |||
| if err != nil { | |||
| return nil, fmt.Errorf("create raft stable store: %w", err) | |||
| } | |||
| snapshotStore, err := raft.NewFileSnapshotStore(c.cfg.StoreBase, 1, os.Stdout) | |||
| if err != nil { | |||
| return nil, fmt.Errorf("create raft snapshot store: %w", err) | |||
| } | |||
| fsm := NewFSM() | |||
| c.transport = NewTransport(c.cfg.Announce, *poolCfg) | |||
| rft, err := raft.NewRaft(raftCfg, fsm, logDB, stableDB, snapshotStore, c.transport) | |||
| if err != nil { | |||
| return nil, fmt.Errorf("create raft: %w", err) | |||
| } | |||
| c.cliPool = clirpc.NewPool(*poolCfg) | |||
| for { | |||
| cli := c.cliPool.Get() | |||
| ctx, cancelFn := context.WithTimeout(context.Background(), 10*time.Second) | |||
| resp, cerr := cli.GetClusterMasterInfo(ctx, &clirpc.GetClusterMasterInfo{}) | |||
| cancelFn() | |||
| if cerr != nil { | |||
| log.Warnf("first report: %v, will retry after 3 seconds", cerr.ToError()) | |||
| time.Sleep(3 * time.Second) | |||
| continue | |||
| hasState, err := raft.HasExistingState(logDB, stableDB, snapshotStore) | |||
| if err != nil { | |||
| return nil, fmt.Errorf("check has existing state: %w", err) | |||
| } | |||
| if !hasState { | |||
| bootCfg := raft.Configuration{} | |||
| if !lo.Contains(c.cfg.Peers, c.cfg.Announce) { | |||
| bootCfg.Servers = append(bootCfg.Servers, raft.Server{ | |||
| ID: raft.ServerID(c.cfg.Announce), | |||
| Address: raft.ServerAddress(c.cfg.Announce), | |||
| }) | |||
| } | |||
| for _, peer := range c.cfg.Peers { | |||
| bootCfg.Servers = append(bootCfg.Servers, raft.Server{ | |||
| ID: raft.ServerID(peer), | |||
| Address: raft.ServerAddress(peer), | |||
| }) | |||
| } | |||
| bootFut := rft.BootstrapCluster(bootCfg) | |||
| if err := bootFut.Error(); err != nil { | |||
| return nil, fmt.Errorf("bootstrap cluster: %w", err) | |||
| } | |||
| log.Infof("bootstrap new cluster") | |||
| } else { | |||
| log.Infof("start existing cluster") | |||
| } | |||
| ch.Send(&BootstrapEvent{}) | |||
| c.raft = rft | |||
| eventCh := make(chan raft.Observation, 1) | |||
| obs := raft.NewObserver(eventCh, true, nil) | |||
| rft.RegisterObserver(obs) | |||
| go func() { | |||
| loop: | |||
| for { | |||
| select { | |||
| case <-c.doneCh: | |||
| break loop | |||
| case e := <-eventCh: | |||
| state, ok := e.Data.(raft.RaftState) | |||
| if !ok { | |||
| continue | |||
| } | |||
| switch state { | |||
| case raft.Leader: | |||
| log.Info("become leader") | |||
| ch.Send(&LeaderEvent{ | |||
| CurrentIsMaster: true, | |||
| Address: c.cfg.Announce, | |||
| }) | |||
| case raft.Follower: | |||
| addr, id := rft.LeaderWithID() | |||
| log.Infof("become follower, master is: %v, %v", id, addr) | |||
| ch.Send(&LeaderEvent{ | |||
| CurrentIsMaster: false, | |||
| Address: string(addr), | |||
| }) | |||
| case raft.Candidate: | |||
| log.Info("become candidate") | |||
| } | |||
| } | |||
| } | |||
| log.Infof("cluster start as slave, master is: %v", resp.Name) | |||
| break | |||
| c.raft.DeregisterObserver(obs) | |||
| }() | |||
| return ch, nil | |||
| } | |||
| func (c *Cluster) Stop() { | |||
| c.raft.Shutdown().Error() | |||
| select { | |||
| case c.doneCh <- nil: | |||
| default: | |||
| } | |||
| return nil | |||
| } | |||
| func (c *Cluster) ID() string { | |||
| return c.cfg.Announce | |||
| } | |||
| func (c *Cluster) IsMaster() bool { | |||
| return c.cfg.IsMaster | |||
| addr, _ := c.raft.LeaderWithID() | |||
| return string(addr) == c.cfg.Announce | |||
| } | |||
| func (c *Cluster) Enabled() bool { | |||
| return c.cfg.Enabled | |||
| } | |||
| func (c *Cluster) Name() string { | |||
| return c.cfg.Name | |||
| return c.cfg.NodeName | |||
| } | |||
| func (c *Cluster) MasterClient() *clirpc.Pool { | |||
| return c.cliPool | |||
| // 由于主节点可能会变化,因此不要缓存MasterClient,每次都重新获取 | |||
| func (c *Cluster) MasterClient() *MasterClient { | |||
| addr, _ := c.raft.LeaderWithID() | |||
| c.masterCli.lock.Lock() | |||
| defer c.masterCli.lock.Unlock() | |||
| addr2 := string(addr) | |||
| if addr2 == "" { | |||
| if c.masterCli.con != nil { | |||
| c.masterCli.con.Close() | |||
| } | |||
| c.masterCli.Client = clirpc.NewFusedClient(rpc.Failed(ecode.ClusterNoMaster, "no master")) | |||
| return &c.masterCli | |||
| } | |||
| if c.masterCli.addr != addr2 { | |||
| if c.masterCli.con != nil { | |||
| c.masterCli.con.Close() | |||
| } | |||
| gcon, err := grpc.NewClient(addr2, grpc.WithTransportCredentials(credentials.NewTLS(&tls.Config{ | |||
| RootCAs: c.poolCfg.Conn.RootCA, | |||
| Certificates: []tls.Certificate{*c.poolCfg.Conn.ClientCert}, | |||
| ServerName: rpc.InternalAPISNIV1, | |||
| NextProtos: []string{"h2"}, | |||
| }))) | |||
| if err != nil { | |||
| c.masterCli.Client = clirpc.NewFusedClient(rpc.Failed(ecode.OperationFailed, "%v", err)) | |||
| c.masterCli.addr = "" | |||
| } else { | |||
| c.masterCli.con = gcon | |||
| c.masterCli.Client = clirpc.NewClient(gcon) | |||
| c.masterCli.addr = addr2 | |||
| } | |||
| } | |||
| return &c.masterCli | |||
| } | |||
| func (c *Cluster) RaftTransport() *Transport { | |||
| return nil | |||
| return c.transport | |||
| } | |||
| // 只有Leader才能调用 | |||
| func (c *Cluster) Apply(data string, timeout time.Duration) error { | |||
| return c.raft.Apply([]byte(data), timeout).Error() | |||
| } | |||
| type ClusterEvent interface { | |||
| IsClusterEvent() bool | |||
| } | |||
| type ClusterEventChan = async.UnboundChannel[ClusterEvent] | |||
| type ExitEvent struct { | |||
| Err error | |||
| } | |||
| func (e *ExitEvent) IsClusterEvent() bool { | |||
| return true | |||
| } | |||
| type BootstrapEvent struct{} | |||
| func (e *BootstrapEvent) IsClusterEvent() bool { | |||
| return true | |||
| } | |||
| type LeaderEvent struct { | |||
| CurrentIsMaster bool | |||
| Address string | |||
| } | |||
| func (e *LeaderEvent) IsClusterEvent() bool { | |||
| return true | |||
| } | |||
| type MasterClient struct { | |||
| *clirpc.Client | |||
| con *grpc.ClientConn | |||
| addr string | |||
| lock sync.Mutex | |||
| } | |||
| func (c *MasterClient) Release() { | |||
| } | |||
| @@ -1,10 +1,12 @@ | |||
| package cluster | |||
| type Config struct { | |||
| IsMaster bool `json:"isMaster"` | |||
| Name string `json:"name"` | |||
| MasterAddress string `json:"masterAddress"` | |||
| RootCA string `json:"rootCA"` | |||
| ClientCert string `json:"clientCert"` | |||
| ClientKey string `json:"clientKey"` | |||
| Enabled bool `json:"enabled"` | |||
| NodeName string `json:"nodeName"` // 节点名称 | |||
| Announce string `json:"announce"` // 其他节点可通过该地址访问该节点,一般是外网地址 | |||
| Peers []string `json:"peers"` // 其他节点的Announce地址列表,可以包含自身的Announce地址 | |||
| RootCA string `json:"rootCA"` | |||
| ClientCert string `json:"clientCert"` | |||
| ClientKey string `json:"clientKey"` | |||
| StoreBase string `json:"storeBase"` // 集群信息的存储路径,必须是一个目录 | |||
| } | |||
| @@ -0,0 +1,67 @@ | |||
| package cluster | |||
| import ( | |||
| "fmt" | |||
| "io" | |||
| "sync" | |||
| "github.com/hashicorp/raft" | |||
| "gitlink.org.cn/cloudream/common/utils/io2" | |||
| ) | |||
| type FSM struct { | |||
| data string | |||
| lock sync.Mutex | |||
| } | |||
| func NewFSM() *FSM { | |||
| return &FSM{data: ""} | |||
| } | |||
| func (f *FSM) Apply(l *raft.Log) interface{} { | |||
| fmt.Printf("log: %v\n", string(l.Data)) | |||
| f.lock.Lock() | |||
| defer f.lock.Unlock() | |||
| f.data = f.data + "\n" + string(l.Data) | |||
| return nil | |||
| } | |||
| func (f *FSM) Snapshot() (raft.FSMSnapshot, error) { | |||
| f.lock.Lock() | |||
| defer f.lock.Unlock() | |||
| return &Snapshot{data: f.data}, nil | |||
| } | |||
| func (f *FSM) Restore(rc io.ReadCloser) error { | |||
| data, err := io.ReadAll(rc) | |||
| if err != nil { | |||
| return err | |||
| } | |||
| f.lock.Lock() | |||
| defer f.lock.Unlock() | |||
| f.data = string(data) | |||
| return nil | |||
| } | |||
| var _ raft.FSM = (*FSM)(nil) | |||
| type Snapshot struct { | |||
| data string | |||
| } | |||
| func (s *Snapshot) Persist(sink raft.SnapshotSink) error { | |||
| err := io2.WriteAll(sink, []byte(s.data)) | |||
| if err != nil { | |||
| return sink.Cancel() | |||
| } | |||
| return sink.Close() | |||
| } | |||
| func (s *Snapshot) Release() { | |||
| } | |||
| @@ -14,14 +14,17 @@ import ( | |||
| ) | |||
| const ( | |||
| CmdAppendEntries = "AppendEntries" | |||
| CmdRequestVote = "RequestVote" | |||
| CmdTimeoutNow = "TimeoutNow" | |||
| CmdAppendEntries = "AppendEntries" | |||
| CmdRequestVote = "RequestVote" | |||
| CmdRequestPreVote = "RequestPreVote" | |||
| CmdTimeoutNow = "TimeoutNow" | |||
| ) | |||
| type Transport struct { | |||
| localAddr string | |||
| shutdownCh chan any | |||
| shutdownLock sync.Mutex | |||
| shutdown bool | |||
| consumeCh chan raft.RPC | |||
| cliPoolCfg clirpc.PoolConfig | |||
| cliPools map[raft.ServerAddress]*clirpc.Pool | |||
| @@ -30,6 +33,36 @@ type Transport struct { | |||
| heartbeatFnLock sync.Mutex | |||
| } | |||
| func NewTransport(localAddr string, cliPoolCfg clirpc.PoolConfig) *Transport { | |||
| return &Transport{ | |||
| localAddr: localAddr, | |||
| shutdownCh: make(chan any), | |||
| consumeCh: make(chan raft.RPC), | |||
| cliPoolCfg: cliPoolCfg, | |||
| cliPools: make(map[raft.ServerAddress]*clirpc.Pool), | |||
| heartbeatFn: nil, | |||
| } | |||
| } | |||
| func (t *Transport) Close() { | |||
| t.shutdownLock.Lock() | |||
| defer t.shutdownLock.Unlock() | |||
| if !t.shutdown { | |||
| close(t.shutdownCh) | |||
| t.shutdown = true | |||
| } | |||
| } | |||
| func (n *Transport) IsClosed() bool { | |||
| select { | |||
| case <-n.shutdownCh: | |||
| return true | |||
| default: | |||
| return false | |||
| } | |||
| } | |||
| func (t *Transport) Consumer() <-chan raft.RPC { | |||
| return t.consumeCh | |||
| } | |||
| @@ -106,6 +139,30 @@ func (t *Transport) RequestVote(id raft.ServerID, target raft.ServerAddress, arg | |||
| return nil | |||
| } | |||
| func (t *Transport) RequestPreVote(id raft.ServerID, target raft.ServerAddress, args *raft.RequestPreVoteRequest, resp *raft.RequestPreVoteResponse) error { | |||
| cli := t.getCli(target) | |||
| reqData, err := serder.ObjectToJSON(args) | |||
| if err != nil { | |||
| return err | |||
| } | |||
| r, cerr := cli.ClusterRaftRPC(context.Background(), &clirpc.ClusterRaftRPC{ | |||
| Type: CmdRequestPreVote, | |||
| Data: string(reqData), | |||
| }) | |||
| if cerr != nil { | |||
| return cerr.ToError() | |||
| } | |||
| err = serder.JSONToObject([]byte(r.Data), resp) | |||
| if err != nil { | |||
| return err | |||
| } | |||
| return nil | |||
| } | |||
| func (t *Transport) InstallSnapshot(id raft.ServerID, target raft.ServerAddress, args *raft.InstallSnapshotRequest, resp *raft.InstallSnapshotResponse, data io.Reader) error { | |||
| cli := t.getCli(target) | |||
| r, cerr := cli.ClusterRaftInstallSnapshot(context.Background(), &clirpc.ClusterRaftInstallSnapshot{ | |||
| @@ -156,7 +213,7 @@ func (t *Transport) TimeoutNow(id raft.ServerID, target raft.ServerAddress, args | |||
| } | |||
| func (t *Transport) OnRPC(req *clirpc.ClusterRaftRPC) (*clirpc.ClusterRaftRPCResp, *rpc.CodeError) { | |||
| respChan := make(chan raft.RPCResponse) | |||
| respChan := make(chan raft.RPCResponse, 1) | |||
| rp := raft.RPC{ | |||
| RespChan: respChan, | |||
| } | |||
| @@ -170,7 +227,7 @@ func (t *Transport) OnRPC(req *clirpc.ClusterRaftRPC) (*clirpc.ClusterRaftRPCRes | |||
| if err != nil { | |||
| return nil, rpc.Failed(ecode.OperationFailed, err.Error()) | |||
| } | |||
| rp.Command = r | |||
| rp.Command = &r | |||
| leaderAddr := r.RPCHeader.Addr | |||
| @@ -187,7 +244,15 @@ func (t *Transport) OnRPC(req *clirpc.ClusterRaftRPC) (*clirpc.ClusterRaftRPCRes | |||
| if err != nil { | |||
| return nil, rpc.Failed(ecode.OperationFailed, err.Error()) | |||
| } | |||
| rp.Command = r | |||
| rp.Command = &r | |||
| case CmdRequestPreVote: | |||
| var r raft.RequestPreVoteRequest | |||
| err := serder.JSONToObject([]byte(req.Data), &r) | |||
| if err != nil { | |||
| return nil, rpc.Failed(ecode.OperationFailed, err.Error()) | |||
| } | |||
| rp.Command = &r | |||
| case CmdTimeoutNow: | |||
| var r raft.TimeoutNowRequest | |||
| @@ -195,7 +260,7 @@ func (t *Transport) OnRPC(req *clirpc.ClusterRaftRPC) (*clirpc.ClusterRaftRPCRes | |||
| if err != nil { | |||
| return nil, rpc.Failed(ecode.OperationFailed, err.Error()) | |||
| } | |||
| rp.Command = r | |||
| rp.Command = &r | |||
| } | |||
| if isHeartbeat { | |||
| @@ -239,7 +304,7 @@ RESP: | |||
| } | |||
| func (t *Transport) OnInstallSnapshot(req *clirpc.ClusterRaftInstallSnapshot) (*clirpc.ClusterRaftInstallSnapshotResp, *rpc.CodeError) { | |||
| respChan := make(chan raft.RPCResponse) | |||
| respChan := make(chan raft.RPCResponse, 1) | |||
| rp := raft.RPC{ | |||
| Command: req.Args, | |||
| Reader: req.Data, | |||
| @@ -267,7 +332,7 @@ func (t *Transport) OnInstallSnapshot(req *clirpc.ClusterRaftInstallSnapshot) (* | |||
| } | |||
| } | |||
| func (t *Transport) getCli(target raft.ServerAddress) *clirpc.Client { | |||
| func (t *Transport) getCli(target raft.ServerAddress) *clirpc.PoolClient { | |||
| t.poolLock.Lock() | |||
| defer t.poolLock.Unlock() | |||
| pool := t.cliPools[target] | |||
| @@ -282,9 +347,10 @@ func (t *Transport) getCli(target raft.ServerAddress) *clirpc.Client { | |||
| } | |||
| var _ raft.Transport = (*Transport)(nil) | |||
| var _ raft.WithPreVote = (*Transport)(nil) | |||
| type AppendPipeline struct { | |||
| cli *clirpc.Client | |||
| cli *clirpc.PoolClient | |||
| consumerCh chan raft.AppendFuture | |||
| done chan any | |||
| closeLock sync.Mutex | |||
| @@ -77,14 +77,6 @@ func serveHTTP(configPath string, opts serveHTTPOptions) { | |||
| stgglb.InitLocal(config.Cfg().Local) | |||
| // 集群模式 | |||
| clster := cluster.New(config.Cfg().Cluster) | |||
| err = clster.Start() | |||
| if err != nil { | |||
| logger.Errorf("start cluster failed, err: %v", err) | |||
| os.Exit(1) | |||
| } | |||
| // 数据库 | |||
| db, err := db.NewDB(&config.Cfg().DB) | |||
| if err != nil { | |||
| @@ -184,6 +176,15 @@ func serveHTTP(configPath string, opts serveHTTPOptions) { | |||
| defer conCol.Stop() | |||
| conCol.CollectInPlace() | |||
| // 集群模式 | |||
| clster := cluster.New(config.Cfg().Cluster) | |||
| clsterCh, err := clster.Start() | |||
| if err != nil { | |||
| logger.Errorf("start cluster: %v", err) | |||
| os.Exit(1) | |||
| } | |||
| defer clster.Stop() | |||
| // 公共锁 | |||
| publock := publock.NewMaster() | |||
| publock.Start() | |||
| @@ -239,7 +240,7 @@ func serveHTTP(configPath string, opts serveHTTPOptions) { | |||
| mntChan := mnt.Start() | |||
| defer mnt.Stop() | |||
| svc := services.NewService(publock, dlder, acStat, uploader, strgSel, spaceMeta, db, evtPub, evtWtchr, mnt, stgPool, spaceSync, tktk, spdStats, accToken) | |||
| svc := services.NewService(publock, dlder, acStat, uploader, strgSel, spaceMeta, db, evtPub, evtWtchr, mnt, stgPool, spaceSync, tktk, spdStats, accToken, clster) | |||
| // HTTP接口 | |||
| httpCfgJSON := config.Cfg().HTTP | |||
| @@ -272,6 +273,7 @@ func serveHTTP(configPath string, opts serveHTTPOptions) { | |||
| evtPubEvt := evtPubChan.Receive() | |||
| evtWtchrEvt := evtWtchrChan.Receive() | |||
| conColEvt := conColChan.Receive() | |||
| clsterEvt := clsterCh.Receive() | |||
| acStatEvt := acStatChan.Receive() | |||
| spaceSyncEvt := spaceSyncChan.Receive() | |||
| // replEvt := replCh.Receive() | |||
| @@ -361,6 +363,23 @@ loop: | |||
| } | |||
| conColEvt = conColChan.Receive() | |||
| case e := <-clsterEvt.Chan(): | |||
| if e.Err != nil { | |||
| logger.Errorf("receive cluster event: %v", err) | |||
| break loop | |||
| } | |||
| switch e := e.Value.(type) { | |||
| case *cluster.ExitEvent: | |||
| if e.Err != nil { | |||
| logger.Errorf("cluster exited with error: %v", e.Err) | |||
| } else { | |||
| logger.Info("cluster exited") | |||
| } | |||
| break loop | |||
| } | |||
| clsterEvt = clsterCh.Receive() | |||
| case e := <-acStatEvt.Chan(): | |||
| if e.Err != nil { | |||
| logger.Errorf("receive access stat event: %v", err) | |||
| @@ -88,14 +88,6 @@ func test(configPath string) { | |||
| stgglb.StandaloneMode = config.Cfg().AccessToken == nil | |||
| // 集群模式 | |||
| clster := cluster.New(config.Cfg().Cluster) | |||
| err = clster.Start() | |||
| if err != nil { | |||
| logger.Errorf("start cluster failed, err: %v", err) | |||
| os.Exit(1) | |||
| } | |||
| // 数据库 | |||
| db, err := db.NewDB(&config.Cfg().DB) | |||
| if err != nil { | |||
| @@ -192,6 +184,15 @@ func test(configPath string) { | |||
| defer conCol.Stop() | |||
| conCol.CollectInPlace() | |||
| // 集群模式 | |||
| clster := cluster.New(config.Cfg().Cluster) | |||
| clsterCh, err := clster.Start() | |||
| if err != nil { | |||
| logger.Errorf("start cluster: %v", err) | |||
| os.Exit(1) | |||
| } | |||
| defer clster.Stop() | |||
| // 公共锁 | |||
| publock := publock.NewMaster() | |||
| publock.Start() | |||
| @@ -225,7 +226,7 @@ func test(configPath string) { | |||
| spaceSyncChan := spaceSync.Start() | |||
| defer spaceSync.Stop() | |||
| svc := services.NewService(publock, dlder, acStat, uploader, strgSel, stgMeta, db, evtPub, evtWtchr, nil, stgPool, spaceSync, nil, spdStats, accToken) | |||
| svc := services.NewService(publock, dlder, acStat, uploader, strgSel, stgMeta, db, evtPub, evtWtchr, nil, stgPool, spaceSync, nil, spdStats, accToken, clster) | |||
| go func() { | |||
| doTest(svc) | |||
| @@ -236,6 +237,7 @@ func test(configPath string) { | |||
| evtPubEvt := evtPubChan.Receive() | |||
| evtWtchrEvt := evtWtchrChan.Receive() | |||
| conColEvt := conColChan.Receive() | |||
| clsterEvt := clsterCh.Receive() | |||
| acStatEvt := acStatChan.Receive() | |||
| spaceSyncEvt := spaceSyncChan.Receive() | |||
| @@ -320,6 +322,23 @@ loop: | |||
| } | |||
| conColEvt = conColChan.Receive() | |||
| case e := <-clsterEvt.Chan(): | |||
| if e.Err != nil { | |||
| logger.Errorf("receive cluster event: %v", err) | |||
| break loop | |||
| } | |||
| switch e := e.Value.(type) { | |||
| case *cluster.ExitEvent: | |||
| if e.Err != nil { | |||
| logger.Errorf("cluster exited with error: %v", e.Err) | |||
| } else { | |||
| logger.Info("cluster exited") | |||
| } | |||
| break loop | |||
| } | |||
| clsterEvt = clsterCh.Receive() | |||
| case e := <-acStatEvt.Chan(): | |||
| if e.Err != nil { | |||
| logger.Errorf("receive access stat event: %v", err) | |||
| @@ -68,14 +68,6 @@ func vfsTest(configPath string, opts serveHTTPOptions) { | |||
| stgglb.StandaloneMode = opts.Standalone || config.Cfg().AccessToken == nil | |||
| // 集群模式 | |||
| clster := cluster.New(config.Cfg().Cluster) | |||
| err = clster.Start() | |||
| if err != nil { | |||
| logger.Errorf("start cluster failed, err: %v", err) | |||
| os.Exit(1) | |||
| } | |||
| // 数据库 | |||
| db, err := db.NewDB(&config.Cfg().DB) | |||
| if err != nil { | |||
| @@ -172,6 +164,15 @@ func vfsTest(configPath string, opts serveHTTPOptions) { | |||
| defer conCol.Stop() | |||
| conCol.CollectInPlace() | |||
| // 集群模式 | |||
| clster := cluster.New(config.Cfg().Cluster) | |||
| clsterCh, err := clster.Start() | |||
| if err != nil { | |||
| logger.Errorf("start cluster: %v", err) | |||
| os.Exit(1) | |||
| } | |||
| defer clster.Stop() | |||
| // 公共锁 | |||
| publock := publock.NewMaster() | |||
| publock.Start() | |||
| @@ -218,7 +219,7 @@ func vfsTest(configPath string, opts serveHTTPOptions) { | |||
| mntChan := mnt.Start() | |||
| defer mnt.Stop() | |||
| svc := services.NewService(publock, dlder, acStat, uploader, strgSel, stgMeta, db, evtPub, evtWtchr, mnt, stgPool, spaceSync, nil, spdStats, accToken) | |||
| svc := services.NewService(publock, dlder, acStat, uploader, strgSel, stgMeta, db, evtPub, evtWtchr, mnt, stgPool, spaceSync, nil, spdStats, accToken, clster) | |||
| // HTTP接口 | |||
| httpCfgJSON := config.Cfg().HTTP | |||
| @@ -255,6 +256,7 @@ func vfsTest(configPath string, opts serveHTTPOptions) { | |||
| evtPubEvt := evtPubChan.Receive() | |||
| evtWtchrEvt := evtWtchrChan.Receive() | |||
| conColEvt := conColChan.Receive() | |||
| clsterEvt := clsterCh.Receive() | |||
| acStatEvt := acStatChan.Receive() | |||
| spaceSyncEvt := spaceSyncChan.Receive() | |||
| httpEvt := httpChan.Receive() | |||
| @@ -341,6 +343,23 @@ loop: | |||
| case connectivity.CollectedEvent: | |||
| } | |||
| case e := <-clsterEvt.Chan(): | |||
| if e.Err != nil { | |||
| logger.Errorf("receive cluster event: %v", err) | |||
| break loop | |||
| } | |||
| switch e := e.Value.(type) { | |||
| case *cluster.ExitEvent: | |||
| if e.Err != nil { | |||
| logger.Errorf("cluster exited with error: %v", e.Err) | |||
| } else { | |||
| logger.Info("cluster exited") | |||
| } | |||
| break loop | |||
| } | |||
| clsterEvt = clsterCh.Receive() | |||
| case e := <-acStatEvt.Chan(): | |||
| if e.Err != nil { | |||
| logger.Errorf("receive access stat event: %v", err) | |||
| @@ -34,7 +34,7 @@ type Config struct { | |||
| RPC rpc.Config `json:"rpc"` | |||
| Mount *mntcfg.Config `json:"mount"` | |||
| AccessToken *accesstoken.Config `json:"accessToken"` | |||
| Cluster *cluster.Config `json:"cluster"` | |||
| Cluster cluster.Config `json:"cluster"` | |||
| } | |||
| var cfg Config | |||
| @@ -27,7 +27,7 @@ func (p *ClusterProxy) Proxy(c *gin.Context) { | |||
| return | |||
| } | |||
| mstCli := p.clster.MasterClient().Get() | |||
| mstCli := p.clster.MasterClient() | |||
| defer mstCli.Release() | |||
| preq := clirpc.HTTPProxyRequest{ | |||
| @@ -0,0 +1,46 @@ | |||
| package http | |||
| import ( | |||
| "context" | |||
| "fmt" | |||
| "net/http" | |||
| "time" | |||
| "github.com/gin-gonic/gin" | |||
| "gitlink.org.cn/cloudream/common/pkgs/logger" | |||
| "gitlink.org.cn/cloudream/jcs-pub/client/internal/http/types" | |||
| cliapi "gitlink.org.cn/cloudream/jcs-pub/client/sdk/api/v1" | |||
| "gitlink.org.cn/cloudream/jcs-pub/common/ecode" | |||
| clirpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/client" | |||
| ) | |||
| type ClusterService struct { | |||
| *Server | |||
| } | |||
| func (s *Server) Cluster() *ClusterService { | |||
| return &ClusterService{ | |||
| Server: s, | |||
| } | |||
| } | |||
| func (s *ClusterService) Status(ctx *gin.Context) { | |||
| log := logger.WithField("HTTP", "Cluster.Status") | |||
| var req cliapi.ClusterStatus | |||
| if err := ctx.ShouldBindQuery(&req); err != nil { | |||
| log.Warnf("binding query: %s", err.Error()) | |||
| ctx.JSON(http.StatusBadRequest, types.Failed(ecode.BadArgument, "missing argument or invalid argument")) | |||
| return | |||
| } | |||
| _, cerr := s.svc.Cluster.MasterClient().ClusterApplyLog(context.TODO(), &clirpc.ClusterApplyLog{ | |||
| Data: fmt.Sprintf("%v: %v", s.svc.Cluster.ID(), time.Now()), | |||
| }) | |||
| if cerr != nil { | |||
| ctx.JSON(http.StatusOK, types.Failed(ecode.ErrorCode(cerr.Code), "%v", cerr.Message)) | |||
| return | |||
| } | |||
| ctx.JSON(http.StatusOK, types.OK(cliapi.ClusterStatus{})) | |||
| } | |||
| @@ -97,4 +97,6 @@ func (s *Server) InitRouters(rt gin.IRoutes, ah *auth.Auth, proxy *proxy.Cluster | |||
| rt.POST(cliapi.PubShardsImportPackagePath, certAuth, s.PubShards().ImportPackage) | |||
| rt.POST(cliapi.SysEventWatchPath, certAuth, s.SysEvent().Watch) | |||
| rt.GET(cliapi.ClusterStatusPath, certAuth, s.Cluster().Status) | |||
| } | |||
| @@ -28,7 +28,7 @@ func WithReason(reason string) AcquireOptionFn { | |||
| type PubLock struct { | |||
| core *Core | |||
| cliCli *clirpc.Client | |||
| cliCli *clirpc.PoolClient | |||
| pubChan clirpc.PubLockMessageChan | |||
| lock sync.Mutex | |||
| acquirings map[string]*acquireInfo | |||
| @@ -43,7 +43,7 @@ func NewMaster() *PubLock { | |||
| } | |||
| } | |||
| func NewSlave(cli *clirpc.Client) *PubLock { | |||
| func NewSlave(cli *clirpc.PoolClient) *PubLock { | |||
| return &PubLock{ | |||
| cliCli: cli, | |||
| acquirings: make(map[string]*acquireInfo), | |||
| @@ -3,6 +3,7 @@ package rpc | |||
| import ( | |||
| "context" | |||
| "gitlink.org.cn/cloudream/jcs-pub/common/ecode" | |||
| "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" | |||
| clirpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/client" | |||
| ) | |||
| @@ -18,3 +19,12 @@ func (s *Service) ClusterRaftRPC(ctx context.Context, msg *clirpc.ClusterRaftRPC | |||
| func (s *Service) ClusterRaftInstallSnapshot(ctx context.Context, msg *clirpc.ClusterRaftInstallSnapshot) (*clirpc.ClusterRaftInstallSnapshotResp, *rpc.CodeError) { | |||
| return s.cluster.RaftTransport().OnInstallSnapshot(msg) | |||
| } | |||
| func (s *Service) ClusterApplyLog(ctx context.Context, msg *clirpc.ClusterApplyLog) (*clirpc.ClusterApplyLogResp, *rpc.CodeError) { | |||
| err := s.cluster.Apply(msg.Data, msg.Timeout) | |||
| if err != nil { | |||
| return nil, rpc.Failed(ecode.OperationFailed, "%v", err) | |||
| } | |||
| return &clirpc.ClusterApplyLogResp{}, nil | |||
| } | |||
| @@ -3,6 +3,7 @@ package services | |||
| import ( | |||
| "gitlink.org.cn/cloudream/jcs-pub/client/internal/accessstat" | |||
| "gitlink.org.cn/cloudream/jcs-pub/client/internal/accesstoken" | |||
| "gitlink.org.cn/cloudream/jcs-pub/client/internal/cluster" | |||
| "gitlink.org.cn/cloudream/jcs-pub/client/internal/db" | |||
| "gitlink.org.cn/cloudream/jcs-pub/client/internal/downloader" | |||
| "gitlink.org.cn/cloudream/jcs-pub/client/internal/downloader/strategy" | |||
| @@ -34,6 +35,7 @@ type Service struct { | |||
| TickTock *ticktock.TickTock | |||
| SpeedStats *speedstats.SpeedStats | |||
| AccToken *accesstoken.Keeper | |||
| Cluster *cluster.Cluster | |||
| } | |||
| func NewService( | |||
| @@ -52,6 +54,7 @@ func NewService( | |||
| tickTock *ticktock.TickTock, | |||
| speedStats *speedstats.SpeedStats, | |||
| accToken *accesstoken.Keeper, | |||
| cluster *cluster.Cluster, | |||
| ) *Service { | |||
| return &Service{ | |||
| PubLock: publock, | |||
| @@ -69,5 +72,6 @@ func NewService( | |||
| TickTock: tickTock, | |||
| SpeedStats: speedStats, | |||
| AccToken: accToken, | |||
| Cluster: cluster, | |||
| } | |||
| } | |||
| @@ -0,0 +1,34 @@ | |||
| package api | |||
| import ( | |||
| "net/http" | |||
| "gitlink.org.cn/cloudream/common/sdks" | |||
| ) | |||
| type ClusterService struct { | |||
| *Client | |||
| } | |||
| func (c *Client) Cluster() *ClusterService { | |||
| return &ClusterService{c} | |||
| } | |||
| var ClusterStatusPath = "/cluster/status" | |||
| type ClusterStatus struct{} | |||
| func (r *ClusterStatus) MakeParam() *sdks.RequestParam { | |||
| return sdks.MakeJSONParam(http.MethodGet, ClusterStatusPath, r) | |||
| } | |||
| type ClusterStatusResp struct { | |||
| } | |||
| func (r *ClusterStatusResp) ParseResponse(resp *http.Response) error { | |||
| return sdks.ParseCodeDataJSONResponse(resp, r) | |||
| } | |||
| func (c *ClusterService) Status(req ClusterStatus) (*ClusterStatusResp, error) { | |||
| return JSONAPI(&c.cfg, c.httpCli, &req, &ClusterStatusResp{}) | |||
| } | |||
| @@ -49,6 +49,12 @@ | |||
| "userSpaceID": 0, | |||
| "maxBodySize": 5242880 | |||
| }, | |||
| "rpc": { | |||
| "listen": "127.0.0.1:7990", | |||
| "rootCA": "", | |||
| "serverCert": "", | |||
| "serverKey": "" | |||
| }, | |||
| "mount": { | |||
| "enabled": false, | |||
| "mountPoint": "", | |||
| @@ -66,5 +72,15 @@ | |||
| "accessToken": { | |||
| "account": "", | |||
| "password": "" | |||
| }, | |||
| "cluster": { | |||
| "enabled": false, | |||
| "nodeName": "", | |||
| "announce": "127.0.0.1:7990", | |||
| "peers": [], | |||
| "rootCA": "", | |||
| "clientCert": "", | |||
| "clientKey": "", | |||
| "storeBase": "" | |||
| } | |||
| } | |||
| @@ -11,4 +11,5 @@ const ( | |||
| TaskNotFound ErrorCode = "TaskNotFound" | |||
| Unauthorized ErrorCode = "Unauthorized" | |||
| ChannelClosed ErrorCode = "ChannelClosed" | |||
| ClusterNoMaster ErrorCode = "ClusterNoMaster" | |||
| ) | |||
| @@ -8,11 +8,28 @@ import ( | |||
| type Client struct { | |||
| con *grpc.ClientConn | |||
| cli ClientClient | |||
| pool *Pool | |||
| fusedErr *rpc.CodeError | |||
| } | |||
| func (c *Client) Release() { | |||
| func NewClient(con *grpc.ClientConn) *Client { | |||
| return &Client{ | |||
| con: con, | |||
| cli: NewClientClient(con), | |||
| } | |||
| } | |||
| func NewFusedClient(err *rpc.CodeError) *Client { | |||
| return &Client{ | |||
| fusedErr: err, | |||
| } | |||
| } | |||
| type PoolClient struct { | |||
| Client | |||
| pool *Pool | |||
| } | |||
| func (c *PoolClient) Release() { | |||
| if c.con != nil { | |||
| c.pool.connPool.Release(c.pool.cfg.Address) | |||
| } | |||
| @@ -234,7 +234,7 @@ var file_pkgs_rpc_client_client_proto_rawDesc = []byte{ | |||
| 0x28, 0x0a, 0x06, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, | |||
| 0x10, 0x2e, 0x63, 0x6c, 0x69, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4b, | |||
| 0x56, 0x52, 0x06, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x42, 0x6f, 0x64, | |||
| 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x32, 0xa2, 0x02, | |||
| 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x32, 0xd2, 0x02, | |||
| 0x0a, 0x06, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x31, 0x0a, 0x0e, 0x50, 0x75, 0x62, 0x4c, | |||
| 0x6f, 0x63, 0x6b, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x0c, 0x2e, 0x72, 0x70, 0x63, | |||
| 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x52, | |||
| @@ -249,6 +249,9 @@ var file_pkgs_rpc_client_client_proto_rawDesc = []byte{ | |||
| 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x10, 0x2e, | |||
| 0x72, 0x70, 0x63, 0x2e, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x1a, | |||
| 0x0d, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, | |||
| 0x12, 0x2e, 0x0a, 0x0f, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x41, 0x70, 0x70, 0x6c, 0x79, | |||
| 0x4c, 0x6f, 0x67, 0x12, 0x0c, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, | |||
| 0x74, 0x1a, 0x0d, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, | |||
| 0x12, 0x40, 0x0a, 0x09, 0x48, 0x54, 0x54, 0x50, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x12, 0x18, 0x2e, | |||
| 0x63, 0x6c, 0x69, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x50, 0x72, 0x6f, 0x78, 0x79, | |||
| 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x63, 0x6c, 0x69, 0x72, 0x70, 0x63, | |||
| @@ -288,14 +291,16 @@ var file_pkgs_rpc_client_client_proto_depIdxs = []int32{ | |||
| 3, // 3: clirpc.Client.GetClusterMasterInfo:input_type -> rpc.Request | |||
| 3, // 4: clirpc.Client.ClusterRaftRPC:input_type -> rpc.Request | |||
| 4, // 5: clirpc.Client.ClusterRaftInstallSnapshot:input_type -> rpc.ChunkedData | |||
| 1, // 6: clirpc.Client.HTTPProxy:input_type -> clirpc.HTTPProxyRequest | |||
| 5, // 7: clirpc.Client.PubLockChannel:output_type -> rpc.Response | |||
| 5, // 8: clirpc.Client.GetClusterMasterInfo:output_type -> rpc.Response | |||
| 5, // 9: clirpc.Client.ClusterRaftRPC:output_type -> rpc.Response | |||
| 5, // 10: clirpc.Client.ClusterRaftInstallSnapshot:output_type -> rpc.Response | |||
| 2, // 11: clirpc.Client.HTTPProxy:output_type -> clirpc.HTTPProxyResponse | |||
| 7, // [7:12] is the sub-list for method output_type | |||
| 2, // [2:7] is the sub-list for method input_type | |||
| 3, // 6: clirpc.Client.ClusterApplyLog:input_type -> rpc.Request | |||
| 1, // 7: clirpc.Client.HTTPProxy:input_type -> clirpc.HTTPProxyRequest | |||
| 5, // 8: clirpc.Client.PubLockChannel:output_type -> rpc.Response | |||
| 5, // 9: clirpc.Client.GetClusterMasterInfo:output_type -> rpc.Response | |||
| 5, // 10: clirpc.Client.ClusterRaftRPC:output_type -> rpc.Response | |||
| 5, // 11: clirpc.Client.ClusterRaftInstallSnapshot:output_type -> rpc.Response | |||
| 5, // 12: clirpc.Client.ClusterApplyLog:output_type -> rpc.Response | |||
| 2, // 13: clirpc.Client.HTTPProxy:output_type -> clirpc.HTTPProxyResponse | |||
| 8, // [8:14] is the sub-list for method output_type | |||
| 2, // [2:8] is the sub-list for method input_type | |||
| 2, // [2:2] is the sub-list for extension type_name | |||
| 2, // [2:2] is the sub-list for extension extendee | |||
| 0, // [0:2] is the sub-list for field type_name | |||
| @@ -31,6 +31,7 @@ service Client { | |||
| rpc GetClusterMasterInfo(rpc.Request) returns(rpc.Response); | |||
| rpc ClusterRaftRPC(rpc.Request) returns(rpc.Response); | |||
| rpc ClusterRaftInstallSnapshot(stream rpc.ChunkedData) returns(rpc.Response); | |||
| rpc ClusterApplyLog(rpc.Request) returns(rpc.Response); | |||
| rpc HTTPProxy(HTTPProxyRequest) returns(HTTPProxyResponse); | |||
| @@ -24,6 +24,7 @@ const ( | |||
| Client_GetClusterMasterInfo_FullMethodName = "/clirpc.Client/GetClusterMasterInfo" | |||
| Client_ClusterRaftRPC_FullMethodName = "/clirpc.Client/ClusterRaftRPC" | |||
| Client_ClusterRaftInstallSnapshot_FullMethodName = "/clirpc.Client/ClusterRaftInstallSnapshot" | |||
| Client_ClusterApplyLog_FullMethodName = "/clirpc.Client/ClusterApplyLog" | |||
| Client_HTTPProxy_FullMethodName = "/clirpc.Client/HTTPProxy" | |||
| ) | |||
| @@ -35,6 +36,7 @@ type ClientClient interface { | |||
| GetClusterMasterInfo(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) | |||
| ClusterRaftRPC(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) | |||
| ClusterRaftInstallSnapshot(ctx context.Context, opts ...grpc.CallOption) (Client_ClusterRaftInstallSnapshotClient, error) | |||
| ClusterApplyLog(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) | |||
| HTTPProxy(ctx context.Context, in *HTTPProxyRequest, opts ...grpc.CallOption) (*HTTPProxyResponse, error) | |||
| } | |||
| @@ -129,6 +131,15 @@ func (x *clientClusterRaftInstallSnapshotClient) CloseAndRecv() (*rpc.Response, | |||
| return m, nil | |||
| } | |||
| func (c *clientClient) ClusterApplyLog(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) { | |||
| out := new(rpc.Response) | |||
| err := c.cc.Invoke(ctx, Client_ClusterApplyLog_FullMethodName, in, out, opts...) | |||
| if err != nil { | |||
| return nil, err | |||
| } | |||
| return out, nil | |||
| } | |||
| func (c *clientClient) HTTPProxy(ctx context.Context, in *HTTPProxyRequest, opts ...grpc.CallOption) (*HTTPProxyResponse, error) { | |||
| out := new(HTTPProxyResponse) | |||
| err := c.cc.Invoke(ctx, Client_HTTPProxy_FullMethodName, in, out, opts...) | |||
| @@ -146,6 +157,7 @@ type ClientServer interface { | |||
| GetClusterMasterInfo(context.Context, *rpc.Request) (*rpc.Response, error) | |||
| ClusterRaftRPC(context.Context, *rpc.Request) (*rpc.Response, error) | |||
| ClusterRaftInstallSnapshot(Client_ClusterRaftInstallSnapshotServer) error | |||
| ClusterApplyLog(context.Context, *rpc.Request) (*rpc.Response, error) | |||
| HTTPProxy(context.Context, *HTTPProxyRequest) (*HTTPProxyResponse, error) | |||
| mustEmbedUnimplementedClientServer() | |||
| } | |||
| @@ -166,6 +178,9 @@ func (UnimplementedClientServer) ClusterRaftRPC(context.Context, *rpc.Request) ( | |||
| func (UnimplementedClientServer) ClusterRaftInstallSnapshot(Client_ClusterRaftInstallSnapshotServer) error { | |||
| return status.Errorf(codes.Unimplemented, "method ClusterRaftInstallSnapshot not implemented") | |||
| } | |||
| func (UnimplementedClientServer) ClusterApplyLog(context.Context, *rpc.Request) (*rpc.Response, error) { | |||
| return nil, status.Errorf(codes.Unimplemented, "method ClusterApplyLog not implemented") | |||
| } | |||
| func (UnimplementedClientServer) HTTPProxy(context.Context, *HTTPProxyRequest) (*HTTPProxyResponse, error) { | |||
| return nil, status.Errorf(codes.Unimplemented, "method HTTPProxy not implemented") | |||
| } | |||
| @@ -270,6 +285,24 @@ func (x *clientClusterRaftInstallSnapshotServer) Recv() (*rpc.ChunkedData, error | |||
| return m, nil | |||
| } | |||
| func _Client_ClusterApplyLog_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { | |||
| in := new(rpc.Request) | |||
| if err := dec(in); err != nil { | |||
| return nil, err | |||
| } | |||
| if interceptor == nil { | |||
| return srv.(ClientServer).ClusterApplyLog(ctx, in) | |||
| } | |||
| info := &grpc.UnaryServerInfo{ | |||
| Server: srv, | |||
| FullMethod: Client_ClusterApplyLog_FullMethodName, | |||
| } | |||
| handler := func(ctx context.Context, req interface{}) (interface{}, error) { | |||
| return srv.(ClientServer).ClusterApplyLog(ctx, req.(*rpc.Request)) | |||
| } | |||
| return interceptor(ctx, in, info, handler) | |||
| } | |||
| func _Client_HTTPProxy_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { | |||
| in := new(HTTPProxyRequest) | |||
| if err := dec(in); err != nil { | |||
| @@ -303,6 +336,10 @@ var Client_ServiceDesc = grpc.ServiceDesc{ | |||
| MethodName: "ClusterRaftRPC", | |||
| Handler: _Client_ClusterRaftRPC_Handler, | |||
| }, | |||
| { | |||
| MethodName: "ClusterApplyLog", | |||
| Handler: _Client_ClusterApplyLog_Handler, | |||
| }, | |||
| { | |||
| MethodName: "HTTPProxy", | |||
| Handler: _Client_HTTPProxy_Handler, | |||
| @@ -3,6 +3,7 @@ package clirpc | |||
| import ( | |||
| "context" | |||
| "io" | |||
| "time" | |||
| "github.com/hashicorp/raft" | |||
| "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" | |||
| @@ -12,6 +13,7 @@ type ClusterService interface { | |||
| GetClusterMasterInfo(ctx context.Context, req *GetClusterMasterInfo) (*GetClusterMasterInfoResp, *rpc.CodeError) | |||
| ClusterRaftRPC(ctx context.Context, req *ClusterRaftRPC) (*ClusterRaftRPCResp, *rpc.CodeError) | |||
| ClusterRaftInstallSnapshot(ctx context.Context, req *ClusterRaftInstallSnapshot) (*ClusterRaftInstallSnapshotResp, *rpc.CodeError) | |||
| ClusterApplyLog(ctx context.Context, req *ClusterApplyLog) (*ClusterApplyLogResp, *rpc.CodeError) | |||
| } | |||
| type GetClusterMasterInfo struct { | |||
| @@ -80,3 +82,20 @@ func (c *Client) ClusterRaftInstallSnapshot(ctx context.Context, msg *ClusterRaf | |||
| func (s *Server) ClusterRaftInstallSnapshot(req Client_ClusterRaftInstallSnapshotServer) error { | |||
| return rpc.UploadStreamServer(s.svrImpl.ClusterRaftInstallSnapshot, req) | |||
| } | |||
| type ClusterApplyLog struct { | |||
| Data string | |||
| Timeout time.Duration | |||
| } | |||
| type ClusterApplyLogResp struct{} | |||
| func (c *Client) ClusterApplyLog(ctx context.Context, msg *ClusterApplyLog) (*ClusterApplyLogResp, *rpc.CodeError) { | |||
| if c.fusedErr != nil { | |||
| return nil, c.fusedErr | |||
| } | |||
| return rpc.UnaryClient[*ClusterApplyLogResp](c.cli.ClusterApplyLog, ctx, msg) | |||
| } | |||
| func (s *Server) ClusterApplyLog(ctx context.Context, msg *rpc.Request) (*rpc.Response, error) { | |||
| return rpc.UnaryServer(s.svrImpl.ClusterApplyLog, ctx, msg) | |||
| } | |||
| @@ -74,7 +74,6 @@ func (c *PoolConfigJSON) BuildTempClient() (*TempClient, error) { | |||
| Client: Client{ | |||
| con: gcon, | |||
| cli: NewClientClient(gcon), | |||
| pool: nil, | |||
| fusedErr: nil, | |||
| }, | |||
| }, nil | |||
| @@ -92,21 +91,25 @@ func NewPool(cfg PoolConfig) *Pool { | |||
| } | |||
| } | |||
| func (p *Pool) Get() *Client { | |||
| func (p *Pool) Get() *PoolClient { | |||
| con, err := p.connPool.GetConnection(p.cfg.Address) | |||
| if err != nil { | |||
| return &Client{ | |||
| con: nil, | |||
| cli: nil, | |||
| pool: p, | |||
| fusedErr: rpc.Failed(errorcode.OperationFailed, err.Error()), | |||
| return &PoolClient{ | |||
| Client: Client{ | |||
| con: nil, | |||
| cli: nil, | |||
| fusedErr: rpc.Failed(errorcode.OperationFailed, err.Error()), | |||
| }, | |||
| pool: p, | |||
| } | |||
| } | |||
| return &Client{ | |||
| con: con, | |||
| cli: NewClientClient(con), | |||
| pool: p, | |||
| fusedErr: nil, | |||
| return &PoolClient{ | |||
| Client: Client{ | |||
| con: con, | |||
| cli: NewClientClient(con), | |||
| fusedErr: nil, | |||
| }, | |||
| pool: p, | |||
| } | |||
| } | |||
| @@ -141,6 +141,16 @@ func (p *ConnPool) Release(addr string) { | |||
| } | |||
| } | |||
| func (p *ConnPool) CloseAll() { | |||
| p.lock.Lock() | |||
| defer p.lock.Unlock() | |||
| for addr, grpcCon := range p.grpcCons { | |||
| grpcCon.grpcCon.Close() | |||
| delete(p.grpcCons, addr) | |||
| } | |||
| } | |||
| func (p *ConnPool) populateAccessTokenUnary(ctx context.Context, method string, req, reply any, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { | |||
| authInfo, err := p.cfg.AccessTokenProvider.MakeAuthInfo() | |||
| if err != nil { | |||
| @@ -50,6 +50,7 @@ require ( | |||
| github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.15 // indirect | |||
| github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.6 // indirect | |||
| github.com/aws/smithy-go v1.22.2 // indirect | |||
| github.com/boltdb/bolt v1.3.1 // indirect | |||
| github.com/bytedance/sonic v1.11.6 // indirect | |||
| github.com/bytedance/sonic/loader v0.1.1 // indirect | |||
| github.com/cloudwego/base64x v0.1.4 // indirect | |||
| @@ -64,9 +65,11 @@ require ( | |||
| github.com/hashicorp/go-hclog v1.6.2 // indirect | |||
| github.com/hashicorp/go-immutable-radix v1.0.0 // indirect | |||
| github.com/hashicorp/go-metrics v0.5.4 // indirect | |||
| github.com/hashicorp/go-msgpack v0.5.5 // indirect | |||
| github.com/hashicorp/go-msgpack/v2 v2.1.2 // indirect | |||
| github.com/hashicorp/golang-lru v0.5.0 // indirect | |||
| github.com/hashicorp/raft v1.7.3 // indirect | |||
| github.com/hashicorp/raft-boltdb v0.0.0-20250701115049-6cdf087e85ed // indirect | |||
| github.com/jinzhu/inflection v1.0.0 // indirect | |||
| github.com/jinzhu/now v1.1.5 // indirect | |||
| github.com/jonboulle/clockwork v0.4.0 // indirect | |||
| @@ -4,6 +4,8 @@ filippo.io/edwards25519 v1.1.0 h1:FNf4tywRC1HmFuKW5xopWpigGjJKiJSV0Cqo0cJWDaA= | |||
| filippo.io/edwards25519 v1.1.0/go.mod h1:BxyFTGdWcka3PhytdK4V28tE5sGfRvvvRV7EaN4VDT4= | |||
| github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= | |||
| github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= | |||
| github.com/DataDog/zstd v1.5.2/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= | |||
| github.com/Sereal/Sereal/Go/sereal v0.0.0-20231009093132-b9187f1a92c6/go.mod h1:JwrycNnC8+sZPDyzM3MQ86LvaGzSpfxg885KOOwFRW4= | |||
| github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= | |||
| github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= | |||
| github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= | |||
| @@ -40,6 +42,8 @@ github.com/aws/smithy-go v1.22.2/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxY | |||
| github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= | |||
| github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= | |||
| github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= | |||
| github.com/boltdb/bolt v1.3.1 h1:JQmyP4ZBrce+ZQu0dY660FMfatumYDLun9hBCUVIkF4= | |||
| github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= | |||
| github.com/bytedance/sonic v1.11.6 h1:oUp34TzMlL+OY1OUWxHqsdkgC/Zfc85zGqw9siXjrc0= | |||
| github.com/bytedance/sonic v1.11.6/go.mod h1:LysEHSvpvDySVdC2f87zGWf6CIKJcAvqab1ZaiQtds4= | |||
| github.com/bytedance/sonic/loader v0.1.1 h1:c+e5Pt1k/cy5wMveRDyk2X4B9hF4g7an8N3zCYjJFNM= | |||
| @@ -68,6 +72,9 @@ github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs | |||
| github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= | |||
| github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= | |||
| github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= | |||
| github.com/davecgh/go-xdr v0.0.0-20161123171359-e6a2ba005892/go.mod h1:CTDl0pzVzE5DEzZhPfvhY/9sPFMQIxaJ9VAMs9AagrE= | |||
| github.com/dchest/siphash v1.2.3/go.mod h1:0NvQU092bT0ipiFN++/rXm69QG9tVxLAlQHIXMPAkHc= | |||
| github.com/dgryski/go-ddmin v0.0.0-20210904190556-96a6d69f1034/go.mod h1:zz4KxBkcXUWKjIcrc+uphJ1gPh/t18ymGm3PmQ+VGTk= | |||
| github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= | |||
| github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= | |||
| github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= | |||
| @@ -122,7 +129,10 @@ github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:W | |||
| github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= | |||
| github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= | |||
| github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= | |||
| github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= | |||
| github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= | |||
| github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= | |||
| github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= | |||
| github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= | |||
| github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= | |||
| github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= | |||
| @@ -130,6 +140,7 @@ github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ | |||
| github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= | |||
| github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= | |||
| github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= | |||
| github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= | |||
| github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= | |||
| github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= | |||
| github.com/google/go-querystring v1.1.0 h1:AnCroh3fv4ZBgVIf1Iwtovgjaw/GiKJo8M8yD/fhyJ8= | |||
| @@ -151,6 +162,8 @@ github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxB | |||
| github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= | |||
| github.com/hashicorp/go-metrics v0.5.4 h1:8mmPiIJkTPPEbAiV97IxdAGNdRdaWwVap1BU6elejKY= | |||
| github.com/hashicorp/go-metrics v0.5.4/go.mod h1:CG5yz4NZ/AI/aQt9Ucm/vdBnbh7fvmv4lxZ350i+QQI= | |||
| github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI= | |||
| github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= | |||
| github.com/hashicorp/go-msgpack/v2 v2.1.2 h1:4Ee8FTp834e+ewB71RDrQ0VKpyFdrKOjvYtnQ/ltVj0= | |||
| github.com/hashicorp/go-msgpack/v2 v2.1.2/go.mod h1:upybraOAblm4S7rx0+jeNy+CWWhzywQsSRV5033mMu4= | |||
| github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= | |||
| @@ -163,6 +176,8 @@ github.com/hashicorp/golang-lru/v2 v2.0.7 h1:a+bsQ5rvGLjzHuww6tVxozPZFVghXaHOwFs | |||
| github.com/hashicorp/golang-lru/v2 v2.0.7/go.mod h1:QeFd9opnmA6QUJc5vARoKUSoFhyfM2/ZepoAG6RGpeM= | |||
| github.com/hashicorp/raft v1.7.3 h1:DxpEqZJysHN0wK+fviai5mFcSYsCkNpFUl1xpAW8Rbo= | |||
| github.com/hashicorp/raft v1.7.3/go.mod h1:DfvCGFxpAUPE0L4Uc8JLlTPtc3GzSbdH0MTJCLgnmJQ= | |||
| github.com/hashicorp/raft-boltdb v0.0.0-20250701115049-6cdf087e85ed h1:l6K4AyoSw31EyQPj0wQTXHpeUOXtDIOu6+jL5ifrmGM= | |||
| github.com/hashicorp/raft-boltdb v0.0.0-20250701115049-6cdf087e85ed/go.mod h1:sgCxzMuvQ3huVxgmeDdj73YIMmezWZ40HQu2IPmjJWk= | |||
| github.com/huaweicloud/huaweicloud-sdk-go-obs v3.24.9+incompatible h1:XQVXdk+WAJ4fSNB6mMRuYNvFWou7BZs6SZB925hPrnk= | |||
| github.com/huaweicloud/huaweicloud-sdk-go-obs v3.24.9+incompatible/go.mod h1:l7VUhRbTKCzdOacdT4oWCwATKyvZqUOlOqr0Ous3k4s= | |||
| github.com/huaweicloud/huaweicloud-sdk-go-v3 v0.1.131 h1:34E2+lzM/yi0GlYAEQEUuf4/3mAoAadA+7oaq9q3Mys= | |||
| @@ -181,6 +196,7 @@ github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= | |||
| github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= | |||
| github.com/jonboulle/clockwork v0.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST4RZ4= | |||
| github.com/jonboulle/clockwork v0.4.0/go.mod h1:xgRqUGwRcjKCO1vbZUEtSLrqKoPSsUpK7fnezOII0kc= | |||
| github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y= | |||
| github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= | |||
| github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= | |||
| github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= | |||
| @@ -203,6 +219,7 @@ github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxv | |||
| github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= | |||
| github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= | |||
| github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= | |||
| github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= | |||
| github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= | |||
| github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= | |||
| github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= | |||
| @@ -215,6 +232,7 @@ github.com/leodido/go-urn v1.4.0 h1:WT9HwE9SGECu3lg4d/dIA+jxlljEa1/ffXKmRjqdmIQ= | |||
| github.com/leodido/go-urn v1.4.0/go.mod h1:bvxc+MVxLKB4z00jd1z+Dvzr47oO32F/QSNjSBOlFxI= | |||
| github.com/magefile/mage v1.15.0 h1:BvGheCMAsG3bWUDbZ8AyXXpCNwU9u5CB6sM+HNb9HYg= | |||
| github.com/magefile/mage v1.15.0/go.mod h1:z5UZb/iS3GoOSn0JgWuiw7dxlurVYTu+/jHXqQg881A= | |||
| github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= | |||
| github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= | |||
| github.com/mattn/go-colorable v0.1.7/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= | |||
| github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= | |||
| @@ -254,6 +272,7 @@ github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRW | |||
| github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= | |||
| github.com/pelletier/go-toml/v2 v2.2.2 h1:aYUidT7k73Pcl9nb2gScu7NSrKCSHIDE89b3+6Wq+LM= | |||
| github.com/pelletier/go-toml/v2 v2.2.2/go.mod h1:1t835xjRzz80PqgE6HHgN2JOsmgYu/h4qDAS4n929Rs= | |||
| github.com/philhofer/fwd v1.1.2/go.mod h1:qkPdfjR2SIEbspLqpe1tO4n5yICnr2DY7mqEx2tUTP0= | |||
| github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= | |||
| github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= | |||
| github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= | |||
| @@ -263,6 +282,7 @@ github.com/pkg/term v1.2.0-beta.2/go.mod h1:E25nymQcrSllhX42Ok8MRm1+hyBdHY0dCeiK | |||
| github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= | |||
| github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= | |||
| github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= | |||
| github.com/pquerna/ffjson v0.0.0-20190930134022-aa0246cd15f7/go.mod h1:YARuvh7BUWHNhzDq2OM5tzR2RiCcN2D7sapiKyCel/M= | |||
| github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= | |||
| github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= | |||
| github.com/prometheus/client_golang v1.4.0/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= | |||
| @@ -323,10 +343,12 @@ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ | |||
| github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals= | |||
| github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= | |||
| github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= | |||
| github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= | |||
| github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= | |||
| github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= | |||
| github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= | |||
| github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= | |||
| github.com/tinylib/msgp v1.1.8/go.mod h1:qkpG+2ldGg4xRFmx+jfTvZPxfGFhi64BcnL9vkCm/Tw= | |||
| github.com/tjfoc/gmsm v1.4.1 h1:aMe1GlZb+0bLjn+cKTPEvvn9oUEBlJitaZiiBwsbgho= | |||
| github.com/tjfoc/gmsm v1.4.1/go.mod h1:j4INPkHWMrhJb38G+J6W4Tw0AbuN8Thu3PbdVYhVcTE= | |||
| github.com/tmaxmax/go-sse v0.11.0 h1:nogmJM6rJUoOLoAwEKeQe5XlVpt9l7N82SS1jI7lWFg= | |||
| @@ -356,6 +378,8 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh | |||
| golang.org/x/crypto v0.0.0-20201012173705-84dcc777aaee/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= | |||
| golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= | |||
| golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= | |||
| golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= | |||
| golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= | |||
| golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= | |||
| golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= | |||
| golang.org/x/crypto v0.38.0 h1:jt+WWG8IZlBnVbomuhg2Mdq0+BBQaHbtqHEFEigjUV8= | |||
| @@ -367,7 +391,10 @@ golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTk | |||
| golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= | |||
| golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= | |||
| golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= | |||
| golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= | |||
| golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= | |||
| golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= | |||
| golang.org/x/mod v0.13.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= | |||
| golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= | |||
| golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= | |||
| golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= | |||
| @@ -375,6 +402,7 @@ golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73r | |||
| golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= | |||
| golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= | |||
| golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= | |||
| golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= | |||
| golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= | |||
| golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= | |||
| golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= | |||
| @@ -382,8 +410,11 @@ golang.org/x/net v0.0.0-20201010224723-4f7140c49acb/go.mod h1:sp8m0HH+o8qH0wwXwY | |||
| golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= | |||
| golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= | |||
| golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= | |||
| golang.org/x/net v0.3.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE= | |||
| golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= | |||
| golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= | |||
| golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= | |||
| golang.org/x/net v0.16.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= | |||
| golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= | |||
| golang.org/x/net v0.35.0 h1:T5GQRQb2y08kTAByq9L4/bz8cipCdA8FbRTXewonqY8= | |||
| golang.org/x/net v0.35.0/go.mod h1:EglIi67kWsHKlRzzVMUD93VMSWGFOMSZgxFjparz1Qk= | |||
| @@ -397,6 +428,8 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ | |||
| golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= | |||
| golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= | |||
| golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= | |||
| golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= | |||
| golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= | |||
| golang.org/x/sync v0.14.0 h1:woo0S4Yywslg6hp4eUFjTVOyKt0RookbpAHG4c1HmhQ= | |||
| golang.org/x/sync v0.14.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= | |||
| golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= | |||
| @@ -431,17 +464,23 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc | |||
| golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | |||
| golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | |||
| golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | |||
| golang.org/x/sys v0.3.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | |||
| golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | |||
| golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | |||
| golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | |||
| golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | |||
| golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | |||
| golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= | |||
| golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= | |||
| golang.org/x/sys v0.33.0 h1:q3i8TbbEz+JRD9ywIRlyRAQbM0qF7hu24q3teo2hbuw= | |||
| golang.org/x/sys v0.33.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= | |||
| golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= | |||
| golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= | |||
| golang.org/x/term v0.3.0/go.mod h1:q750SLmJuPmVoN1blW3UFBPREJfb1KmY3vwxfr+nFDA= | |||
| golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= | |||
| golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= | |||
| golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= | |||
| golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= | |||
| golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= | |||
| golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= | |||
| golang.org/x/term v0.32.0 h1:DR4lr0TjUs3epypdhTOkMmuF5CDFJ/8pOnbzMZPQ7bg= | |||
| @@ -452,8 +491,10 @@ golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= | |||
| golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= | |||
| golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= | |||
| golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= | |||
| golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= | |||
| golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= | |||
| golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= | |||
| golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= | |||
| golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= | |||
| golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= | |||
| golang.org/x/text v0.25.0 h1:qVyWApTSYLk/drJRO5mDlNYskwQznZmkpV2c8q9zls4= | |||
| @@ -465,11 +506,15 @@ golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3 | |||
| golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= | |||
| golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= | |||
| golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= | |||
| golang.org/x/tools v0.4.0/go.mod h1:UE5sM2OK9E/d67R0ANs2xJizIymRP5gJU295PvKXxjQ= | |||
| golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= | |||
| golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= | |||
| golang.org/x/tools v0.14.0/go.mod h1:uYBEerGOWcJyEORxN+Ek8+TT266gXkNlHdJBwexUsBg= | |||
| golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= | |||
| golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= | |||
| google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= | |||
| google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= | |||
| google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= | |||
| google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= | |||
| google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= | |||
| google.golang.org/genproto/googleapis/rpc v0.0.0-20241021214115-324edc3d5d38 h1:zciRKQ4kBpFgpfC5QQCVtnnNAcLIqweL7plyZRQHVpI= | |||
| @@ -487,6 +532,8 @@ google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miE | |||
| google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= | |||
| google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= | |||
| google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= | |||
| google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= | |||
| google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= | |||
| google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= | |||
| google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= | |||
| gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= | |||
| @@ -496,11 +543,14 @@ gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntN | |||
| gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= | |||
| gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= | |||
| gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= | |||
| gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= | |||
| gopkg.in/vmihailenco/msgpack.v2 v2.9.2/go.mod h1:/3Dn1Npt9+MYyLpYYXjInO/5jvMLamn+AEGwNEOatn8= | |||
| gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= | |||
| gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= | |||
| gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= | |||
| gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= | |||
| gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= | |||
| gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= | |||
| gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= | |||
| gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= | |||
| gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= | |||
| @@ -0,0 +1,15 @@ | |||
| package cluster | |||
| import ( | |||
| "github.com/spf13/cobra" | |||
| "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/admin" | |||
| ) | |||
| var ClusterCmd = &cobra.Command{ | |||
| Use: "cluster", | |||
| Aliases: []string{"cls"}, | |||
| } | |||
| func init() { | |||
| admin.AdminCmd.AddCommand(ClusterCmd) | |||
| } | |||
| @@ -0,0 +1,34 @@ | |||
| package cluster | |||
| import ( | |||
| "fmt" | |||
| "github.com/spf13/cobra" | |||
| cliapi "gitlink.org.cn/cloudream/jcs-pub/client/sdk/api/v1" | |||
| "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd" | |||
| ) | |||
| func init() { | |||
| var opt statusOpt | |||
| cmd := cobra.Command{ | |||
| Use: "status", | |||
| Args: cobra.ExactArgs(0), | |||
| RunE: func(c *cobra.Command, args []string) error { | |||
| ctx := cmd.GetCmdCtx(c) | |||
| return status(c, ctx, opt, args) | |||
| }, | |||
| } | |||
| ClusterCmd.AddCommand(&cmd) | |||
| } | |||
| type statusOpt struct { | |||
| } | |||
| func status(c *cobra.Command, ctx *cmd.CommandContext, opt statusOpt, args []string) error { | |||
| _, err := ctx.Client.Cluster().Status(cliapi.ClusterStatus{}) | |||
| if err != nil { | |||
| return fmt.Errorf("get status: %v", err) | |||
| } | |||
| return nil | |||
| } | |||
| @@ -2,6 +2,7 @@ package all | |||
| import ( | |||
| _ "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/admin" | |||
| _ "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/admin/cluster" | |||
| _ "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/admin/sysevent" | |||
| _ "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/admin/ticktock" | |||
| _ "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/bucket" | |||