| @@ -1,41 +1,54 @@ | |||||
| package cluster | package cluster | ||||
| import ( | import ( | ||||
| "context" | |||||
| "crypto/tls" | |||||
| "fmt" | |||||
| "os" | |||||
| "path/filepath" | |||||
| "sync" | |||||
| "time" | "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/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" | clirpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/client" | ||||
| "google.golang.org/grpc" | |||||
| "google.golang.org/grpc/credentials" | |||||
| ) | ) | ||||
| type Cluster struct { | 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{ | 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") | 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{ | poolCfgJSON := clirpc.PoolConfigJSON{ | ||||
| Address: c.cfg.MasterAddress, | |||||
| RootCA: c.cfg.RootCA, | RootCA: c.cfg.RootCA, | ||||
| ClientCert: c.cfg.ClientCert, | ClientCert: c.cfg.ClientCert, | ||||
| ClientKey: c.cfg.ClientKey, | ClientKey: c.cfg.ClientKey, | ||||
| @@ -43,40 +56,238 @@ func (c *Cluster) Start() error { | |||||
| poolCfg, err := poolCfgJSON.Build() | poolCfg, err := poolCfgJSON.Build() | ||||
| if err != nil { | 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 { | 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 { | 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 { | 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 | package cluster | ||||
| type Config struct { | 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 ( | const ( | ||||
| CmdAppendEntries = "AppendEntries" | |||||
| CmdRequestVote = "RequestVote" | |||||
| CmdTimeoutNow = "TimeoutNow" | |||||
| CmdAppendEntries = "AppendEntries" | |||||
| CmdRequestVote = "RequestVote" | |||||
| CmdRequestPreVote = "RequestPreVote" | |||||
| CmdTimeoutNow = "TimeoutNow" | |||||
| ) | ) | ||||
| type Transport struct { | type Transport struct { | ||||
| localAddr string | localAddr string | ||||
| shutdownCh chan any | shutdownCh chan any | ||||
| shutdownLock sync.Mutex | |||||
| shutdown bool | |||||
| consumeCh chan raft.RPC | consumeCh chan raft.RPC | ||||
| cliPoolCfg clirpc.PoolConfig | cliPoolCfg clirpc.PoolConfig | ||||
| cliPools map[raft.ServerAddress]*clirpc.Pool | cliPools map[raft.ServerAddress]*clirpc.Pool | ||||
| @@ -30,6 +33,36 @@ type Transport struct { | |||||
| heartbeatFnLock sync.Mutex | 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 { | func (t *Transport) Consumer() <-chan raft.RPC { | ||||
| return t.consumeCh | return t.consumeCh | ||||
| } | } | ||||
| @@ -106,6 +139,30 @@ func (t *Transport) RequestVote(id raft.ServerID, target raft.ServerAddress, arg | |||||
| return nil | 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 { | func (t *Transport) InstallSnapshot(id raft.ServerID, target raft.ServerAddress, args *raft.InstallSnapshotRequest, resp *raft.InstallSnapshotResponse, data io.Reader) error { | ||||
| cli := t.getCli(target) | cli := t.getCli(target) | ||||
| r, cerr := cli.ClusterRaftInstallSnapshot(context.Background(), &clirpc.ClusterRaftInstallSnapshot{ | 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) { | 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{ | rp := raft.RPC{ | ||||
| RespChan: respChan, | RespChan: respChan, | ||||
| } | } | ||||
| @@ -170,7 +227,7 @@ func (t *Transport) OnRPC(req *clirpc.ClusterRaftRPC) (*clirpc.ClusterRaftRPCRes | |||||
| if err != nil { | if err != nil { | ||||
| return nil, rpc.Failed(ecode.OperationFailed, err.Error()) | return nil, rpc.Failed(ecode.OperationFailed, err.Error()) | ||||
| } | } | ||||
| rp.Command = r | |||||
| rp.Command = &r | |||||
| leaderAddr := r.RPCHeader.Addr | leaderAddr := r.RPCHeader.Addr | ||||
| @@ -187,7 +244,15 @@ func (t *Transport) OnRPC(req *clirpc.ClusterRaftRPC) (*clirpc.ClusterRaftRPCRes | |||||
| if err != nil { | if err != nil { | ||||
| return nil, rpc.Failed(ecode.OperationFailed, err.Error()) | 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: | case CmdTimeoutNow: | ||||
| var r raft.TimeoutNowRequest | var r raft.TimeoutNowRequest | ||||
| @@ -195,7 +260,7 @@ func (t *Transport) OnRPC(req *clirpc.ClusterRaftRPC) (*clirpc.ClusterRaftRPCRes | |||||
| if err != nil { | if err != nil { | ||||
| return nil, rpc.Failed(ecode.OperationFailed, err.Error()) | return nil, rpc.Failed(ecode.OperationFailed, err.Error()) | ||||
| } | } | ||||
| rp.Command = r | |||||
| rp.Command = &r | |||||
| } | } | ||||
| if isHeartbeat { | if isHeartbeat { | ||||
| @@ -239,7 +304,7 @@ RESP: | |||||
| } | } | ||||
| func (t *Transport) OnInstallSnapshot(req *clirpc.ClusterRaftInstallSnapshot) (*clirpc.ClusterRaftInstallSnapshotResp, *rpc.CodeError) { | 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{ | rp := raft.RPC{ | ||||
| Command: req.Args, | Command: req.Args, | ||||
| Reader: req.Data, | 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() | t.poolLock.Lock() | ||||
| defer t.poolLock.Unlock() | defer t.poolLock.Unlock() | ||||
| pool := t.cliPools[target] | pool := t.cliPools[target] | ||||
| @@ -282,9 +347,10 @@ func (t *Transport) getCli(target raft.ServerAddress) *clirpc.Client { | |||||
| } | } | ||||
| var _ raft.Transport = (*Transport)(nil) | var _ raft.Transport = (*Transport)(nil) | ||||
| var _ raft.WithPreVote = (*Transport)(nil) | |||||
| type AppendPipeline struct { | type AppendPipeline struct { | ||||
| cli *clirpc.Client | |||||
| cli *clirpc.PoolClient | |||||
| consumerCh chan raft.AppendFuture | consumerCh chan raft.AppendFuture | ||||
| done chan any | done chan any | ||||
| closeLock sync.Mutex | closeLock sync.Mutex | ||||
| @@ -77,14 +77,6 @@ func serveHTTP(configPath string, opts serveHTTPOptions) { | |||||
| stgglb.InitLocal(config.Cfg().Local) | 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) | db, err := db.NewDB(&config.Cfg().DB) | ||||
| if err != nil { | if err != nil { | ||||
| @@ -184,6 +176,15 @@ func serveHTTP(configPath string, opts serveHTTPOptions) { | |||||
| defer conCol.Stop() | defer conCol.Stop() | ||||
| conCol.CollectInPlace() | 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 := publock.NewMaster() | ||||
| publock.Start() | publock.Start() | ||||
| @@ -239,7 +240,7 @@ func serveHTTP(configPath string, opts serveHTTPOptions) { | |||||
| mntChan := mnt.Start() | mntChan := mnt.Start() | ||||
| defer mnt.Stop() | 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接口 | // HTTP接口 | ||||
| httpCfgJSON := config.Cfg().HTTP | httpCfgJSON := config.Cfg().HTTP | ||||
| @@ -272,6 +273,7 @@ func serveHTTP(configPath string, opts serveHTTPOptions) { | |||||
| evtPubEvt := evtPubChan.Receive() | evtPubEvt := evtPubChan.Receive() | ||||
| evtWtchrEvt := evtWtchrChan.Receive() | evtWtchrEvt := evtWtchrChan.Receive() | ||||
| conColEvt := conColChan.Receive() | conColEvt := conColChan.Receive() | ||||
| clsterEvt := clsterCh.Receive() | |||||
| acStatEvt := acStatChan.Receive() | acStatEvt := acStatChan.Receive() | ||||
| spaceSyncEvt := spaceSyncChan.Receive() | spaceSyncEvt := spaceSyncChan.Receive() | ||||
| // replEvt := replCh.Receive() | // replEvt := replCh.Receive() | ||||
| @@ -361,6 +363,23 @@ loop: | |||||
| } | } | ||||
| conColEvt = conColChan.Receive() | 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(): | case e := <-acStatEvt.Chan(): | ||||
| if e.Err != nil { | if e.Err != nil { | ||||
| logger.Errorf("receive access stat event: %v", err) | logger.Errorf("receive access stat event: %v", err) | ||||
| @@ -88,14 +88,6 @@ func test(configPath string) { | |||||
| stgglb.StandaloneMode = config.Cfg().AccessToken == nil | 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) | db, err := db.NewDB(&config.Cfg().DB) | ||||
| if err != nil { | if err != nil { | ||||
| @@ -192,6 +184,15 @@ func test(configPath string) { | |||||
| defer conCol.Stop() | defer conCol.Stop() | ||||
| conCol.CollectInPlace() | 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 := publock.NewMaster() | ||||
| publock.Start() | publock.Start() | ||||
| @@ -225,7 +226,7 @@ func test(configPath string) { | |||||
| spaceSyncChan := spaceSync.Start() | spaceSyncChan := spaceSync.Start() | ||||
| defer spaceSync.Stop() | 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() { | go func() { | ||||
| doTest(svc) | doTest(svc) | ||||
| @@ -236,6 +237,7 @@ func test(configPath string) { | |||||
| evtPubEvt := evtPubChan.Receive() | evtPubEvt := evtPubChan.Receive() | ||||
| evtWtchrEvt := evtWtchrChan.Receive() | evtWtchrEvt := evtWtchrChan.Receive() | ||||
| conColEvt := conColChan.Receive() | conColEvt := conColChan.Receive() | ||||
| clsterEvt := clsterCh.Receive() | |||||
| acStatEvt := acStatChan.Receive() | acStatEvt := acStatChan.Receive() | ||||
| spaceSyncEvt := spaceSyncChan.Receive() | spaceSyncEvt := spaceSyncChan.Receive() | ||||
| @@ -320,6 +322,23 @@ loop: | |||||
| } | } | ||||
| conColEvt = conColChan.Receive() | 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(): | case e := <-acStatEvt.Chan(): | ||||
| if e.Err != nil { | if e.Err != nil { | ||||
| logger.Errorf("receive access stat event: %v", err) | 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 | 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) | db, err := db.NewDB(&config.Cfg().DB) | ||||
| if err != nil { | if err != nil { | ||||
| @@ -172,6 +164,15 @@ func vfsTest(configPath string, opts serveHTTPOptions) { | |||||
| defer conCol.Stop() | defer conCol.Stop() | ||||
| conCol.CollectInPlace() | 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 := publock.NewMaster() | ||||
| publock.Start() | publock.Start() | ||||
| @@ -218,7 +219,7 @@ func vfsTest(configPath string, opts serveHTTPOptions) { | |||||
| mntChan := mnt.Start() | mntChan := mnt.Start() | ||||
| defer mnt.Stop() | 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接口 | // HTTP接口 | ||||
| httpCfgJSON := config.Cfg().HTTP | httpCfgJSON := config.Cfg().HTTP | ||||
| @@ -255,6 +256,7 @@ func vfsTest(configPath string, opts serveHTTPOptions) { | |||||
| evtPubEvt := evtPubChan.Receive() | evtPubEvt := evtPubChan.Receive() | ||||
| evtWtchrEvt := evtWtchrChan.Receive() | evtWtchrEvt := evtWtchrChan.Receive() | ||||
| conColEvt := conColChan.Receive() | conColEvt := conColChan.Receive() | ||||
| clsterEvt := clsterCh.Receive() | |||||
| acStatEvt := acStatChan.Receive() | acStatEvt := acStatChan.Receive() | ||||
| spaceSyncEvt := spaceSyncChan.Receive() | spaceSyncEvt := spaceSyncChan.Receive() | ||||
| httpEvt := httpChan.Receive() | httpEvt := httpChan.Receive() | ||||
| @@ -341,6 +343,23 @@ loop: | |||||
| case connectivity.CollectedEvent: | 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(): | case e := <-acStatEvt.Chan(): | ||||
| if e.Err != nil { | if e.Err != nil { | ||||
| logger.Errorf("receive access stat event: %v", err) | logger.Errorf("receive access stat event: %v", err) | ||||
| @@ -34,7 +34,7 @@ type Config struct { | |||||
| RPC rpc.Config `json:"rpc"` | RPC rpc.Config `json:"rpc"` | ||||
| Mount *mntcfg.Config `json:"mount"` | Mount *mntcfg.Config `json:"mount"` | ||||
| AccessToken *accesstoken.Config `json:"accessToken"` | AccessToken *accesstoken.Config `json:"accessToken"` | ||||
| Cluster *cluster.Config `json:"cluster"` | |||||
| Cluster cluster.Config `json:"cluster"` | |||||
| } | } | ||||
| var cfg Config | var cfg Config | ||||
| @@ -27,7 +27,7 @@ func (p *ClusterProxy) Proxy(c *gin.Context) { | |||||
| return | return | ||||
| } | } | ||||
| mstCli := p.clster.MasterClient().Get() | |||||
| mstCli := p.clster.MasterClient() | |||||
| defer mstCli.Release() | defer mstCli.Release() | ||||
| preq := clirpc.HTTPProxyRequest{ | 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.PubShardsImportPackagePath, certAuth, s.PubShards().ImportPackage) | ||||
| rt.POST(cliapi.SysEventWatchPath, certAuth, s.SysEvent().Watch) | 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 { | type PubLock struct { | ||||
| core *Core | core *Core | ||||
| cliCli *clirpc.Client | |||||
| cliCli *clirpc.PoolClient | |||||
| pubChan clirpc.PubLockMessageChan | pubChan clirpc.PubLockMessageChan | ||||
| lock sync.Mutex | lock sync.Mutex | ||||
| acquirings map[string]*acquireInfo | 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{ | return &PubLock{ | ||||
| cliCli: cli, | cliCli: cli, | ||||
| acquirings: make(map[string]*acquireInfo), | acquirings: make(map[string]*acquireInfo), | ||||
| @@ -3,6 +3,7 @@ package rpc | |||||
| import ( | import ( | ||||
| "context" | "context" | ||||
| "gitlink.org.cn/cloudream/jcs-pub/common/ecode" | |||||
| "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" | "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" | ||||
| clirpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/client" | 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) { | func (s *Service) ClusterRaftInstallSnapshot(ctx context.Context, msg *clirpc.ClusterRaftInstallSnapshot) (*clirpc.ClusterRaftInstallSnapshotResp, *rpc.CodeError) { | ||||
| return s.cluster.RaftTransport().OnInstallSnapshot(msg) | 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 ( | import ( | ||||
| "gitlink.org.cn/cloudream/jcs-pub/client/internal/accessstat" | "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/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/db" | ||||
| "gitlink.org.cn/cloudream/jcs-pub/client/internal/downloader" | "gitlink.org.cn/cloudream/jcs-pub/client/internal/downloader" | ||||
| "gitlink.org.cn/cloudream/jcs-pub/client/internal/downloader/strategy" | "gitlink.org.cn/cloudream/jcs-pub/client/internal/downloader/strategy" | ||||
| @@ -34,6 +35,7 @@ type Service struct { | |||||
| TickTock *ticktock.TickTock | TickTock *ticktock.TickTock | ||||
| SpeedStats *speedstats.SpeedStats | SpeedStats *speedstats.SpeedStats | ||||
| AccToken *accesstoken.Keeper | AccToken *accesstoken.Keeper | ||||
| Cluster *cluster.Cluster | |||||
| } | } | ||||
| func NewService( | func NewService( | ||||
| @@ -52,6 +54,7 @@ func NewService( | |||||
| tickTock *ticktock.TickTock, | tickTock *ticktock.TickTock, | ||||
| speedStats *speedstats.SpeedStats, | speedStats *speedstats.SpeedStats, | ||||
| accToken *accesstoken.Keeper, | accToken *accesstoken.Keeper, | ||||
| cluster *cluster.Cluster, | |||||
| ) *Service { | ) *Service { | ||||
| return &Service{ | return &Service{ | ||||
| PubLock: publock, | PubLock: publock, | ||||
| @@ -69,5 +72,6 @@ func NewService( | |||||
| TickTock: tickTock, | TickTock: tickTock, | ||||
| SpeedStats: speedStats, | SpeedStats: speedStats, | ||||
| AccToken: accToken, | 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, | "userSpaceID": 0, | ||||
| "maxBodySize": 5242880 | "maxBodySize": 5242880 | ||||
| }, | }, | ||||
| "rpc": { | |||||
| "listen": "127.0.0.1:7990", | |||||
| "rootCA": "", | |||||
| "serverCert": "", | |||||
| "serverKey": "" | |||||
| }, | |||||
| "mount": { | "mount": { | ||||
| "enabled": false, | "enabled": false, | ||||
| "mountPoint": "", | "mountPoint": "", | ||||
| @@ -66,5 +72,15 @@ | |||||
| "accessToken": { | "accessToken": { | ||||
| "account": "", | "account": "", | ||||
| "password": "" | "password": "" | ||||
| }, | |||||
| "cluster": { | |||||
| "enabled": false, | |||||
| "nodeName": "", | |||||
| "announce": "127.0.0.1:7990", | |||||
| "peers": [], | |||||
| "rootCA": "", | |||||
| "clientCert": "", | |||||
| "clientKey": "", | |||||
| "storeBase": "" | |||||
| } | } | ||||
| } | } | ||||
| @@ -11,4 +11,5 @@ const ( | |||||
| TaskNotFound ErrorCode = "TaskNotFound" | TaskNotFound ErrorCode = "TaskNotFound" | ||||
| Unauthorized ErrorCode = "Unauthorized" | Unauthorized ErrorCode = "Unauthorized" | ||||
| ChannelClosed ErrorCode = "ChannelClosed" | ChannelClosed ErrorCode = "ChannelClosed" | ||||
| ClusterNoMaster ErrorCode = "ClusterNoMaster" | |||||
| ) | ) | ||||
| @@ -8,11 +8,28 @@ import ( | |||||
| type Client struct { | type Client struct { | ||||
| con *grpc.ClientConn | con *grpc.ClientConn | ||||
| cli ClientClient | cli ClientClient | ||||
| pool *Pool | |||||
| fusedErr *rpc.CodeError | 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 { | if c.con != nil { | ||||
| c.pool.connPool.Release(c.pool.cfg.Address) | 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, | 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, | 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, | 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, | 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, | 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, | 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, | 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, | 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, | 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, | 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, | 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, | 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, // 3: clirpc.Client.GetClusterMasterInfo:input_type -> rpc.Request | ||||
| 3, // 4: clirpc.Client.ClusterRaftRPC:input_type -> rpc.Request | 3, // 4: clirpc.Client.ClusterRaftRPC:input_type -> rpc.Request | ||||
| 4, // 5: clirpc.Client.ClusterRaftInstallSnapshot:input_type -> rpc.ChunkedData | 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 type_name | ||||
| 2, // [2:2] is the sub-list for extension extendee | 2, // [2:2] is the sub-list for extension extendee | ||||
| 0, // [0:2] is the sub-list for field type_name | 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 GetClusterMasterInfo(rpc.Request) returns(rpc.Response); | ||||
| rpc ClusterRaftRPC(rpc.Request) returns(rpc.Response); | rpc ClusterRaftRPC(rpc.Request) returns(rpc.Response); | ||||
| rpc ClusterRaftInstallSnapshot(stream rpc.ChunkedData) returns(rpc.Response); | rpc ClusterRaftInstallSnapshot(stream rpc.ChunkedData) returns(rpc.Response); | ||||
| rpc ClusterApplyLog(rpc.Request) returns(rpc.Response); | |||||
| rpc HTTPProxy(HTTPProxyRequest) returns(HTTPProxyResponse); | rpc HTTPProxy(HTTPProxyRequest) returns(HTTPProxyResponse); | ||||
| @@ -24,6 +24,7 @@ const ( | |||||
| Client_GetClusterMasterInfo_FullMethodName = "/clirpc.Client/GetClusterMasterInfo" | Client_GetClusterMasterInfo_FullMethodName = "/clirpc.Client/GetClusterMasterInfo" | ||||
| Client_ClusterRaftRPC_FullMethodName = "/clirpc.Client/ClusterRaftRPC" | Client_ClusterRaftRPC_FullMethodName = "/clirpc.Client/ClusterRaftRPC" | ||||
| Client_ClusterRaftInstallSnapshot_FullMethodName = "/clirpc.Client/ClusterRaftInstallSnapshot" | Client_ClusterRaftInstallSnapshot_FullMethodName = "/clirpc.Client/ClusterRaftInstallSnapshot" | ||||
| Client_ClusterApplyLog_FullMethodName = "/clirpc.Client/ClusterApplyLog" | |||||
| Client_HTTPProxy_FullMethodName = "/clirpc.Client/HTTPProxy" | 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) | 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) | ClusterRaftRPC(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) | ||||
| ClusterRaftInstallSnapshot(ctx context.Context, opts ...grpc.CallOption) (Client_ClusterRaftInstallSnapshotClient, 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) | HTTPProxy(ctx context.Context, in *HTTPProxyRequest, opts ...grpc.CallOption) (*HTTPProxyResponse, error) | ||||
| } | } | ||||
| @@ -129,6 +131,15 @@ func (x *clientClusterRaftInstallSnapshotClient) CloseAndRecv() (*rpc.Response, | |||||
| return m, nil | 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) { | func (c *clientClient) HTTPProxy(ctx context.Context, in *HTTPProxyRequest, opts ...grpc.CallOption) (*HTTPProxyResponse, error) { | ||||
| out := new(HTTPProxyResponse) | out := new(HTTPProxyResponse) | ||||
| err := c.cc.Invoke(ctx, Client_HTTPProxy_FullMethodName, in, out, opts...) | 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) | GetClusterMasterInfo(context.Context, *rpc.Request) (*rpc.Response, error) | ||||
| ClusterRaftRPC(context.Context, *rpc.Request) (*rpc.Response, error) | ClusterRaftRPC(context.Context, *rpc.Request) (*rpc.Response, error) | ||||
| ClusterRaftInstallSnapshot(Client_ClusterRaftInstallSnapshotServer) error | ClusterRaftInstallSnapshot(Client_ClusterRaftInstallSnapshotServer) error | ||||
| ClusterApplyLog(context.Context, *rpc.Request) (*rpc.Response, error) | |||||
| HTTPProxy(context.Context, *HTTPProxyRequest) (*HTTPProxyResponse, error) | HTTPProxy(context.Context, *HTTPProxyRequest) (*HTTPProxyResponse, error) | ||||
| mustEmbedUnimplementedClientServer() | mustEmbedUnimplementedClientServer() | ||||
| } | } | ||||
| @@ -166,6 +178,9 @@ func (UnimplementedClientServer) ClusterRaftRPC(context.Context, *rpc.Request) ( | |||||
| func (UnimplementedClientServer) ClusterRaftInstallSnapshot(Client_ClusterRaftInstallSnapshotServer) error { | func (UnimplementedClientServer) ClusterRaftInstallSnapshot(Client_ClusterRaftInstallSnapshotServer) error { | ||||
| return status.Errorf(codes.Unimplemented, "method ClusterRaftInstallSnapshot not implemented") | 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) { | func (UnimplementedClientServer) HTTPProxy(context.Context, *HTTPProxyRequest) (*HTTPProxyResponse, error) { | ||||
| return nil, status.Errorf(codes.Unimplemented, "method HTTPProxy not implemented") | return nil, status.Errorf(codes.Unimplemented, "method HTTPProxy not implemented") | ||||
| } | } | ||||
| @@ -270,6 +285,24 @@ func (x *clientClusterRaftInstallSnapshotServer) Recv() (*rpc.ChunkedData, error | |||||
| return m, nil | 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) { | func _Client_HTTPProxy_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { | ||||
| in := new(HTTPProxyRequest) | in := new(HTTPProxyRequest) | ||||
| if err := dec(in); err != nil { | if err := dec(in); err != nil { | ||||
| @@ -303,6 +336,10 @@ var Client_ServiceDesc = grpc.ServiceDesc{ | |||||
| MethodName: "ClusterRaftRPC", | MethodName: "ClusterRaftRPC", | ||||
| Handler: _Client_ClusterRaftRPC_Handler, | Handler: _Client_ClusterRaftRPC_Handler, | ||||
| }, | }, | ||||
| { | |||||
| MethodName: "ClusterApplyLog", | |||||
| Handler: _Client_ClusterApplyLog_Handler, | |||||
| }, | |||||
| { | { | ||||
| MethodName: "HTTPProxy", | MethodName: "HTTPProxy", | ||||
| Handler: _Client_HTTPProxy_Handler, | Handler: _Client_HTTPProxy_Handler, | ||||
| @@ -3,6 +3,7 @@ package clirpc | |||||
| import ( | import ( | ||||
| "context" | "context" | ||||
| "io" | "io" | ||||
| "time" | |||||
| "github.com/hashicorp/raft" | "github.com/hashicorp/raft" | ||||
| "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" | "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) | GetClusterMasterInfo(ctx context.Context, req *GetClusterMasterInfo) (*GetClusterMasterInfoResp, *rpc.CodeError) | ||||
| ClusterRaftRPC(ctx context.Context, req *ClusterRaftRPC) (*ClusterRaftRPCResp, *rpc.CodeError) | ClusterRaftRPC(ctx context.Context, req *ClusterRaftRPC) (*ClusterRaftRPCResp, *rpc.CodeError) | ||||
| ClusterRaftInstallSnapshot(ctx context.Context, req *ClusterRaftInstallSnapshot) (*ClusterRaftInstallSnapshotResp, *rpc.CodeError) | ClusterRaftInstallSnapshot(ctx context.Context, req *ClusterRaftInstallSnapshot) (*ClusterRaftInstallSnapshotResp, *rpc.CodeError) | ||||
| ClusterApplyLog(ctx context.Context, req *ClusterApplyLog) (*ClusterApplyLogResp, *rpc.CodeError) | |||||
| } | } | ||||
| type GetClusterMasterInfo struct { | type GetClusterMasterInfo struct { | ||||
| @@ -80,3 +82,20 @@ func (c *Client) ClusterRaftInstallSnapshot(ctx context.Context, msg *ClusterRaf | |||||
| func (s *Server) ClusterRaftInstallSnapshot(req Client_ClusterRaftInstallSnapshotServer) error { | func (s *Server) ClusterRaftInstallSnapshot(req Client_ClusterRaftInstallSnapshotServer) error { | ||||
| return rpc.UploadStreamServer(s.svrImpl.ClusterRaftInstallSnapshot, req) | 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{ | Client: Client{ | ||||
| con: gcon, | con: gcon, | ||||
| cli: NewClientClient(gcon), | cli: NewClientClient(gcon), | ||||
| pool: nil, | |||||
| fusedErr: nil, | fusedErr: nil, | ||||
| }, | }, | ||||
| }, 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) | con, err := p.connPool.GetConnection(p.cfg.Address) | ||||
| if err != nil { | 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 { | 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() | authInfo, err := p.cfg.AccessTokenProvider.MakeAuthInfo() | ||||
| if err != nil { | 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/presigned-url v1.12.15 // indirect | ||||
| github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.6 // 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/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 v1.11.6 // indirect | ||||
| github.com/bytedance/sonic/loader v0.1.1 // indirect | github.com/bytedance/sonic/loader v0.1.1 // indirect | ||||
| github.com/cloudwego/base64x v0.1.4 // 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-hclog v1.6.2 // indirect | ||||
| github.com/hashicorp/go-immutable-radix v1.0.0 // indirect | github.com/hashicorp/go-immutable-radix v1.0.0 // indirect | ||||
| github.com/hashicorp/go-metrics v0.5.4 // 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/go-msgpack/v2 v2.1.2 // indirect | ||||
| github.com/hashicorp/golang-lru v0.5.0 // indirect | github.com/hashicorp/golang-lru v0.5.0 // indirect | ||||
| github.com/hashicorp/raft v1.7.3 // 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/inflection v1.0.0 // indirect | ||||
| github.com/jinzhu/now v1.1.5 // indirect | github.com/jinzhu/now v1.1.5 // indirect | ||||
| github.com/jonboulle/clockwork v0.4.0 // 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= | filippo.io/edwards25519 v1.1.0/go.mod h1:BxyFTGdWcka3PhytdK4V28tE5sGfRvvvRV7EaN4VDT4= | ||||
| github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= | 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/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-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= | ||||
| github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/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= | 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 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.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= | ||||
| github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= | 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 h1:oUp34TzMlL+OY1OUWxHqsdkgC/Zfc85zGqw9siXjrc0= | ||||
| github.com/bytedance/sonic v1.11.6/go.mod h1:LysEHSvpvDySVdC2f87zGWf6CIKJcAvqab1ZaiQtds4= | github.com/bytedance/sonic v1.11.6/go.mod h1:LysEHSvpvDySVdC2f87zGWf6CIKJcAvqab1ZaiQtds4= | ||||
| github.com/bytedance/sonic/loader v0.1.1 h1:c+e5Pt1k/cy5wMveRDyk2X4B9hF4g7an8N3zCYjJFNM= | 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.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 h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= | ||||
| github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= | 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 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= | ||||
| github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= | 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= | 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.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= | ||||
| github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= | 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.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.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.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.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= | ||||
| github.com/google/go-cmp v0.3.1/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.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.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.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 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= | ||||
| github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= | 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= | 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-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 h1:8mmPiIJkTPPEbAiV97IxdAGNdRdaWwVap1BU6elejKY= | ||||
| github.com/hashicorp/go-metrics v0.5.4/go.mod h1:CG5yz4NZ/AI/aQt9Ucm/vdBnbh7fvmv4lxZ350i+QQI= | 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 h1:4Ee8FTp834e+ewB71RDrQ0VKpyFdrKOjvYtnQ/ltVj0= | ||||
| github.com/hashicorp/go-msgpack/v2 v2.1.2/go.mod h1:upybraOAblm4S7rx0+jeNy+CWWhzywQsSRV5033mMu4= | 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= | 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/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 h1:DxpEqZJysHN0wK+fviai5mFcSYsCkNpFUl1xpAW8Rbo= | ||||
| github.com/hashicorp/raft v1.7.3/go.mod h1:DfvCGFxpAUPE0L4Uc8JLlTPtc3GzSbdH0MTJCLgnmJQ= | 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 h1:XQVXdk+WAJ4fSNB6mMRuYNvFWou7BZs6SZB925hPrnk= | ||||
| github.com/huaweicloud/huaweicloud-sdk-go-obs v3.24.9+incompatible/go.mod h1:l7VUhRbTKCzdOacdT4oWCwATKyvZqUOlOqr0Ous3k4s= | 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= | 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/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 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST4RZ4= | ||||
| github.com/jonboulle/clockwork v0.4.0/go.mod h1:xgRqUGwRcjKCO1vbZUEtSLrqKoPSsUpK7fnezOII0kc= | 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/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.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= | ||||
| github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= | 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/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/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.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 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= | ||||
| github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= | github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= | ||||
| github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= | 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/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 h1:BvGheCMAsG3bWUDbZ8AyXXpCNwU9u5CB6sM+HNb9HYg= | ||||
| github.com/magefile/mage v1.15.0/go.mod h1:z5UZb/iS3GoOSn0JgWuiw7dxlurVYTu+/jHXqQg881A= | 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.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.7/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= | ||||
| github.com/mattn/go-colorable v0.1.9/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/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 h1:aYUidT7k73Pcl9nb2gScu7NSrKCSHIDE89b3+6Wq+LM= | ||||
| github.com/pelletier/go-toml/v2 v2.2.2/go.mod h1:1t835xjRzz80PqgE6HHgN2JOsmgYu/h4qDAS4n929Rs= | 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/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.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= | ||||
| github.com/pkg/errors v0.8.1/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.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 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= | ||||
| github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= | 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 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.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= | ||||
| github.com/prometheus/client_golang v1.4.0/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= | 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.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.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.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.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.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 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= | ||||
| github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= | 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 h1:aMe1GlZb+0bLjn+cKTPEvvn9oUEBlJitaZiiBwsbgho= | ||||
| github.com/tjfoc/gmsm v1.4.1/go.mod h1:j4INPkHWMrhJb38G+J6W4Tw0AbuN8Thu3PbdVYhVcTE= | github.com/tjfoc/gmsm v1.4.1/go.mod h1:j4INPkHWMrhJb38G+J6W4Tw0AbuN8Thu3PbdVYhVcTE= | ||||
| github.com/tmaxmax/go-sse v0.11.0 h1:nogmJM6rJUoOLoAwEKeQe5XlVpt9l7N82SS1jI7lWFg= | 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-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-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.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.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.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= | ||||
| golang.org/x/crypto v0.38.0 h1:jt+WWG8IZlBnVbomuhg2Mdq0+BBQaHbtqHEFEigjUV8= | 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-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= | ||||
| golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= | 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.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.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-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-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= | ||||
| golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/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-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-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-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-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-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= | ||||
| golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= | 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-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-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.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.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.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.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= | ||||
| golang.org/x/net v0.35.0 h1:T5GQRQb2y08kTAByq9L4/bz8cipCdA8FbRTXewonqY8= | golang.org/x/net v0.35.0 h1:T5GQRQb2y08kTAByq9L4/bz8cipCdA8FbRTXewonqY8= | ||||
| golang.org/x/net v0.35.0/go.mod h1:EglIi67kWsHKlRzzVMUD93VMSWGFOMSZgxFjparz1Qk= | 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-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.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.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 h1:woo0S4Yywslg6hp4eUFjTVOyKt0RookbpAHG4c1HmhQ= | ||||
| golang.org/x/sync v0.14.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= | 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= | 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-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-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.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.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | ||||
| golang.org/x/sys v0.6.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.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.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.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 h1:q3i8TbbEz+JRD9ywIRlyRAQbM0qF7hu24q3teo2hbuw= | ||||
| golang.org/x/sys v0.33.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= | 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-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.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.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.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.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= | ||||
| golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= | golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= | ||||
| golang.org/x/term v0.32.0 h1:DR4lr0TjUs3epypdhTOkMmuF5CDFJ/8pOnbzMZPQ7bg= | 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.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.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= | ||||
| golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= | 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.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.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.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= | ||||
| golang.org/x/text v0.15.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= | 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-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.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.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.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-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= | ||||
| golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/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.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.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-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= | ||||
| google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= | 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= | 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.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= | ||||
| google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= | 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-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 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= | ||||
| google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= | 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= | 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/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 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= | ||||
| gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= | 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.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.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.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.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.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.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 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= | ||||
| gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= | 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 ( | import ( | ||||
| _ "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/admin" | _ "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/sysevent" | ||||
| _ "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/admin/ticktock" | _ "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/admin/ticktock" | ||||
| _ "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/bucket" | _ "gitlink.org.cn/cloudream/jcs-pub/jcsctl/cmd/bucket" | ||||