Browse Source

按照实现集群raft要求的接口

master
Sydonian 2 months ago
parent
commit
32f1b25c5f
29 changed files with 818 additions and 107 deletions
  1. +248
    -37
      client/internal/cluster/cluster.go
  2. +8
    -6
      client/internal/cluster/config.go
  3. +67
    -0
      client/internal/cluster/fsm.go
  4. +76
    -10
      client/internal/cluster/transport.go
  5. +28
    -9
      client/internal/cmdline/serve.go
  6. +28
    -9
      client/internal/cmdline/test.go
  7. +28
    -9
      client/internal/cmdline/vfstest.go
  8. +1
    -1
      client/internal/config/config.go
  9. +1
    -1
      client/internal/http/proxy/proxy.go
  10. +46
    -0
      client/internal/http/v1/cluster.go
  11. +2
    -0
      client/internal/http/v1/server.go
  12. +2
    -2
      client/internal/publock/service.go
  13. +10
    -0
      client/internal/rpc/cluster.go
  14. +4
    -0
      client/internal/services/service.go
  15. +34
    -0
      client/sdk/api/v1/cluster.go
  16. +16
    -0
      common/assets/confs/client.config.json
  17. +1
    -0
      common/ecode/ecode.go
  18. +19
    -2
      common/pkgs/rpc/client/client.go
  19. +14
    -9
      common/pkgs/rpc/client/client.pb.go
  20. +1
    -0
      common/pkgs/rpc/client/client.proto
  21. +37
    -0
      common/pkgs/rpc/client/client_grpc.pb.go
  22. +19
    -0
      common/pkgs/rpc/client/cluster.go
  23. +15
    -12
      common/pkgs/rpc/client/pool.go
  24. +10
    -0
      common/pkgs/rpc/pool.go
  25. +3
    -0
      go.mod
  26. +50
    -0
      go.sum
  27. +15
    -0
      jcsctl/cmd/admin/cluster/cluster.go
  28. +34
    -0
      jcsctl/cmd/admin/cluster/status.go
  29. +1
    -0
      jcsctl/cmd/all/all.go

+ 248
- 37
client/internal/cluster/cluster.go View File

@@ -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() {

} }

+ 8
- 6
client/internal/cluster/config.go View File

@@ -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"` // 集群信息的存储路径,必须是一个目录
} }

+ 67
- 0
client/internal/cluster/fsm.go View File

@@ -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() {
}

+ 76
- 10
client/internal/cluster/transport.go View File

@@ -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


+ 28
- 9
client/internal/cmdline/serve.go View File

@@ -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)


+ 28
- 9
client/internal/cmdline/test.go View File

@@ -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)


+ 28
- 9
client/internal/cmdline/vfstest.go View File

@@ -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)


+ 1
- 1
client/internal/config/config.go View File

@@ -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


+ 1
- 1
client/internal/http/proxy/proxy.go View File

@@ -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{


+ 46
- 0
client/internal/http/v1/cluster.go View File

@@ -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{}))
}

+ 2
- 0
client/internal/http/v1/server.go View File

@@ -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)
} }

+ 2
- 2
client/internal/publock/service.go View File

@@ -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),


+ 10
- 0
client/internal/rpc/cluster.go View File

@@ -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
}

+ 4
- 0
client/internal/services/service.go View File

@@ -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,
} }
} }

+ 34
- 0
client/sdk/api/v1/cluster.go View File

@@ -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{})
}

+ 16
- 0
common/assets/confs/client.config.json View File

@@ -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": ""
} }
} }

+ 1
- 0
common/ecode/ecode.go View File

@@ -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"
) )

+ 19
- 2
common/pkgs/rpc/client/client.go View File

@@ -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)
} }


+ 14
- 9
common/pkgs/rpc/client/client.pb.go View File

@@ -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


+ 1
- 0
common/pkgs/rpc/client/client.proto View File

@@ -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);

+ 37
- 0
common/pkgs/rpc/client/client_grpc.pb.go View File

@@ -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,


+ 19
- 0
common/pkgs/rpc/client/cluster.go View File

@@ -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)
}

+ 15
- 12
common/pkgs/rpc/client/pool.go View File

@@ -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,
} }
} }

+ 10
- 0
common/pkgs/rpc/pool.go View File

@@ -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 {


+ 3
- 0
go.mod View File

@@ -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


+ 50
- 0
go.sum View File

@@ -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=


+ 15
- 0
jcsctl/cmd/admin/cluster/cluster.go View File

@@ -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)
}

+ 34
- 0
jcsctl/cmd/admin/cluster/status.go View File

@@ -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
}

+ 1
- 0
jcsctl/cmd/all/all.go View File

@@ -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"


Loading…
Cancel
Save