diff --git a/client/internal/cmdline/serve.go b/client/internal/cmdline/serve.go index e5c1043..50fd85f 100644 --- a/client/internal/cmdline/serve.go +++ b/client/internal/cmdline/serve.go @@ -68,7 +68,7 @@ func serveHTTP(configPath string, opts serveHTTPOptions) { stgglb.InitLocal(config.Cfg().Local) stgglb.InitMQPool(config.Cfg().RabbitMQ) - stgglb.InitHubRPCPool(&config.Cfg().HubGRPC) + stgglb.InitHubRPCPool(config.Cfg().HubRPC) // 数据库 db, err := db.NewDB(&config.Cfg().DB) diff --git a/client/internal/cmdline/test.go b/client/internal/cmdline/test.go index 0f876e2..94c0f55 100644 --- a/client/internal/cmdline/test.go +++ b/client/internal/cmdline/test.go @@ -81,7 +81,7 @@ func test(configPath string) { stgglb.InitLocal(config.Cfg().Local) stgglb.InitMQPool(config.Cfg().RabbitMQ) - stgglb.InitHubRPCPool(&config.Cfg().HubGRPC) + stgglb.InitHubRPCPool(config.Cfg().HubRPC) // 数据库 db, err := db.NewDB(&config.Cfg().DB) diff --git a/client/internal/cmdline/vfstest.go b/client/internal/cmdline/vfstest.go index 8e739a5..443af32 100644 --- a/client/internal/cmdline/vfstest.go +++ b/client/internal/cmdline/vfstest.go @@ -61,7 +61,7 @@ func vfsTest(configPath string, opts serveHTTPOptions) { stgglb.InitLocal(config.Cfg().Local) stgglb.InitMQPool(config.Cfg().RabbitMQ) - stgglb.InitHubRPCPool(&config.Cfg().HubGRPC) + stgglb.InitHubRPCPool(config.Cfg().HubRPC) // 数据库 db, err := db.NewDB(&config.Cfg().DB) diff --git a/client/internal/config/config.go b/client/internal/config/config.go index c2c519a..d8841dc 100644 --- a/client/internal/config/config.go +++ b/client/internal/config/config.go @@ -12,12 +12,12 @@ import ( "gitlink.org.cn/cloudream/jcs-pub/client/internal/ticktock" stgglb "gitlink.org.cn/cloudream/jcs-pub/common/globals" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/connectivity" - hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/grpc/hub" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" ) type Config struct { Local stgglb.LocalMachineInfo `json:"local"` - HubGRPC hubrpc.PoolConfig `json:"hubGRPC"` + HubRPC hubrpc.PoolConfig `json:"hubRPC"` Logger logger.Config `json:"logger"` DB db.Config `json:"db"` RabbitMQ mq.Config `json:"rabbitMQ"` diff --git a/client/internal/services/user_space.go b/client/internal/services/user_space.go index 9c1eb72..e3fa1c9 100644 --- a/client/internal/services/user_space.go +++ b/client/internal/services/user_space.go @@ -11,6 +11,7 @@ import ( "gitlink.org.cn/cloudream/common/pkgs/trie" cdssdk "gitlink.org.cn/cloudream/common/sdks/storage" clitypes "gitlink.org.cn/cloudream/jcs-pub/client/types" + cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" "gitlink.org.cn/cloudream/jcs-pub/client/internal/db" "gitlink.org.cn/cloudream/jcs-pub/client/internal/downloader/strategy" @@ -18,7 +19,7 @@ import ( "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/distlock/reqbuilder" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/ioswitch2" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/ioswitch2/parser" - hubmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/hub" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/storage/types" ) @@ -126,11 +127,13 @@ func (svc *UserSpaceService) SpaceToSpace(srcSpaceID clitypes.UserSpaceID, srcPa if srcSpace.MasterHub == nil { return clitypes.SpaceToSpaceResult{}, fmt.Errorf("source userspace %v has no master hub", srcSpaceID) } - srcSpaceCli, err := stgglb.HubMQPool.Acquire(srcSpace.MasterHub.HubID) - if err != nil { - return clitypes.SpaceToSpaceResult{}, fmt.Errorf("new source userspace client: %w", err) + + srcAddr, ok := srcSpace.MasterHub.Address.(*cortypes.GRPCAddressInfo) + if !ok { + return clitypes.SpaceToSpaceResult{}, fmt.Errorf("source userspace %v has no grpc address", srcSpaceID) } - defer stgglb.HubMQPool.Release(srcSpaceCli) + srcSpaceCli := stgglb.HubRPCPool.Get(stgglb.SelectGRPCAddress(*srcSpace.MasterHub, *srcAddr)) + defer srcSpaceCli.Release() dstSpace := svc.UserSpaceMeta.Get(dstSpaceID) if dstSpace == nil { @@ -139,11 +142,12 @@ func (svc *UserSpaceService) SpaceToSpace(srcSpaceID clitypes.UserSpaceID, srcPa if dstSpace.MasterHub == nil { return clitypes.SpaceToSpaceResult{}, fmt.Errorf("destination userspace %v has no master hub", dstSpaceID) } - dstSpaceCli, err := stgglb.HubMQPool.Acquire(dstSpace.MasterHub.HubID) - if err != nil { - return clitypes.SpaceToSpaceResult{}, fmt.Errorf("new destination userspace client: %w", err) + dstAddr, ok := dstSpace.MasterHub.Address.(*cortypes.GRPCAddressInfo) + if !ok { + return clitypes.SpaceToSpaceResult{}, fmt.Errorf("destination userspace %v has no grpc address", srcSpaceID) } - defer stgglb.HubMQPool.Release(dstSpaceCli) + dstSpaceCli := stgglb.HubRPCPool.Get(stgglb.SelectGRPCAddress(*dstSpace.MasterHub, *dstAddr)) + defer dstSpaceCli.Release() srcPath = strings.Trim(srcPath, cdssdk.ObjectPathSeparator) dstPath = strings.Trim(dstPath, cdssdk.ObjectPathSeparator) @@ -156,12 +160,12 @@ func (svc *UserSpaceService) SpaceToSpace(srcSpaceID clitypes.UserSpaceID, srcPa return clitypes.SpaceToSpaceResult{}, fmt.Errorf("destination path is empty") } - listAllResp, err := srcSpaceCli.PublicStoreListAll(&hubmq.PublicStoreListAll{ + listAllResp, cerr := srcSpaceCli.PublicStoreListAll(context.Background(), &hubrpc.PublicStoreListAll{ UserSpace: *srcSpace, Path: srcPath, }) - if err != nil { - return clitypes.SpaceToSpaceResult{}, fmt.Errorf("list all from source userspace: %w", err) + if cerr != nil { + return clitypes.SpaceToSpaceResult{}, fmt.Errorf("list all from source userspace: %w", cerr) } srcPathComps := clitypes.SplitObjectPath(srcPath) @@ -222,17 +226,17 @@ func (svc *UserSpaceService) SpaceToSpace(srcSpaceID clitypes.UserSpaceID, srcPa ft.AddTo(ioswitch2.NewToPublicStore(*dstSpace.MasterHub, *dstSpace, newPath)) plans := exec.NewPlanBuilder() - err = parser.Parse(ft, plans) + err := parser.Parse(ft, plans) if err != nil { failed = append(failed, f) logger.Warnf("s2s: parse plan of file %v: %v", f, err) continue } - _, err = plans.Execute(exec.NewExecContext()).Wait(context.Background()) - if err != nil { + _, cerr := plans.Execute(exec.NewExecContext()).Wait(context.Background()) + if cerr != nil { failed = append(failed, f) - logger.Warnf("s2s: execute plan of file %v: %v", f, err) + logger.Warnf("s2s: execute plan of file %v: %v", f, cerr) continue } @@ -244,7 +248,7 @@ func (svc *UserSpaceService) SpaceToSpace(srcSpaceID clitypes.UserSpaceID, srcPa newDirPathes = append(newDirPathes, strings.Replace(dirPathes[i], srcPath, dstPath, 1)) } - mkdirResp, err := dstSpaceCli.PublicStoreMkdirs(&hubmq.PublicStoreMkdirs{ + mkdirResp, err := dstSpaceCli.PublicStoreMkdirs(context.Background(), &hubrpc.PublicStoreMkdirs{ UserSpace: *dstSpace, Pathes: newDirPathes, }) diff --git a/client/internal/ticktock/check_shardstore.go b/client/internal/ticktock/check_shardstore.go index 97fe7d6..57f4170 100644 --- a/client/internal/ticktock/check_shardstore.go +++ b/client/internal/ticktock/check_shardstore.go @@ -1,18 +1,19 @@ package ticktock import ( + "context" "fmt" "time" "github.com/samber/lo" "gitlink.org.cn/cloudream/common/pkgs/logger" - "gitlink.org.cn/cloudream/common/pkgs/mq" "gitlink.org.cn/cloudream/common/utils/reflect2" "gitlink.org.cn/cloudream/jcs-pub/client/internal/db" clitypes "gitlink.org.cn/cloudream/jcs-pub/client/types" stgglb "gitlink.org.cn/cloudream/jcs-pub/common/globals" - hubmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/hub" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" + cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" ) // CheckShardStore 代表一个用于处理代理缓存检查事件的结构体 @@ -62,13 +63,19 @@ func (j *CheckShardStore) checkOne(t *TickTock, space *clitypes.UserSpaceDetail) return nil } - agtCli, err := stgglb.HubMQPool.Acquire(space.MasterHub.HubID) - if err != nil { - return fmt.Errorf("new hub mq client: %w", err) + addr, ok := space.MasterHub.Address.(*cortypes.GRPCAddressInfo) + if !ok { + return fmt.Errorf("master of user space %v has no grpc address", space.UserSpace) } - defer stgglb.HubMQPool.Release(agtCli) + agtCli := stgglb.HubRPCPool.Get(stgglb.SelectGRPCAddress(*space.MasterHub, *addr)) + defer agtCli.Release() + + ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(time.Minute)) + defer cancel() - checkResp, err := agtCli.CheckCache(hubmq.NewCheckCache(*space), mq.RequestOption{Timeout: time.Minute}) + checkResp, err := agtCli.CheckCache(ctx, &hubrpc.CheckCache{ + UserSpace: *space, + }) if err != nil { return fmt.Errorf("request to check cache: %w", err) } diff --git a/client/internal/ticktock/shardstore_gc.go b/client/internal/ticktock/shardstore_gc.go index 9bea1c6..9cad74e 100644 --- a/client/internal/ticktock/shardstore_gc.go +++ b/client/internal/ticktock/shardstore_gc.go @@ -1,18 +1,19 @@ package ticktock import ( + "context" "fmt" "time" "gitlink.org.cn/cloudream/common/pkgs/logger" - "gitlink.org.cn/cloudream/common/pkgs/mq" "gitlink.org.cn/cloudream/common/utils/reflect2" "gitlink.org.cn/cloudream/jcs-pub/client/internal/db" "gitlink.org.cn/cloudream/jcs-pub/client/types" stgglb "gitlink.org.cn/cloudream/jcs-pub/common/globals" + cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/distlock/reqbuilder" - hubmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/hub" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" ) type ShardStoreGC struct { @@ -86,14 +87,21 @@ func (j *ShardStoreGC) gcOne(t *TickTock, space *types.UserSpaceDetail) error { } // 获取与节点通信的代理客户端 - agtCli, err := stgglb.HubMQPool.Acquire(space.MasterHub.HubID) - if err != nil { - return fmt.Errorf("new hub mq client: %w", err) + addr, ok := space.MasterHub.Address.(*cortypes.GRPCAddressInfo) + if !ok { + return fmt.Errorf("master of user space %v has no grpc address", space.UserSpace) } - defer stgglb.HubMQPool.Release(agtCli) + agtCli := stgglb.HubRPCPool.Get(stgglb.SelectGRPCAddress(*space.MasterHub, *addr)) + defer agtCli.Release() // 向代理发送垃圾回收请求 - _, err = agtCli.CacheGC(hubmq.ReqCacheGC(*space, allFileHashes), mq.RequestOption{Timeout: time.Minute}) + ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(time.Minute)) + defer cancel() + + _, err = agtCli.CacheGC(ctx, &hubrpc.CacheGC{ + UserSpace: *space, + Availables: allFileHashes, + }) if err != nil { return fmt.Errorf("request to cache gc: %w", err) } diff --git a/client/internal/uploader/user_space_upload.go b/client/internal/uploader/user_space_upload.go index 0f254e3..da3d1b3 100644 --- a/client/internal/uploader/user_space_upload.go +++ b/client/internal/uploader/user_space_upload.go @@ -16,7 +16,7 @@ import ( "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/ioswitch2/ops2" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/ioswitch2/parser" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/coordinator" - hubmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/hub" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/storage/types" cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" ) @@ -101,14 +101,15 @@ func (u *Uploader) UserSpaceUpload(userSpaceID clitypes.UserSpaceID, rootPath st targetSapce := u.chooseUploadStorage(uploadSpaces, uploadAffinity) - srcHubCli, err := stgglb.HubMQPool.Acquire(srcSpace.MasterHub.HubID) - if err != nil { + addr, ok := srcSpace.MasterHub.Address.(*cortypes.GRPCAddressInfo) + if !ok { delPkg() - return nil, fmt.Errorf("acquiring source hub mq client: %w", err) + return nil, fmt.Errorf("master of user space %v has no grpc address", srcSpace.UserSpace) } - defer stgglb.HubMQPool.Release(srcHubCli) + srcHubCli := stgglb.HubRPCPool.Get(stgglb.SelectGRPCAddress(*srcSpace.MasterHub, *addr)) + defer srcHubCli.Release() - listAllResp, err := srcHubCli.PublicStoreListAll(&hubmq.PublicStoreListAll{ + listAllResp, err := srcHubCli.PublicStoreListAll(context.Background(), &hubrpc.PublicStoreListAll{ UserSpace: *srcSpace, Path: rootPath, }) diff --git a/common/assets/confs/client.config.json b/common/assets/confs/client.config.json index 3915e04..4668194 100644 --- a/common/assets/confs/client.config.json +++ b/common/assets/confs/client.config.json @@ -5,9 +5,7 @@ "externalIP": "127.0.0.1", "locationID": 1 }, - "hubGRPC": { - "port": 5010 - }, + "hubRPC": {}, "logger": { "output": "stdout", "level": "debug" diff --git a/common/assets/confs/hub.config.json b/common/assets/confs/hub.config.json index 0fa8b4f..7f44e31 100644 --- a/common/assets/confs/hub.config.json +++ b/common/assets/confs/hub.config.json @@ -5,9 +5,8 @@ "externalIP": "127.0.0.1", "locationID": 1 }, - "grpc": { - "ip": "127.0.0.1", - "port": 5010 + "rpc": { + "listen": "127.0.0.1:5010" }, "logger": { "output": "file", diff --git a/common/globals/pools.go b/common/globals/pools.go index e082a3b..2a82937 100644 --- a/common/globals/pools.go +++ b/common/globals/pools.go @@ -2,14 +2,11 @@ package stgglb import ( "gitlink.org.cn/cloudream/common/pkgs/mq" - hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/grpc/hub" coormq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/coordinator" - hubmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/hub" scmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/scanner" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" ) -var HubMQPool hubmq.Pool - var CoordinatorMQPool coormq.Pool var ScannerMQPool scmq.Pool @@ -19,8 +16,6 @@ var ScannerMQPool scmq.Pool // @Description: 初始化MQ连接池 // @param cfg func InitMQPool(cfg mq.Config) { - HubMQPool = hubmq.NewPool(cfg) - CoordinatorMQPool = coormq.NewPool(cfg) ScannerMQPool = scmq.NewPool(cfg) @@ -33,6 +28,6 @@ var HubRPCPool *hubrpc.Pool // // @Description: 初始化HubRPC连接池 // @param cfg -func InitHubRPCPool(cfg *hubrpc.PoolConfig) { +func InitHubRPCPool(cfg hubrpc.PoolConfig) { HubRPCPool = hubrpc.NewPool(cfg) } diff --git a/common/magefiles/main.go b/common/magefiles/main.go index c036f3c..6823c93 100644 --- a/common/magefiles/main.go +++ b/common/magefiles/main.go @@ -3,15 +3,26 @@ package main import ( + "io/fs" "path/filepath" "github.com/magefile/mage/sh" ) func Protos() error { - return proto("pkgs/grpc/hub", "hub.proto") -} + var fileNames []string + + filepath.WalkDir("pkgs/rpc", func(path string, d fs.DirEntry, err error) error { + if d.IsDir() { + return nil + } + if filepath.Ext(path) == ".proto" { + fileNames = append(fileNames, path) + } + return nil + }) -func proto(dir string, fileName string) error { - return sh.Run("protoc", "--go_out="+dir, "--go-grpc_out="+dir, filepath.Join(dir, fileName)) + args := []string{"--go_out=.", "--go_opt=paths=source_relative", "--go-grpc_out=.", "--go-grpc_opt=paths=source_relative"} + args = append(args, fileNames...) + return sh.Run("protoc", args...) } diff --git a/common/pkgs/connectivity/collector.go b/common/pkgs/connectivity/collector.go index 3768330..6b6cf63 100644 --- a/common/pkgs/connectivity/collector.go +++ b/common/pkgs/connectivity/collector.go @@ -1,6 +1,7 @@ package connectivity import ( + "context" "math/rand" "sync" "time" @@ -8,6 +9,7 @@ import ( "gitlink.org.cn/cloudream/common/pkgs/logger" stgglb "gitlink.org.cn/cloudream/jcs-pub/common/globals" coormq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/coordinator" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" ) @@ -184,19 +186,11 @@ func (r *Collector) ping(hub cortypes.Hub) Connectivity { } } - agtCli, err := stgglb.HubRPCPool.Acquire(ip, port) - if err != nil { - log.Warnf("new hub %v:%v rpc client: %w", ip, port, err) - return Connectivity{ - ToHubID: hub.HubID, - Latency: nil, - TestTime: time.Now(), - } - } - defer stgglb.HubRPCPool.Release(agtCli) + agtCli := stgglb.HubRPCPool.Get(ip, port) + defer agtCli.Release() // 第一次ping保证网络连接建立成功 - err = agtCli.Ping() + _, err := agtCli.Ping(context.Background(), &hubrpc.Ping{}) if err != nil { log.Warnf("pre ping: %v", err) return Connectivity{ @@ -210,7 +204,7 @@ func (r *Collector) ping(hub cortypes.Hub) Connectivity { var avgLatency time.Duration for i := 0; i < 3; i++ { start := time.Now() - err = agtCli.Ping() + _, err := agtCli.Ping(context.Background(), &hubrpc.Ping{}) if err != nil { log.Warnf("ping: %v", err) return Connectivity{ diff --git a/common/pkgs/ioswitch2/hub_worker.go b/common/pkgs/ioswitch2/hub_worker.go index 83176f4..7aa4b21 100644 --- a/common/pkgs/ioswitch2/hub_worker.go +++ b/common/pkgs/ioswitch2/hub_worker.go @@ -9,7 +9,7 @@ import ( "gitlink.org.cn/cloudream/common/utils/io2" "gitlink.org.cn/cloudream/common/utils/serder" stgglb "gitlink.org.cn/cloudream/jcs-pub/common/globals" - hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/grpc/hub" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" ) @@ -24,11 +24,7 @@ type HubWorker struct { } func (w *HubWorker) NewClient() (exec.WorkerClient, error) { - cli, err := stgglb.HubRPCPool.Acquire(stgglb.SelectGRPCAddress(w.Hub, w.Address)) - if err != nil { - return nil, err - } - + cli := stgglb.HubRPCPool.Get(stgglb.SelectGRPCAddress(w.Hub, w.Address)) return &HubWorkerClient{hubID: w.Hub.HubID, cli: cli}, nil } @@ -47,38 +43,55 @@ func (w *HubWorker) Equals(worker exec.WorkerInfo) bool { type HubWorkerClient struct { hubID cortypes.HubID - cli *hubrpc.PoolClient + cli *hubrpc.Client } func (c *HubWorkerClient) ExecutePlan(ctx context.Context, plan exec.Plan) error { - return c.cli.ExecuteIOPlan(ctx, plan) + _, err := c.cli.ExecuteIOPlan(ctx, &hubrpc.ExecuteIOPlan{Plan: plan}) + return err } func (c *HubWorkerClient) SendStream(ctx context.Context, planID exec.PlanID, id exec.VarID, stream io.ReadCloser) error { - return c.cli.SendStream(ctx, planID, id, io2.CounterCloser(stream, func(cnt int64, err error) { - if stgglb.Stats.HubTransfer != nil { - stgglb.Stats.HubTransfer.RecordOutput(c.hubID, cnt, err == nil || err == io.EOF) - } - })) + _, err := c.cli.SendIOStream(ctx, &hubrpc.SendIOStream{ + PlanID: planID, + VarID: id, + Stream: io2.CounterCloser(stream, func(cnt int64, err error) { + if stgglb.Stats.HubTransfer != nil { + stgglb.Stats.HubTransfer.RecordOutput(c.hubID, cnt, err == nil || err == io.EOF) + } + }), + }) + return err } func (c *HubWorkerClient) SendVar(ctx context.Context, planID exec.PlanID, id exec.VarID, value exec.VarValue) error { - return c.cli.SendVar(ctx, planID, id, value) + _, err := c.cli.SendIOVar(ctx, &hubrpc.SendIOVar{ + PlanID: planID, VarID: id, Value: value, + }) + return err } func (c *HubWorkerClient) GetStream(ctx context.Context, planID exec.PlanID, streamID exec.VarID, signalID exec.VarID, signal exec.VarValue) (io.ReadCloser, error) { - str, err := c.cli.GetStream(ctx, planID, streamID, signalID, signal) + resp, err := c.cli.GetIOStream(ctx, &hubrpc.GetIOStream{ + PlanID: planID, VarID: streamID, SignalID: signalID, Signal: signal, + }) if err != nil { return nil, err } - return io2.CounterCloser(str, func(cnt int64, err error) { + return io2.CounterCloser(resp.Stream, func(cnt int64, err error) { if stgglb.Stats.HubTransfer != nil { stgglb.Stats.HubTransfer.RecordInput(c.hubID, cnt, err == nil || err == io.EOF) } }), nil } func (c *HubWorkerClient) GetVar(ctx context.Context, planID exec.PlanID, varID exec.VarID, signalID exec.VarID, signal exec.VarValue) (exec.VarValue, error) { - return c.cli.GetVar(ctx, planID, varID, signalID, signal) + resp, err := c.cli.GetIOVar(ctx, &hubrpc.GetIOVar{ + PlanID: planID, VarID: varID, SignalID: signalID, Signal: signal, + }) + if err != nil { + return nil, err + } + return resp.Value, nil } func (c *HubWorkerClient) Close() error { - stgglb.HubRPCPool.Release(c.cli) + c.cli.Release() return nil } diff --git a/common/pkgs/ioswitchlrc/hub_worker.go b/common/pkgs/ioswitchlrc/hub_worker.go index e4cd5b9..28ecddd 100644 --- a/common/pkgs/ioswitchlrc/hub_worker.go +++ b/common/pkgs/ioswitchlrc/hub_worker.go @@ -6,7 +6,7 @@ import ( "gitlink.org.cn/cloudream/common/pkgs/ioswitch/exec" stgglb "gitlink.org.cn/cloudream/jcs-pub/common/globals" - hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/grpc/hub" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" ) @@ -20,11 +20,7 @@ type HubWorker struct { } func (w *HubWorker) NewClient() (exec.WorkerClient, error) { - cli, err := stgglb.HubRPCPool.Acquire(stgglb.SelectGRPCAddress(w.Hub, w.Address)) - if err != nil { - return nil, err - } - + cli := stgglb.HubRPCPool.Get(stgglb.SelectGRPCAddress(w.Hub, w.Address)) return &HubWorkerClient{cli: cli}, nil } @@ -42,25 +38,47 @@ func (w *HubWorker) Equals(worker exec.WorkerInfo) bool { } type HubWorkerClient struct { - cli *hubrpc.PoolClient + cli *hubrpc.Client } func (c *HubWorkerClient) ExecutePlan(ctx context.Context, plan exec.Plan) error { - return c.cli.ExecuteIOPlan(ctx, plan) + _, err := c.cli.ExecuteIOPlan(ctx, &hubrpc.ExecuteIOPlan{Plan: plan}) + return err } func (c *HubWorkerClient) SendStream(ctx context.Context, planID exec.PlanID, id exec.VarID, stream io.ReadCloser) error { - return c.cli.SendStream(ctx, planID, id, stream) + _, err := c.cli.SendIOStream(ctx, &hubrpc.SendIOStream{ + PlanID: planID, + VarID: id, + Stream: stream, + }) + return err } func (c *HubWorkerClient) SendVar(ctx context.Context, planID exec.PlanID, id exec.VarID, value exec.VarValue) error { - return c.cli.SendVar(ctx, planID, id, value) + _, err := c.cli.SendIOVar(ctx, &hubrpc.SendIOVar{ + PlanID: planID, VarID: id, Value: value, + }) + return err } func (c *HubWorkerClient) GetStream(ctx context.Context, planID exec.PlanID, streamID exec.VarID, signalID exec.VarID, signal exec.VarValue) (io.ReadCloser, error) { - return c.cli.GetStream(ctx, planID, streamID, signalID, signal) + resp, err := c.cli.GetIOStream(ctx, &hubrpc.GetIOStream{ + PlanID: planID, VarID: streamID, SignalID: signalID, Signal: signal, + }) + if err != nil { + return nil, err + } + + return resp.Stream, nil } func (c *HubWorkerClient) GetVar(ctx context.Context, planID exec.PlanID, varID exec.VarID, signalID exec.VarID, signal exec.VarValue) (exec.VarValue, error) { - return c.cli.GetVar(ctx, planID, varID, signalID, signal) + resp, err := c.cli.GetIOVar(ctx, &hubrpc.GetIOVar{ + PlanID: planID, VarID: varID, SignalID: signalID, Signal: signal, + }) + if err != nil { + return nil, err + } + return resp.Value, nil } func (c *HubWorkerClient) Close() error { - stgglb.HubRPCPool.Release(c.cli) + c.cli.Release() return nil } diff --git a/common/pkgs/mq/hub/cache.go b/common/pkgs/mq/hub/cache.go deleted file mode 100644 index a5802eb..0000000 --- a/common/pkgs/mq/hub/cache.go +++ /dev/null @@ -1,61 +0,0 @@ -package hub - -import ( - "gitlink.org.cn/cloudream/common/pkgs/mq" - clitypes "gitlink.org.cn/cloudream/jcs-pub/client/types" -) - -type CacheService interface { - CheckCache(msg *CheckCache) (*CheckCacheResp, *mq.CodeMessage) - - CacheGC(msg *CacheGC) (*CacheGCResp, *mq.CodeMessage) -} - -// 检查节点上的IPFS -var _ = Register(Service.CheckCache) - -type CheckCache struct { - mq.MessageBodyBase - UserSpace clitypes.UserSpaceDetail `json:"userSpace"` -} -type CheckCacheResp struct { - mq.MessageBodyBase - FileHashes []clitypes.FileHash `json:"fileHashes"` -} - -func NewCheckCache(space clitypes.UserSpaceDetail) *CheckCache { - return &CheckCache{UserSpace: space} -} -func NewCheckCacheResp(fileHashes []clitypes.FileHash) *CheckCacheResp { - return &CheckCacheResp{ - FileHashes: fileHashes, - } -} -func (client *Client) CheckCache(msg *CheckCache, opts ...mq.RequestOption) (*CheckCacheResp, error) { - return mq.Request(Service.CheckCache, client.rabbitCli, msg, opts...) -} - -// 清理Cache中不用的文件 -var _ = Register(Service.CacheGC) - -type CacheGC struct { - mq.MessageBodyBase - UserSpace clitypes.UserSpaceDetail `json:"userSpace"` - Avaiables []clitypes.FileHash `json:"avaiables"` -} -type CacheGCResp struct { - mq.MessageBodyBase -} - -func ReqCacheGC(space clitypes.UserSpaceDetail, avaiables []clitypes.FileHash) *CacheGC { - return &CacheGC{ - UserSpace: space, - Avaiables: avaiables, - } -} -func RespCacheGC() *CacheGCResp { - return &CacheGCResp{} -} -func (client *Client) CacheGC(msg *CacheGC, opts ...mq.RequestOption) (*CacheGCResp, error) { - return mq.Request(Service.CacheGC, client.rabbitCli, msg, opts...) -} diff --git a/common/pkgs/mq/hub/client.go b/common/pkgs/mq/hub/client.go deleted file mode 100644 index 209f613..0000000 --- a/common/pkgs/mq/hub/client.go +++ /dev/null @@ -1,68 +0,0 @@ -package hub - -import ( - "sync" - - "gitlink.org.cn/cloudream/common/pkgs/mq" - stgmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq" - cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" -) - -type Client struct { - rabbitCli *mq.RabbitMQTransport - id cortypes.HubID -} - -func NewClient(id cortypes.HubID, cfg mq.Config) (*Client, error) { - rabbitCli, err := mq.NewRabbitMQTransport(cfg, stgmq.MakeHubQueueName(int64(id)), "") - if err != nil { - return nil, err - } - - return &Client{ - rabbitCli: rabbitCli, - id: id, - }, nil -} - -func (c *Client) Close() { - c.rabbitCli.Close() -} - -type Pool interface { - Acquire(id cortypes.HubID) (*Client, error) - Release(cli *Client) -} - -type pool struct { - mqcfg mq.Config - shareds map[cortypes.HubID]*Client - lock sync.Mutex -} - -func NewPool(mqcfg mq.Config) Pool { - return &pool{ - mqcfg: mqcfg, - shareds: make(map[cortypes.HubID]*Client), - } -} -func (p *pool) Acquire(id cortypes.HubID) (*Client, error) { - p.lock.Lock() - defer p.lock.Unlock() - - cli, ok := p.shareds[id] - if !ok { - var err error - cli, err = NewClient(id, p.mqcfg) - if err != nil { - return nil, err - } - p.shareds[id] = cli - } - - return cli, nil -} - -func (p *pool) Release(cli *Client) { - // TODO 定时关闭 -} diff --git a/common/pkgs/mq/hub/hub.go b/common/pkgs/mq/hub/hub.go deleted file mode 100644 index 9007335..0000000 --- a/common/pkgs/mq/hub/hub.go +++ /dev/null @@ -1,29 +0,0 @@ -package hub - -import ( - "gitlink.org.cn/cloudream/common/pkgs/mq" -) - -type HubService interface { - GetState(msg *GetState) (*GetStateResp, *mq.CodeMessage) -} - -// 获取hub状态 -var _ = Register(Service.GetState) - -type GetState struct { - mq.MessageBodyBase -} -type GetStateResp struct { - mq.MessageBodyBase -} - -func NewGetState() *GetState { - return &GetState{} -} -func NewGetStateResp() *GetStateResp { - return &GetStateResp{} -} -func (client *Client) GetState(msg *GetState, opts ...mq.RequestOption) (*GetStateResp, error) { - return mq.Request(Service.GetState, client.rabbitCli, msg, opts...) -} diff --git a/common/pkgs/mq/hub/server.go b/common/pkgs/mq/hub/server.go deleted file mode 100644 index 029a3a2..0000000 --- a/common/pkgs/mq/hub/server.go +++ /dev/null @@ -1,75 +0,0 @@ -package hub - -import ( - "gitlink.org.cn/cloudream/common/pkgs/mq" - "gitlink.org.cn/cloudream/common/utils/sync2" - mymq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq" - cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" -) - -type Service interface { - UserSpaceService - - CacheService - - HubService -} - -type Server struct { - service Service - rabbitSvr mq.RabbitMQServer -} - -func NewServer(svc Service, id cortypes.HubID, cfg mq.Config) (*Server, error) { - srv := &Server{ - service: svc, - } - - rabbitSvr, err := mq.NewRabbitMQServer( - cfg, - mymq.MakeHubQueueName(int64(id)), - func(msg *mq.Message) (*mq.Message, error) { - return msgDispatcher.Handle(srv.service, msg) - }, - ) - if err != nil { - return nil, err - } - - srv.rabbitSvr = *rabbitSvr - - return srv, nil -} - -func (s *Server) Stop() { - s.rabbitSvr.Close() -} - -func (s *Server) Start() *sync2.UnboundChannel[mq.RabbitMQServerEvent] { - return s.rabbitSvr.Start() -} - -func (s *Server) OnError(callback func(error)) { - s.rabbitSvr.OnError = callback -} - -var msgDispatcher mq.MessageDispatcher = mq.NewMessageDispatcher() - -// Register 将Service中的一个接口函数作为指定类型消息的处理函数,同时会注册请求和响应的消息类型 -// TODO 需要约束:Service实现了TSvc接口 -func Register[TReq mq.MessageBody, TResp mq.MessageBody](svcFn func(svc Service, msg TReq) (TResp, *mq.CodeMessage)) any { - mq.AddServiceFn(&msgDispatcher, svcFn) - mq.RegisterMessage[TReq]() - mq.RegisterMessage[TResp]() - - return nil -} - -// RegisterNoReply 将Service中的一个*没有返回值的*接口函数作为指定类型消息的处理函数,同时会注册请求和响应的消息类型 -// TODO 需要约束:Service实现了TSvc接口 -func RegisterNoReply[TReq mq.MessageBody](svcFn func(svc Service, msg TReq)) any { - mq.AddNoRespServiceFn(&msgDispatcher, svcFn) - mq.RegisterMessage[TReq]() - - return nil -} diff --git a/common/pkgs/mq/hub/user_space.go b/common/pkgs/mq/hub/user_space.go deleted file mode 100644 index 94c6eb8..0000000 --- a/common/pkgs/mq/hub/user_space.go +++ /dev/null @@ -1,47 +0,0 @@ -package hub - -import ( - "gitlink.org.cn/cloudream/common/pkgs/mq" - clitypes "gitlink.org.cn/cloudream/jcs-pub/client/types" - stgtypes "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/storage/types" -) - -type UserSpaceService interface { - PublicStoreListAll(msg *PublicStoreListAll) (*PublicStoreListAllResp, *mq.CodeMessage) - - PublicStoreMkdirs(msg *PublicStoreMkdirs) (*PublicStoreMkdirsResp, *mq.CodeMessage) -} - -// 启动从UserSpace上传Package的任务 -var _ = Register(Service.PublicStoreListAll) - -type PublicStoreListAll struct { - mq.MessageBodyBase - UserSpace clitypes.UserSpaceDetail - Path string -} -type PublicStoreListAllResp struct { - mq.MessageBodyBase - Entries []stgtypes.PublicStoreEntry -} - -func (client *Client) PublicStoreListAll(msg *PublicStoreListAll, opts ...mq.RequestOption) (*PublicStoreListAllResp, error) { - return mq.Request(Service.PublicStoreListAll, client.rabbitCli, msg, opts...) -} - -var _ = Register(Service.PublicStoreMkdirs) - -type PublicStoreMkdirs struct { - mq.MessageBodyBase - UserSpace clitypes.UserSpaceDetail - Pathes []string -} - -type PublicStoreMkdirsResp struct { - mq.MessageBodyBase - Successes []bool -} - -func (client *Client) PublicStoreMkdirs(msg *PublicStoreMkdirs, opts ...mq.RequestOption) (*PublicStoreMkdirsResp, error) { - return mq.Request(Service.PublicStoreMkdirs, client.rabbitCli, msg, opts...) -} diff --git a/common/pkgs/rpc/chunked.go b/common/pkgs/rpc/chunked.go new file mode 100644 index 0000000..bd8fcf2 --- /dev/null +++ b/common/pkgs/rpc/chunked.go @@ -0,0 +1,186 @@ +package rpc + +import ( + "fmt" + "io" + + "gitlink.org.cn/cloudream/common/utils/io2" +) + +type GRPCChunkedWriter interface { + Send(*ChunkedData) error +} + +type GRPCChunkedReader interface { + Recv() (*ChunkedData, error) +} + +type ChunkedWriter struct { + gw GRPCChunkedWriter +} + +func NewChunkedWriter(stream GRPCChunkedWriter) *ChunkedWriter { + return &ChunkedWriter{gw: stream} +} + +// 开始写入一个新Part。每次只能有一个Part在写入。 +func (w *ChunkedWriter) BeginPart(name string) io.Writer { + err := w.gw.Send(&ChunkedData{Type: ChunkedDataType_NewPart, Data: []byte(name)}) + if err != nil { + return io2.ErrorWriter(fmt.Errorf("write part name: %w", err)) + } + return &PartWriter{cw: w} +} + +func (w *ChunkedWriter) WriteDataPart(name string, data []byte) error { + pw := w.BeginPart(name) + return io2.WriteAll(pw, data) +} + +func (w *ChunkedWriter) WriteStreamPart(name string, stream io.Reader) (int64, error) { + pw := w.BeginPart(name) + n, err := io.Copy(pw, stream) + return n, err +} + +// 发送ErrorPart,不会关闭连接。 +func (w *ChunkedWriter) Abort(msg string) error { + return w.gw.Send(&ChunkedData{Type: ChunkedDataType_Error, Data: []byte(msg)}) +} + +// 发送EOFPart,不会关闭连接。 +func (w *ChunkedWriter) Finish() error { + err := w.gw.Send(&ChunkedData{Type: ChunkedDataType_EOF, Data: []byte{}}) + if err != nil { + return fmt.Errorf("write EOF: %w", err) + } + + return nil +} + +type PartWriter struct { + cw *ChunkedWriter +} + +func (w *PartWriter) Write(data []byte) (int, error) { + err := w.cw.gw.Send(&ChunkedData{Type: ChunkedDataType_Data, Data: data}) + if err != nil { + return 0, fmt.Errorf("write data: %w", err) + } + + return len(data), nil +} + +type ChunkedAbortError struct { + Message string +} + +func (e *ChunkedAbortError) Error() string { + return e.Message +} + +type ChunkedReader struct { + gr GRPCChunkedReader + chunk *ChunkedData + err error +} + +func NewChunkedReader(gr GRPCChunkedReader) *ChunkedReader { + return &ChunkedReader{gr: gr} +} + +// 读取下一个Part。每次只能读取一个Part,且必须将其全部读取完毕才能读取下一个 +func (r *ChunkedReader) NextPart() (string, io.Reader, error) { + if r.err != nil { + return "", nil, r.err + } + + if r.chunk == nil { + var err error + r.chunk, err = r.gr.Recv() + if err != nil { + r.err = fmt.Errorf("receive chunk: %w", err) + return "", nil, r.err + } + } + + switch r.chunk.Type { + case ChunkedDataType_NewPart: + return string(r.chunk.Data), &PartReader{creader: r}, nil + + case ChunkedDataType_Data: + r.err = fmt.Errorf("unexpected data part") + return "", nil, r.err + + case ChunkedDataType_EOF: + r.err = io.EOF + return "", nil, r.err + + case ChunkedDataType_Error: + r.err = &ChunkedAbortError{Message: string(r.chunk.Data)} + return "", nil, r.err + + default: + r.err = fmt.Errorf("unknown part type: %d", r.chunk.Type) + return "", nil, r.err + } +} + +func (r *ChunkedReader) NextDataPart() (string, []byte, error) { + partName, partReader, err := r.NextPart() + if err != nil { + return "", nil, err + } + + data, err := io.ReadAll(partReader) + if err != nil { + return "", nil, err + } + + return partName, data, nil +} + +type PartReader struct { + creader *ChunkedReader + data []byte +} + +func (r *PartReader) Read(p []byte) (int, error) { + if len(r.data) > 0 { + n := copy(p, r.data) + r.data = r.data[n:] + return n, nil + } + + chunk, err := r.creader.gr.Recv() + if err == io.EOF { + r.creader.err = io.ErrUnexpectedEOF + return 0, io.ErrUnexpectedEOF + } + if err != nil { + r.creader.err = fmt.Errorf("receive chunk: %w", err) + return 0, r.creader.err + } + + switch chunk.Type { + case ChunkedDataType_NewPart: + r.creader.chunk = chunk + return 0, io.EOF + + case ChunkedDataType_Data: + r.data = chunk.Data + return r.Read(p) + + case ChunkedDataType_EOF: + r.creader.err = io.EOF + return 0, io.EOF + + case ChunkedDataType_Error: + r.creader.err = &ChunkedAbortError{Message: string(chunk.Data)} + return 0, r.creader.err + + default: + r.creader.err = fmt.Errorf("unknown part type: %d", chunk.Type) + return 0, r.creader.err + } +} diff --git a/common/pkgs/rpc/hub/cache.go b/common/pkgs/rpc/hub/cache.go new file mode 100644 index 0000000..d74281f --- /dev/null +++ b/common/pkgs/rpc/hub/cache.go @@ -0,0 +1,48 @@ +package hubrpc + +import ( + "context" + + clitypes "gitlink.org.cn/cloudream/jcs-pub/client/types" + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" +) + +type CacheSvc interface { + CheckCache(ctx context.Context, req *CheckCache) (*CheckCacheResp, *rpc.CodeError) + CacheGC(ctx context.Context, req *CacheGC) (*CacheGCResp, *rpc.CodeError) +} + +// 获取Cache中文件列表 +type CheckCache struct { + UserSpace clitypes.UserSpaceDetail +} +type CheckCacheResp struct { + FileHashes []clitypes.FileHash +} + +func (c *Client) CheckCache(ctx context.Context, req *CheckCache) (*CheckCacheResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + return rpc.UnaryClient[*CheckCacheResp](c.cli.CheckCache, ctx, req) +} +func (s *Server) CheckCache(ctx context.Context, req *rpc.Request) (*rpc.Response, error) { + return rpc.UnaryServer(s.svrImpl.CheckCache, ctx, req) +} + +// 清理Cache中不用的文件 +type CacheGC struct { + UserSpace clitypes.UserSpaceDetail + Availables []clitypes.FileHash +} +type CacheGCResp struct{} + +func (c *Client) CacheGC(ctx context.Context, req *CacheGC) (*CacheGCResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + return rpc.UnaryClient[*CacheGCResp](c.cli.CacheGC, ctx, req) +} +func (s *Server) CacheGC(ctx context.Context, req *rpc.Request) (*rpc.Response, error) { + return rpc.UnaryServer(s.svrImpl.CacheGC, ctx, req) +} diff --git a/common/pkgs/rpc/hub/client.go b/common/pkgs/rpc/hub/client.go new file mode 100644 index 0000000..441afb2 --- /dev/null +++ b/common/pkgs/rpc/hub/client.go @@ -0,0 +1,23 @@ +package hubrpc + +import ( + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" + "google.golang.org/grpc" +) + +type Client struct { + addr grpcAddr + con *grpc.ClientConn + cli HubClient + pool *Pool + fusedErr *rpc.CodeError +} + +func (c *Client) Release() { + if c.con != nil { + c.pool.release(c.addr) + } +} + +// 客户端的API要和服务端的API保持一致 +var _ HubAPI = (*Client)(nil) diff --git a/common/pkgs/rpc/hub/hub.pb.go b/common/pkgs/rpc/hub/hub.pb.go new file mode 100644 index 0000000..52b2492 --- /dev/null +++ b/common/pkgs/rpc/hub/hub.pb.go @@ -0,0 +1,125 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.34.2 +// protoc v4.22.3 +// source: pkgs/rpc/hub/hub.proto + +package hubrpc + +import ( + rpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +var File_pkgs_rpc_hub_hub_proto protoreflect.FileDescriptor + +var file_pkgs_rpc_hub_hub_proto_rawDesc = []byte{ + 0x0a, 0x16, 0x70, 0x6b, 0x67, 0x73, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x68, 0x75, 0x62, 0x2f, 0x68, + 0x75, 0x62, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x06, 0x68, 0x75, 0x62, 0x72, 0x70, 0x63, + 0x1a, 0x12, 0x70, 0x6b, 0x67, 0x73, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x72, 0x70, 0x63, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x32, 0xf0, 0x03, 0x0a, 0x03, 0x48, 0x75, 0x62, 0x12, 0x2c, 0x0a, 0x0d, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x49, 0x4f, 0x50, 0x6c, 0x61, 0x6e, 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, 0x31, 0x0a, 0x0c, 0x53, 0x65, + 0x6e, 0x64, 0x49, 0x4f, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x10, 0x2e, 0x72, 0x70, 0x63, + 0x2e, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x1a, 0x0d, 0x2e, 0x72, + 0x70, 0x63, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x12, 0x2f, 0x0a, + 0x0b, 0x47, 0x65, 0x74, 0x49, 0x4f, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x0c, 0x2e, 0x72, + 0x70, 0x63, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x72, 0x70, 0x63, + 0x2e, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x30, 0x01, 0x12, 0x28, + 0x0a, 0x09, 0x53, 0x65, 0x6e, 0x64, 0x49, 0x4f, 0x56, 0x61, 0x72, 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, 0x27, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x49, + 0x4f, 0x56, 0x61, 0x72, 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, 0x31, 0x0a, 0x12, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x53, 0x74, 0x6f, 0x72, 0x65, + 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 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, 0x30, 0x0a, 0x11, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x53, 0x74, + 0x6f, 0x72, 0x65, 0x4d, 0x6b, 0x64, 0x69, 0x72, 0x73, 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, 0x29, 0x0a, 0x0a, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x43, + 0x61, 0x63, 0x68, 0x65, 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, 0x26, 0x0a, 0x07, 0x43, 0x61, 0x63, 0x68, 0x65, 0x47, 0x43, 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, 0x23, 0x0a, 0x04, 0x50, 0x69, 0x6e, + 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, 0x27, + 0x0a, 0x08, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 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, 0x42, 0x40, 0x5a, 0x3e, 0x67, 0x69, 0x74, 0x6c, 0x69, + 0x6e, 0x6b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x63, 0x6e, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x72, + 0x65, 0x61, 0x6d, 0x2f, 0x6a, 0x63, 0x73, 0x2d, 0x70, 0x75, 0x62, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2f, 0x70, 0x6b, 0x67, 0x73, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x68, 0x75, 0x62, 0x72, + 0x70, 0x63, 0x3b, 0x68, 0x75, 0x62, 0x72, 0x70, 0x63, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, +} + +var file_pkgs_rpc_hub_hub_proto_goTypes = []any{ + (*rpc.Request)(nil), // 0: rpc.Request + (*rpc.ChunkedData)(nil), // 1: rpc.ChunkedData + (*rpc.Response)(nil), // 2: rpc.Response +} +var file_pkgs_rpc_hub_hub_proto_depIdxs = []int32{ + 0, // 0: hubrpc.Hub.ExecuteIOPlan:input_type -> rpc.Request + 1, // 1: hubrpc.Hub.SendIOStream:input_type -> rpc.ChunkedData + 0, // 2: hubrpc.Hub.GetIOStream:input_type -> rpc.Request + 0, // 3: hubrpc.Hub.SendIOVar:input_type -> rpc.Request + 0, // 4: hubrpc.Hub.GetIOVar:input_type -> rpc.Request + 0, // 5: hubrpc.Hub.PublicStoreListAll:input_type -> rpc.Request + 0, // 6: hubrpc.Hub.PublicStoreMkdirs:input_type -> rpc.Request + 0, // 7: hubrpc.Hub.CheckCache:input_type -> rpc.Request + 0, // 8: hubrpc.Hub.CacheGC:input_type -> rpc.Request + 0, // 9: hubrpc.Hub.Ping:input_type -> rpc.Request + 0, // 10: hubrpc.Hub.GetState:input_type -> rpc.Request + 2, // 11: hubrpc.Hub.ExecuteIOPlan:output_type -> rpc.Response + 2, // 12: hubrpc.Hub.SendIOStream:output_type -> rpc.Response + 1, // 13: hubrpc.Hub.GetIOStream:output_type -> rpc.ChunkedData + 2, // 14: hubrpc.Hub.SendIOVar:output_type -> rpc.Response + 2, // 15: hubrpc.Hub.GetIOVar:output_type -> rpc.Response + 2, // 16: hubrpc.Hub.PublicStoreListAll:output_type -> rpc.Response + 2, // 17: hubrpc.Hub.PublicStoreMkdirs:output_type -> rpc.Response + 2, // 18: hubrpc.Hub.CheckCache:output_type -> rpc.Response + 2, // 19: hubrpc.Hub.CacheGC:output_type -> rpc.Response + 2, // 20: hubrpc.Hub.Ping:output_type -> rpc.Response + 2, // 21: hubrpc.Hub.GetState:output_type -> rpc.Response + 11, // [11:22] is the sub-list for method output_type + 0, // [0:11] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_pkgs_rpc_hub_hub_proto_init() } +func file_pkgs_rpc_hub_hub_proto_init() { + if File_pkgs_rpc_hub_hub_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_pkgs_rpc_hub_hub_proto_rawDesc, + NumEnums: 0, + NumMessages: 0, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_pkgs_rpc_hub_hub_proto_goTypes, + DependencyIndexes: file_pkgs_rpc_hub_hub_proto_depIdxs, + }.Build() + File_pkgs_rpc_hub_hub_proto = out.File + file_pkgs_rpc_hub_hub_proto_rawDesc = nil + file_pkgs_rpc_hub_hub_proto_goTypes = nil + file_pkgs_rpc_hub_hub_proto_depIdxs = nil +} diff --git a/common/pkgs/rpc/hub/hub.proto b/common/pkgs/rpc/hub/hub.proto new file mode 100644 index 0000000..754a241 --- /dev/null +++ b/common/pkgs/rpc/hub/hub.proto @@ -0,0 +1,25 @@ +syntax = "proto3"; + +import "pkgs/rpc/rpc.proto"; + +package hubrpc; + +option go_package = "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hubrpc;hubrpc"; + + +service Hub { + rpc ExecuteIOPlan(rpc.Request) returns(rpc.Response); + rpc SendIOStream(stream rpc.ChunkedData)returns(rpc.Response); + rpc GetIOStream(rpc.Request)returns(stream rpc.ChunkedData); + rpc SendIOVar(rpc.Request)returns(rpc.Response); + rpc GetIOVar(rpc.Request)returns(rpc.Response); + + rpc PublicStoreListAll(rpc.Request) returns(rpc.Response); + rpc PublicStoreMkdirs(rpc.Request) returns(rpc.Response); + + rpc CheckCache(rpc.Request) returns(rpc.Response); + rpc CacheGC(rpc.Request) returns(rpc.Response); + + rpc Ping(rpc.Request) returns(rpc.Response); + rpc GetState(rpc.Request) returns(rpc.Response); +} \ No newline at end of file diff --git a/common/pkgs/rpc/hub/hub_grpc.pb.go b/common/pkgs/rpc/hub/hub_grpc.pb.go new file mode 100644 index 0000000..f7a2b95 --- /dev/null +++ b/common/pkgs/rpc/hub/hub_grpc.pb.go @@ -0,0 +1,542 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.3.0 +// - protoc v4.22.3 +// source: pkgs/rpc/hub/hub.proto + +package hubrpc + +import ( + context "context" + rpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +const ( + Hub_ExecuteIOPlan_FullMethodName = "/hubrpc.Hub/ExecuteIOPlan" + Hub_SendIOStream_FullMethodName = "/hubrpc.Hub/SendIOStream" + Hub_GetIOStream_FullMethodName = "/hubrpc.Hub/GetIOStream" + Hub_SendIOVar_FullMethodName = "/hubrpc.Hub/SendIOVar" + Hub_GetIOVar_FullMethodName = "/hubrpc.Hub/GetIOVar" + Hub_PublicStoreListAll_FullMethodName = "/hubrpc.Hub/PublicStoreListAll" + Hub_PublicStoreMkdirs_FullMethodName = "/hubrpc.Hub/PublicStoreMkdirs" + Hub_CheckCache_FullMethodName = "/hubrpc.Hub/CheckCache" + Hub_CacheGC_FullMethodName = "/hubrpc.Hub/CacheGC" + Hub_Ping_FullMethodName = "/hubrpc.Hub/Ping" + Hub_GetState_FullMethodName = "/hubrpc.Hub/GetState" +) + +// HubClient is the client API for Hub service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type HubClient interface { + ExecuteIOPlan(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) + SendIOStream(ctx context.Context, opts ...grpc.CallOption) (Hub_SendIOStreamClient, error) + GetIOStream(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (Hub_GetIOStreamClient, error) + SendIOVar(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) + GetIOVar(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) + PublicStoreListAll(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) + PublicStoreMkdirs(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) + CheckCache(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) + CacheGC(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) + Ping(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) + GetState(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) +} + +type hubClient struct { + cc grpc.ClientConnInterface +} + +func NewHubClient(cc grpc.ClientConnInterface) HubClient { + return &hubClient{cc} +} + +func (c *hubClient) ExecuteIOPlan(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) { + out := new(rpc.Response) + err := c.cc.Invoke(ctx, Hub_ExecuteIOPlan_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *hubClient) SendIOStream(ctx context.Context, opts ...grpc.CallOption) (Hub_SendIOStreamClient, error) { + stream, err := c.cc.NewStream(ctx, &Hub_ServiceDesc.Streams[0], Hub_SendIOStream_FullMethodName, opts...) + if err != nil { + return nil, err + } + x := &hubSendIOStreamClient{stream} + return x, nil +} + +type Hub_SendIOStreamClient interface { + Send(*rpc.ChunkedData) error + CloseAndRecv() (*rpc.Response, error) + grpc.ClientStream +} + +type hubSendIOStreamClient struct { + grpc.ClientStream +} + +func (x *hubSendIOStreamClient) Send(m *rpc.ChunkedData) error { + return x.ClientStream.SendMsg(m) +} + +func (x *hubSendIOStreamClient) CloseAndRecv() (*rpc.Response, error) { + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + m := new(rpc.Response) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *hubClient) GetIOStream(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (Hub_GetIOStreamClient, error) { + stream, err := c.cc.NewStream(ctx, &Hub_ServiceDesc.Streams[1], Hub_GetIOStream_FullMethodName, opts...) + if err != nil { + return nil, err + } + x := &hubGetIOStreamClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Hub_GetIOStreamClient interface { + Recv() (*rpc.ChunkedData, error) + grpc.ClientStream +} + +type hubGetIOStreamClient struct { + grpc.ClientStream +} + +func (x *hubGetIOStreamClient) Recv() (*rpc.ChunkedData, error) { + m := new(rpc.ChunkedData) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *hubClient) SendIOVar(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) { + out := new(rpc.Response) + err := c.cc.Invoke(ctx, Hub_SendIOVar_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *hubClient) GetIOVar(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) { + out := new(rpc.Response) + err := c.cc.Invoke(ctx, Hub_GetIOVar_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *hubClient) PublicStoreListAll(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) { + out := new(rpc.Response) + err := c.cc.Invoke(ctx, Hub_PublicStoreListAll_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *hubClient) PublicStoreMkdirs(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) { + out := new(rpc.Response) + err := c.cc.Invoke(ctx, Hub_PublicStoreMkdirs_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *hubClient) CheckCache(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) { + out := new(rpc.Response) + err := c.cc.Invoke(ctx, Hub_CheckCache_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *hubClient) CacheGC(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) { + out := new(rpc.Response) + err := c.cc.Invoke(ctx, Hub_CacheGC_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *hubClient) Ping(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) { + out := new(rpc.Response) + err := c.cc.Invoke(ctx, Hub_Ping_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *hubClient) GetState(ctx context.Context, in *rpc.Request, opts ...grpc.CallOption) (*rpc.Response, error) { + out := new(rpc.Response) + err := c.cc.Invoke(ctx, Hub_GetState_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// HubServer is the server API for Hub service. +// All implementations must embed UnimplementedHubServer +// for forward compatibility +type HubServer interface { + ExecuteIOPlan(context.Context, *rpc.Request) (*rpc.Response, error) + SendIOStream(Hub_SendIOStreamServer) error + GetIOStream(*rpc.Request, Hub_GetIOStreamServer) error + SendIOVar(context.Context, *rpc.Request) (*rpc.Response, error) + GetIOVar(context.Context, *rpc.Request) (*rpc.Response, error) + PublicStoreListAll(context.Context, *rpc.Request) (*rpc.Response, error) + PublicStoreMkdirs(context.Context, *rpc.Request) (*rpc.Response, error) + CheckCache(context.Context, *rpc.Request) (*rpc.Response, error) + CacheGC(context.Context, *rpc.Request) (*rpc.Response, error) + Ping(context.Context, *rpc.Request) (*rpc.Response, error) + GetState(context.Context, *rpc.Request) (*rpc.Response, error) + mustEmbedUnimplementedHubServer() +} + +// UnimplementedHubServer must be embedded to have forward compatible implementations. +type UnimplementedHubServer struct { +} + +func (UnimplementedHubServer) ExecuteIOPlan(context.Context, *rpc.Request) (*rpc.Response, error) { + return nil, status.Errorf(codes.Unimplemented, "method ExecuteIOPlan not implemented") +} +func (UnimplementedHubServer) SendIOStream(Hub_SendIOStreamServer) error { + return status.Errorf(codes.Unimplemented, "method SendIOStream not implemented") +} +func (UnimplementedHubServer) GetIOStream(*rpc.Request, Hub_GetIOStreamServer) error { + return status.Errorf(codes.Unimplemented, "method GetIOStream not implemented") +} +func (UnimplementedHubServer) SendIOVar(context.Context, *rpc.Request) (*rpc.Response, error) { + return nil, status.Errorf(codes.Unimplemented, "method SendIOVar not implemented") +} +func (UnimplementedHubServer) GetIOVar(context.Context, *rpc.Request) (*rpc.Response, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetIOVar not implemented") +} +func (UnimplementedHubServer) PublicStoreListAll(context.Context, *rpc.Request) (*rpc.Response, error) { + return nil, status.Errorf(codes.Unimplemented, "method PublicStoreListAll not implemented") +} +func (UnimplementedHubServer) PublicStoreMkdirs(context.Context, *rpc.Request) (*rpc.Response, error) { + return nil, status.Errorf(codes.Unimplemented, "method PublicStoreMkdirs not implemented") +} +func (UnimplementedHubServer) CheckCache(context.Context, *rpc.Request) (*rpc.Response, error) { + return nil, status.Errorf(codes.Unimplemented, "method CheckCache not implemented") +} +func (UnimplementedHubServer) CacheGC(context.Context, *rpc.Request) (*rpc.Response, error) { + return nil, status.Errorf(codes.Unimplemented, "method CacheGC not implemented") +} +func (UnimplementedHubServer) Ping(context.Context, *rpc.Request) (*rpc.Response, error) { + return nil, status.Errorf(codes.Unimplemented, "method Ping not implemented") +} +func (UnimplementedHubServer) GetState(context.Context, *rpc.Request) (*rpc.Response, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetState not implemented") +} +func (UnimplementedHubServer) mustEmbedUnimplementedHubServer() {} + +// UnsafeHubServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to HubServer will +// result in compilation errors. +type UnsafeHubServer interface { + mustEmbedUnimplementedHubServer() +} + +func RegisterHubServer(s grpc.ServiceRegistrar, srv HubServer) { + s.RegisterService(&Hub_ServiceDesc, srv) +} + +func _Hub_ExecuteIOPlan_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.(HubServer).ExecuteIOPlan(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Hub_ExecuteIOPlan_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(HubServer).ExecuteIOPlan(ctx, req.(*rpc.Request)) + } + return interceptor(ctx, in, info, handler) +} + +func _Hub_SendIOStream_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(HubServer).SendIOStream(&hubSendIOStreamServer{stream}) +} + +type Hub_SendIOStreamServer interface { + SendAndClose(*rpc.Response) error + Recv() (*rpc.ChunkedData, error) + grpc.ServerStream +} + +type hubSendIOStreamServer struct { + grpc.ServerStream +} + +func (x *hubSendIOStreamServer) SendAndClose(m *rpc.Response) error { + return x.ServerStream.SendMsg(m) +} + +func (x *hubSendIOStreamServer) Recv() (*rpc.ChunkedData, error) { + m := new(rpc.ChunkedData) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _Hub_GetIOStream_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(rpc.Request) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(HubServer).GetIOStream(m, &hubGetIOStreamServer{stream}) +} + +type Hub_GetIOStreamServer interface { + Send(*rpc.ChunkedData) error + grpc.ServerStream +} + +type hubGetIOStreamServer struct { + grpc.ServerStream +} + +func (x *hubGetIOStreamServer) Send(m *rpc.ChunkedData) error { + return x.ServerStream.SendMsg(m) +} + +func _Hub_SendIOVar_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.(HubServer).SendIOVar(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Hub_SendIOVar_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(HubServer).SendIOVar(ctx, req.(*rpc.Request)) + } + return interceptor(ctx, in, info, handler) +} + +func _Hub_GetIOVar_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.(HubServer).GetIOVar(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Hub_GetIOVar_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(HubServer).GetIOVar(ctx, req.(*rpc.Request)) + } + return interceptor(ctx, in, info, handler) +} + +func _Hub_PublicStoreListAll_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.(HubServer).PublicStoreListAll(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Hub_PublicStoreListAll_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(HubServer).PublicStoreListAll(ctx, req.(*rpc.Request)) + } + return interceptor(ctx, in, info, handler) +} + +func _Hub_PublicStoreMkdirs_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.(HubServer).PublicStoreMkdirs(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Hub_PublicStoreMkdirs_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(HubServer).PublicStoreMkdirs(ctx, req.(*rpc.Request)) + } + return interceptor(ctx, in, info, handler) +} + +func _Hub_CheckCache_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.(HubServer).CheckCache(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Hub_CheckCache_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(HubServer).CheckCache(ctx, req.(*rpc.Request)) + } + return interceptor(ctx, in, info, handler) +} + +func _Hub_CacheGC_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.(HubServer).CacheGC(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Hub_CacheGC_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(HubServer).CacheGC(ctx, req.(*rpc.Request)) + } + return interceptor(ctx, in, info, handler) +} + +func _Hub_Ping_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.(HubServer).Ping(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Hub_Ping_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(HubServer).Ping(ctx, req.(*rpc.Request)) + } + return interceptor(ctx, in, info, handler) +} + +func _Hub_GetState_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.(HubServer).GetState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Hub_GetState_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(HubServer).GetState(ctx, req.(*rpc.Request)) + } + return interceptor(ctx, in, info, handler) +} + +// Hub_ServiceDesc is the grpc.ServiceDesc for Hub service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var Hub_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "hubrpc.Hub", + HandlerType: (*HubServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "ExecuteIOPlan", + Handler: _Hub_ExecuteIOPlan_Handler, + }, + { + MethodName: "SendIOVar", + Handler: _Hub_SendIOVar_Handler, + }, + { + MethodName: "GetIOVar", + Handler: _Hub_GetIOVar_Handler, + }, + { + MethodName: "PublicStoreListAll", + Handler: _Hub_PublicStoreListAll_Handler, + }, + { + MethodName: "PublicStoreMkdirs", + Handler: _Hub_PublicStoreMkdirs_Handler, + }, + { + MethodName: "CheckCache", + Handler: _Hub_CheckCache_Handler, + }, + { + MethodName: "CacheGC", + Handler: _Hub_CacheGC_Handler, + }, + { + MethodName: "Ping", + Handler: _Hub_Ping_Handler, + }, + { + MethodName: "GetState", + Handler: _Hub_GetState_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "SendIOStream", + Handler: _Hub_SendIOStream_Handler, + ClientStreams: true, + }, + { + StreamName: "GetIOStream", + Handler: _Hub_GetIOStream_Handler, + ServerStreams: true, + }, + }, + Metadata: "pkgs/rpc/hub/hub.proto", +} diff --git a/common/pkgs/rpc/hub/ioswitch.go b/common/pkgs/rpc/hub/ioswitch.go new file mode 100644 index 0000000..ccd1185 --- /dev/null +++ b/common/pkgs/rpc/hub/ioswitch.go @@ -0,0 +1,131 @@ +package hubrpc + +import ( + context "context" + "io" + + "gitlink.org.cn/cloudream/common/pkgs/ioswitch/exec" + rpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" +) + +type IOSwitchSvc interface { + ExecuteIOPlan(ctx context.Context, req *ExecuteIOPlan) (*ExecuteIOPlanResp, *rpc.CodeError) + SendIOStream(ctx context.Context, req *SendIOStream) (*SendIOStreamResp, *rpc.CodeError) + GetIOStream(ctx context.Context, req *GetIOStream) (*GetIOStreamResp, *rpc.CodeError) + SendIOVar(ctx context.Context, req *SendIOVar) (*SendIOVarResp, *rpc.CodeError) + GetIOVar(ctx context.Context, req *GetIOVar) (*GetIOVarResp, *rpc.CodeError) +} + +// 执行IO计划 +type ExecuteIOPlan struct { + Plan exec.Plan +} +type ExecuteIOPlanResp struct{} + +func (c *Client) ExecuteIOPlan(ctx context.Context, req *ExecuteIOPlan) (*ExecuteIOPlanResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + return rpc.UnaryClient[*ExecuteIOPlanResp](c.cli.ExecuteIOPlan, ctx, req) +} + +func (s *Server) ExecuteIOPlan(ctx context.Context, req *rpc.Request) (*rpc.Response, error) { + return rpc.UnaryServer(s.svrImpl.ExecuteIOPlan, ctx, req) +} + +// 发送IO流 +type SendIOStream struct { + PlanID exec.PlanID + VarID exec.VarID + Stream io.Reader `json:"-"` +} + +func (s *SendIOStream) GetStream() io.Reader { + return s.Stream +} +func (s *SendIOStream) SetStream(str io.Reader) { + s.Stream = str +} + +type SendIOStreamResp struct{} + +func (c *Client) SendIOStream(ctx context.Context, req *SendIOStream) (*SendIOStreamResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + return rpc.UploadStreamClient[*SendIOStreamResp](c.cli.SendIOStream, ctx, req) +} +func (s *Server) SendIOStream(req Hub_SendIOStreamServer) error { + return rpc.UploadStreamServer(s.svrImpl.SendIOStream, req) +} + +// 获取IO流 +type GetIOStream struct { + PlanID exec.PlanID + VarID exec.VarID + SignalID exec.VarID + Signal exec.VarValue +} + +type GetIOStreamResp struct { + Stream io.ReadCloser `json:"-"` +} + +func (r *GetIOStreamResp) GetStream() io.ReadCloser { + return r.Stream +} +func (r *GetIOStreamResp) SetStream(str io.ReadCloser) { + r.Stream = str +} + +func (c *Client) GetIOStream(ctx context.Context, req *GetIOStream) (*GetIOStreamResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + + return rpc.DownloadStreamClient[*GetIOStreamResp](c.cli.GetIOStream, ctx, req) +} +func (s *Server) GetIOStream(req *rpc.Request, ret Hub_GetIOStreamServer) error { + return rpc.DownloadStreamServer(s.svrImpl.GetIOStream, req, ret) +} + +// 发送IO变量 +type SendIOVar struct { + PlanID exec.PlanID + VarID exec.VarID + Value exec.VarValue +} +type SendIOVarResp struct{} + +func (c *Client) SendIOVar(ctx context.Context, req *SendIOVar) (*SendIOVarResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + return rpc.UnaryClient[*SendIOVarResp](c.cli.SendIOVar, ctx, req) +} + +func (s *Server) SendIOVar(ctx context.Context, req *rpc.Request) (*rpc.Response, error) { + return rpc.UnaryServer(s.svrImpl.SendIOVar, ctx, req) +} + +// 获取IO变量 +type GetIOVar struct { + PlanID exec.PlanID + VarID exec.VarID + SignalID exec.VarID + Signal exec.VarValue +} +type GetIOVarResp struct { + Value exec.VarValue +} + +func (c *Client) GetIOVar(ctx context.Context, req *GetIOVar) (*GetIOVarResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + return rpc.UnaryClient[*GetIOVarResp](c.cli.GetIOVar, ctx, req) +} + +func (s *Server) GetIOVar(ctx context.Context, req *rpc.Request) (*rpc.Response, error) { + return rpc.UnaryServer(s.svrImpl.GetIOVar, ctx, req) +} diff --git a/common/pkgs/rpc/hub/mics.go b/common/pkgs/rpc/hub/mics.go new file mode 100644 index 0000000..e25ad0b --- /dev/null +++ b/common/pkgs/rpc/hub/mics.go @@ -0,0 +1,40 @@ +package hubrpc + +import ( + context "context" + + rpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" +) + +type MicsSvc interface { + Ping(ctx context.Context, req *Ping) (*PingResp, *rpc.CodeError) + GetState(ctx context.Context, req *GetState) (*GetStateResp, *rpc.CodeError) +} + +// 测试延迟 +type Ping struct{} +type PingResp struct{} + +func (c *Client) Ping(ctx context.Context, req *Ping) (*PingResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + return rpc.UnaryClient[*PingResp](c.cli.Ping, ctx, req) +} +func (s *Server) Ping(ctx context.Context, req *rpc.Request) (*rpc.Response, error) { + return rpc.UnaryServer(s.svrImpl.Ping, ctx, req) +} + +// 获取状态 +type GetState struct{} +type GetStateResp struct{} + +func (c *Client) GetState(ctx context.Context, req *GetState) (*GetStateResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + return rpc.UnaryClient[*GetStateResp](c.cli.GetState, ctx, req) +} +func (s *Server) GetState(ctx context.Context, req *rpc.Request) (*rpc.Response, error) { + return rpc.UnaryServer(s.svrImpl.GetState, ctx, req) +} diff --git a/common/pkgs/rpc/hub/pool.go b/common/pkgs/rpc/hub/pool.go new file mode 100644 index 0000000..d6c1c9b --- /dev/null +++ b/common/pkgs/rpc/hub/pool.go @@ -0,0 +1,105 @@ +package hubrpc + +import ( + "fmt" + "sync" + "time" + + "gitlink.org.cn/cloudream/common/consts/errorcode" + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" + grpc "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" +) + +type PoolConfig struct{} + +type Pool struct { + grpcCons map[grpcAddr]*grpcCon + lock sync.Mutex +} + +type grpcAddr struct { + IP string + Port int +} + +type grpcCon struct { + grpcCon *grpc.ClientConn + refCount int + stopClosing chan any +} + +func NewPool(cfg PoolConfig) *Pool { + return &Pool{ + grpcCons: make(map[grpcAddr]*grpcCon), + } +} + +func (p *Pool) Get(ip string, port int) *Client { + p.lock.Lock() + defer p.lock.Unlock() + + ga := grpcAddr{IP: ip, Port: port} + con := p.grpcCons[ga] + if con == nil { + gcon, err := grpc.NewClient(fmt.Sprintf("%v:%v", ip, port), grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + return &Client{ + addr: ga, + con: nil, + pool: p, + fusedErr: rpc.Failed(errorcode.OperationFailed, err.Error()), + } + } + + con = &grpcCon{ + grpcCon: gcon, + refCount: 0, + stopClosing: nil, + } + + p.grpcCons[ga] = con + } + + con.refCount++ + + return &Client{ + addr: ga, + con: con.grpcCon, + pool: p, + } +} + +func (p *Pool) release(addr grpcAddr) { + p.lock.Lock() + defer p.lock.Unlock() + + grpcCon := p.grpcCons[addr] + if grpcCon == nil { + return + } + + grpcCon.refCount-- + grpcCon.refCount = max(grpcCon.refCount, 0) + + if grpcCon.refCount == 0 { + stopClosing := make(chan any) + grpcCon.stopClosing = stopClosing + + go func() { + select { + case <-stopClosing: + return + + case <-time.After(time.Minute): + p.lock.Lock() + defer p.lock.Unlock() + + if grpcCon.refCount == 0 { + grpcCon.grpcCon.Close() + delete(p.grpcCons, addr) + } + } + }() + } +} diff --git a/common/pkgs/rpc/hub/server.go b/common/pkgs/rpc/hub/server.go new file mode 100644 index 0000000..d7f5c25 --- /dev/null +++ b/common/pkgs/rpc/hub/server.go @@ -0,0 +1,26 @@ +package hubrpc + +import ( + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" +) + +type HubAPI interface { + CacheSvc + IOSwitchSvc + MicsSvc + UserSpaceSvc +} + +type Server struct { + UnimplementedHubServer + *rpc.ServerBase + svrImpl HubAPI +} + +func NewServer(cfg rpc.Config, impl HubAPI) *Server { + return &Server{ + ServerBase: rpc.NewServerBase(cfg, impl, &Hub_ServiceDesc), + } +} + +var _ HubServer = (*Server)(nil) diff --git a/common/pkgs/rpc/hub/user_space.go b/common/pkgs/rpc/hub/user_space.go new file mode 100644 index 0000000..6faa940 --- /dev/null +++ b/common/pkgs/rpc/hub/user_space.go @@ -0,0 +1,53 @@ +package hubrpc + +import ( + "context" + + clitypes "gitlink.org.cn/cloudream/jcs-pub/client/types" + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" + stgtypes "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/storage/types" +) + +type UserSpaceSvc interface { + PublicStoreListAll(ctx context.Context, req *PublicStoreListAll) (*PublicStoreListAllResp, *rpc.CodeError) + PublicStoreMkdirs(ctx context.Context, req *PublicStoreMkdirs) (*PublicStoreMkdirsResp, *rpc.CodeError) +} + +// 列出指定PublicStore的指定位置内的所有文件 +type PublicStoreListAll struct { + UserSpace clitypes.UserSpaceDetail + Path string +} +type PublicStoreListAllResp struct { + Entries []stgtypes.PublicStoreEntry +} + +func (c *Client) PublicStoreListAll(ctx context.Context, req *PublicStoreListAll) (*PublicStoreListAllResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + return rpc.UnaryClient[*PublicStoreListAllResp](c.cli.PublicStoreListAll, ctx, req) +} +func (s *Server) PublicStoreListAll(ctx context.Context, req *rpc.Request) (*rpc.Response, error) { + return rpc.UnaryServer(s.svrImpl.PublicStoreListAll, ctx, req) +} + +// 批量在指定PublicStore中创建文件夹 +type PublicStoreMkdirs struct { + UserSpace clitypes.UserSpaceDetail + Pathes []string +} + +type PublicStoreMkdirsResp struct { + Successes []bool +} + +func (c *Client) PublicStoreMkdirs(ctx context.Context, req *PublicStoreMkdirs) (*PublicStoreMkdirsResp, *rpc.CodeError) { + if c.fusedErr != nil { + return nil, c.fusedErr + } + return rpc.UnaryClient[*PublicStoreMkdirsResp](c.cli.PublicStoreMkdirs, ctx, req) +} +func (s *Server) PublicStoreMkdirs(ctx context.Context, req *rpc.Request) (*rpc.Response, error) { + return rpc.UnaryServer(s.svrImpl.PublicStoreMkdirs, ctx, req) +} diff --git a/common/pkgs/rpc/rpc.pb.go b/common/pkgs/rpc/rpc.pb.go new file mode 100644 index 0000000..14eb1de --- /dev/null +++ b/common/pkgs/rpc/rpc.pb.go @@ -0,0 +1,411 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.34.2 +// protoc v4.22.3 +// source: pkgs/rpc/rpc.proto + +package rpc + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type ChunkedDataType int32 + +const ( + ChunkedDataType_Error ChunkedDataType = 0 + ChunkedDataType_EOF ChunkedDataType = 1 + ChunkedDataType_NewPart ChunkedDataType = 2 + ChunkedDataType_Data ChunkedDataType = 3 +) + +// Enum value maps for ChunkedDataType. +var ( + ChunkedDataType_name = map[int32]string{ + 0: "Error", + 1: "EOF", + 2: "NewPart", + 3: "Data", + } + ChunkedDataType_value = map[string]int32{ + "Error": 0, + "EOF": 1, + "NewPart": 2, + "Data": 3, + } +) + +func (x ChunkedDataType) Enum() *ChunkedDataType { + p := new(ChunkedDataType) + *p = x + return p +} + +func (x ChunkedDataType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ChunkedDataType) Descriptor() protoreflect.EnumDescriptor { + return file_pkgs_rpc_rpc_proto_enumTypes[0].Descriptor() +} + +func (ChunkedDataType) Type() protoreflect.EnumType { + return &file_pkgs_rpc_rpc_proto_enumTypes[0] +} + +func (x ChunkedDataType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ChunkedDataType.Descriptor instead. +func (ChunkedDataType) EnumDescriptor() ([]byte, []int) { + return file_pkgs_rpc_rpc_proto_rawDescGZIP(), []int{0} +} + +type Request struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Payload []byte `protobuf:"bytes,1,opt,name=Payload,proto3" json:"Payload,omitempty"` +} + +func (x *Request) Reset() { + *x = Request{} + if protoimpl.UnsafeEnabled { + mi := &file_pkgs_rpc_rpc_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Request) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Request) ProtoMessage() {} + +func (x *Request) ProtoReflect() protoreflect.Message { + mi := &file_pkgs_rpc_rpc_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Request.ProtoReflect.Descriptor instead. +func (*Request) Descriptor() ([]byte, []int) { + return file_pkgs_rpc_rpc_proto_rawDescGZIP(), []int{0} +} + +func (x *Request) GetPayload() []byte { + if x != nil { + return x.Payload + } + return nil +} + +type Response struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Payload []byte `protobuf:"bytes,1,opt,name=Payload,proto3" json:"Payload,omitempty"` +} + +func (x *Response) Reset() { + *x = Response{} + if protoimpl.UnsafeEnabled { + mi := &file_pkgs_rpc_rpc_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Response) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Response) ProtoMessage() {} + +func (x *Response) ProtoReflect() protoreflect.Message { + mi := &file_pkgs_rpc_rpc_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Response.ProtoReflect.Descriptor instead. +func (*Response) Descriptor() ([]byte, []int) { + return file_pkgs_rpc_rpc_proto_rawDescGZIP(), []int{1} +} + +func (x *Response) GetPayload() []byte { + if x != nil { + return x.Payload + } + return nil +} + +type ChunkedData struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Type ChunkedDataType `protobuf:"varint,1,opt,name=Type,proto3,enum=rpc.ChunkedDataType" json:"Type,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=Data,proto3" json:"Data,omitempty"` +} + +func (x *ChunkedData) Reset() { + *x = ChunkedData{} + if protoimpl.UnsafeEnabled { + mi := &file_pkgs_rpc_rpc_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChunkedData) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChunkedData) ProtoMessage() {} + +func (x *ChunkedData) ProtoReflect() protoreflect.Message { + mi := &file_pkgs_rpc_rpc_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ChunkedData.ProtoReflect.Descriptor instead. +func (*ChunkedData) Descriptor() ([]byte, []int) { + return file_pkgs_rpc_rpc_proto_rawDescGZIP(), []int{2} +} + +func (x *ChunkedData) GetType() ChunkedDataType { + if x != nil { + return x.Type + } + return ChunkedDataType_Error +} + +func (x *ChunkedData) GetData() []byte { + if x != nil { + return x.Data + } + return nil +} + +type CodeError struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Code string `protobuf:"bytes,1,opt,name=Code,proto3" json:"Code,omitempty"` + Message string `protobuf:"bytes,2,opt,name=Message,proto3" json:"Message,omitempty"` +} + +func (x *CodeError) Reset() { + *x = CodeError{} + if protoimpl.UnsafeEnabled { + mi := &file_pkgs_rpc_rpc_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CodeError) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CodeError) ProtoMessage() {} + +func (x *CodeError) ProtoReflect() protoreflect.Message { + mi := &file_pkgs_rpc_rpc_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CodeError.ProtoReflect.Descriptor instead. +func (*CodeError) Descriptor() ([]byte, []int) { + return file_pkgs_rpc_rpc_proto_rawDescGZIP(), []int{3} +} + +func (x *CodeError) GetCode() string { + if x != nil { + return x.Code + } + return "" +} + +func (x *CodeError) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +var File_pkgs_rpc_rpc_proto protoreflect.FileDescriptor + +var file_pkgs_rpc_rpc_proto_rawDesc = []byte{ + 0x0a, 0x12, 0x70, 0x6b, 0x67, 0x73, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x72, 0x70, 0x63, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x03, 0x72, 0x70, 0x63, 0x22, 0x23, 0x0a, 0x07, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x24, + 0x0a, 0x08, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x4b, 0x0a, 0x0b, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64, 0x44, + 0x61, 0x74, 0x61, 0x12, 0x28, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x14, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64, 0x44, + 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x44, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x44, 0x61, 0x74, + 0x61, 0x22, 0x39, 0x0a, 0x09, 0x43, 0x6f, 0x64, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, + 0x0a, 0x04, 0x43, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x43, 0x6f, + 0x64, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2a, 0x3c, 0x0a, 0x0f, + 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x09, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x45, 0x4f, + 0x46, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x4e, 0x65, 0x77, 0x50, 0x61, 0x72, 0x74, 0x10, 0x02, + 0x12, 0x08, 0x0a, 0x04, 0x44, 0x61, 0x74, 0x61, 0x10, 0x03, 0x42, 0x36, 0x5a, 0x34, 0x67, 0x69, + 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x63, 0x6e, 0x2f, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x72, 0x65, 0x61, 0x6d, 0x2f, 0x6a, 0x63, 0x73, 0x2d, 0x70, 0x75, 0x62, 0x2f, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x70, 0x6b, 0x67, 0x73, 0x2f, 0x72, 0x70, 0x63, 0x3b, 0x72, + 0x70, 0x63, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_pkgs_rpc_rpc_proto_rawDescOnce sync.Once + file_pkgs_rpc_rpc_proto_rawDescData = file_pkgs_rpc_rpc_proto_rawDesc +) + +func file_pkgs_rpc_rpc_proto_rawDescGZIP() []byte { + file_pkgs_rpc_rpc_proto_rawDescOnce.Do(func() { + file_pkgs_rpc_rpc_proto_rawDescData = protoimpl.X.CompressGZIP(file_pkgs_rpc_rpc_proto_rawDescData) + }) + return file_pkgs_rpc_rpc_proto_rawDescData +} + +var file_pkgs_rpc_rpc_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_pkgs_rpc_rpc_proto_msgTypes = make([]protoimpl.MessageInfo, 4) +var file_pkgs_rpc_rpc_proto_goTypes = []any{ + (ChunkedDataType)(0), // 0: rpc.ChunkedDataType + (*Request)(nil), // 1: rpc.Request + (*Response)(nil), // 2: rpc.Response + (*ChunkedData)(nil), // 3: rpc.ChunkedData + (*CodeError)(nil), // 4: rpc.CodeError +} +var file_pkgs_rpc_rpc_proto_depIdxs = []int32{ + 0, // 0: rpc.ChunkedData.Type:type_name -> rpc.ChunkedDataType + 1, // [1:1] is the sub-list for method output_type + 1, // [1:1] is the sub-list for method input_type + 1, // [1:1] is the sub-list for extension type_name + 1, // [1:1] is the sub-list for extension extendee + 0, // [0:1] is the sub-list for field type_name +} + +func init() { file_pkgs_rpc_rpc_proto_init() } +func file_pkgs_rpc_rpc_proto_init() { + if File_pkgs_rpc_rpc_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_pkgs_rpc_rpc_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*Request); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_pkgs_rpc_rpc_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*Response); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_pkgs_rpc_rpc_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*ChunkedData); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_pkgs_rpc_rpc_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*CodeError); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_pkgs_rpc_rpc_proto_rawDesc, + NumEnums: 1, + NumMessages: 4, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_pkgs_rpc_rpc_proto_goTypes, + DependencyIndexes: file_pkgs_rpc_rpc_proto_depIdxs, + EnumInfos: file_pkgs_rpc_rpc_proto_enumTypes, + MessageInfos: file_pkgs_rpc_rpc_proto_msgTypes, + }.Build() + File_pkgs_rpc_rpc_proto = out.File + file_pkgs_rpc_rpc_proto_rawDesc = nil + file_pkgs_rpc_rpc_proto_goTypes = nil + file_pkgs_rpc_rpc_proto_depIdxs = nil +} diff --git a/common/pkgs/rpc/rpc.proto b/common/pkgs/rpc/rpc.proto new file mode 100644 index 0000000..6c612b5 --- /dev/null +++ b/common/pkgs/rpc/rpc.proto @@ -0,0 +1,31 @@ +syntax = "proto3"; + +package rpc; + +option go_package = "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc;rpc"; + +message Request { + bytes Payload = 1; +} + +message Response { + bytes Payload = 1; +} + + +enum ChunkedDataType { + Error = 0; + EOF = 1; + NewPart = 2; + Data = 3; +} + +message ChunkedData { + ChunkedDataType Type = 1; + bytes Data = 2; +} + +message CodeError { + string Code = 1; + string Message = 2; +} \ No newline at end of file diff --git a/common/pkgs/rpc/server.go b/common/pkgs/rpc/server.go new file mode 100644 index 0000000..30b03da --- /dev/null +++ b/common/pkgs/rpc/server.go @@ -0,0 +1,60 @@ +package rpc + +import ( + "net" + + "gitlink.org.cn/cloudream/common/pkgs/async" + "google.golang.org/grpc" +) + +type ServerEventChan = async.UnboundChannel[RPCServerEvent] + +type RPCServerEvent interface { + IsRPCServerEvent() +} + +type ExitEvent struct { + RPCServerEvent + Err error +} + +type Config struct { + Listen string `json:"listen"` +} + +type ServerBase struct { + cfg Config + grpcSvr *grpc.Server + srvImpl any + svcDesc *grpc.ServiceDesc +} + +func NewServerBase(cfg Config, srvImpl any, svcDesc *grpc.ServiceDesc) *ServerBase { + return &ServerBase{ + cfg: cfg, + srvImpl: srvImpl, + svcDesc: svcDesc, + } +} + +func (s *ServerBase) Start() *ServerEventChan { + ch := async.NewUnboundChannel[RPCServerEvent]() + go func() { + lis, err := net.Listen("tcp", s.cfg.Listen) + if err != nil { + ch.Send(&ExitEvent{Err: err}) + return + } + s.grpcSvr = grpc.NewServer() + s.grpcSvr.RegisterService(s.svcDesc, s.srvImpl) + err = s.grpcSvr.Serve(lis) + ch.Send(&ExitEvent{Err: err}) + }() + return ch +} + +func (s *ServerBase) Stop() { + if s.grpcSvr != nil { + s.grpcSvr.Stop() + } +} diff --git a/common/pkgs/rpc/utils.go b/common/pkgs/rpc/utils.go new file mode 100644 index 0000000..562b1bd --- /dev/null +++ b/common/pkgs/rpc/utils.go @@ -0,0 +1,276 @@ +package rpc + +import ( + "fmt" + "io" + + "gitlink.org.cn/cloudream/common/consts/errorcode" + "gitlink.org.cn/cloudream/common/utils/io2" + "gitlink.org.cn/cloudream/common/utils/serder" + "golang.org/x/net/context" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +func UnaryClient[Resp, Req any](apiFn func(context.Context, *Request, ...grpc.CallOption) (*Response, error), ctx context.Context, req Req) (Resp, *CodeError) { + data, err := serder.ObjectToJSONEx(req) + if err != nil { + var resp Resp + return resp, Failed(errorcode.OperationFailed, err.Error()) + } + + resp, err := apiFn(ctx, &Request{ + Payload: data, + }) + if err != nil { + var resp Resp + return resp, getCodeError(err) + } + + ret, err := serder.JSONToObjectEx[Resp](resp.Payload) + if err != nil { + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + return ret, nil +} + +func UnaryServer[Resp, Req any](apiFn func(context.Context, Req) (Resp, *CodeError), ctx context.Context, req *Request) (*Response, error) { + rreq, err := serder.JSONToObjectEx[Req](req.Payload) + if err != nil { + return nil, makeCodeError(errorcode.OperationFailed, err.Error()) + } + + ret, cerr := apiFn(ctx, rreq) + if cerr != nil { + return nil, wrapCodeError(cerr) + } + + data, err := serder.ObjectToJSONEx(ret) + if err != nil { + return nil, makeCodeError(errorcode.OperationFailed, err.Error()) + } + + return &Response{ + Payload: data, + }, nil +} + +type UploadStreamAPIClient interface { + GRPCChunkedWriter + CloseAndRecv() (*Response, error) +} +type UploadStreamAPIServer interface { + GRPCChunkedReader + SendAndClose(*Response) error + Context() context.Context +} +type UploadStreamReq interface { + GetStream() io.Reader + SetStream(io.Reader) +} + +// 封装了上传流API的客户端逻辑。记得将Req里的Stream字段设置为不需要序列化(json:"-") +func UploadStreamClient[Resp any, Req UploadStreamReq, APIRet UploadStreamAPIClient](apiFn func(context.Context, ...grpc.CallOption) (APIRet, error), ctx context.Context, req Req) (Resp, *CodeError) { + stream := req.GetStream() + + var ret Resp + data, err := serder.ObjectToJSONEx(req) + if err != nil { + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + + ctx2, cancelFn := context.WithCancel(ctx) + defer cancelFn() + + cli, err := apiFn(ctx2) + if err != nil { + return ret, getCodeError(err) + } + + cw := NewChunkedWriter(cli) + err = cw.WriteDataPart("", data) + if err != nil { + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + + _, err = cw.WriteStreamPart("", stream) + if err != nil { + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + + err = cw.Finish() + if err != nil { + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + + resp, err := cli.CloseAndRecv() + if err != nil { + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + + ret, err = serder.JSONToObjectEx[Resp](resp.Payload) + if err != nil { + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + return ret, nil +} + +func UploadStreamServer[Resp any, Req UploadStreamReq, APIRet UploadStreamAPIServer](apiFn func(context.Context, Req) (Resp, *CodeError), req APIRet) error { + cr := NewChunkedReader(req) + _, data, err := cr.NextDataPart() + if err != nil { + return makeCodeError(errorcode.OperationFailed, err.Error()) + } + + _, pr, err := cr.NextPart() + if err != nil { + return makeCodeError(errorcode.OperationFailed, err.Error()) + } + + rreq, err := serder.JSONToObjectEx[Req](data) + if err != nil { + return makeCodeError(errorcode.OperationFailed, err.Error()) + } + rreq.SetStream(pr) + + resp, cerr := apiFn(req.Context(), rreq) + if cerr != nil { + return wrapCodeError(cerr) + } + + respData, err := serder.ObjectToJSONEx(resp) + if err != nil { + return makeCodeError(errorcode.OperationFailed, err.Error()) + } + + err = req.SendAndClose(&Response{Payload: respData}) + if err != nil { + return makeCodeError(errorcode.OperationFailed, err.Error()) + } + + return nil +} + +type DownloadStreamAPIClient interface { + GRPCChunkedReader +} +type DownloadStreamAPIServer interface { + GRPCChunkedWriter + Context() context.Context +} +type DownloadStreamResp interface { + GetStream() io.ReadCloser + SetStream(io.ReadCloser) +} + +// 封装了下载流API的客户端逻辑。记得将Resp里的Stream字段设置为不需要序列化(json:"-") +func DownloadStreamClient[Resp DownloadStreamResp, Req any, APIRet DownloadStreamAPIClient](apiFn func(context.Context, *Request, ...grpc.CallOption) (APIRet, error), ctx context.Context, req Req) (Resp, *CodeError) { + var ret Resp + data, err := serder.ObjectToJSONEx(req) + if err != nil { + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + + ctx2, cancelFn := context.WithCancel(ctx) + + cli, err := apiFn(ctx2, &Request{Payload: data}) + if err != nil { + cancelFn() + return ret, getCodeError(err) + } + + cr := NewChunkedReader(cli) + + _, data, err = cr.NextDataPart() + if err != nil { + cancelFn() + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + + resp, err := serder.JSONToObjectEx[Resp](data) + if err != nil { + cancelFn() + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + + _, pr, err := cr.NextPart() + if err != nil { + cancelFn() + return ret, Failed(errorcode.OperationFailed, err.Error()) + } + + resp.SetStream(io2.DelegateReadCloser(pr, func() error { + cancelFn() + return nil + })) + + return resp, nil +} + +func DownloadStreamServer[Resp DownloadStreamResp, Req any, APIRet DownloadStreamAPIServer](apiFn func(context.Context, Req) (Resp, *CodeError), req *Request, ret APIRet) error { + rreq, err := serder.JSONToObjectEx[Req](req.Payload) + if err != nil { + return makeCodeError(errorcode.OperationFailed, err.Error()) + } + + resp, cerr := apiFn(ret.Context(), rreq) + if cerr != nil { + return wrapCodeError(cerr) + } + + cw := NewChunkedWriter(ret) + data, err := serder.ObjectToJSONEx(resp) + if err != nil { + return makeCodeError(errorcode.OperationFailed, err.Error()) + } + + err = cw.WriteDataPart("", data) + if err != nil { + return makeCodeError(errorcode.OperationFailed, err.Error()) + } + + _, err = cw.WriteStreamPart("", resp.GetStream()) + if err != nil { + return makeCodeError(errorcode.OperationFailed, err.Error()) + } + + return nil +} + +func Failed(errCode string, format string, args ...any) *CodeError { + return &CodeError{ + Code: errCode, + Message: fmt.Sprintf(format, args...), + } +} + +func (c *CodeError) Error() string { + return fmt.Sprintf("code: %s, message: %s", c.Code, c.Message) +} + +func getCodeError(err error) *CodeError { + status, ok := status.FromError(err) + + if ok { + dts := status.Details() + if len(dts) > 0 { + ce, ok := dts[0].(*CodeError) + if ok { + return ce + } + } + } + + return Failed(errorcode.OperationFailed, err.Error()) +} + +func makeCodeError(code string, msg string) error { + ce, _ := status.New(codes.Unknown, "custom error").WithDetails(Failed(code, msg)) + return ce.Err() +} + +func wrapCodeError(ce *CodeError) error { + e, _ := status.New(codes.Unknown, "custom error").WithDetails(ce) + return e.Err() +} diff --git a/coordinator/internal/ticktock/check_hub_state.go b/coordinator/internal/ticktock/check_hub_state.go index 063d42c..6743cbf 100644 --- a/coordinator/internal/ticktock/check_hub_state.go +++ b/coordinator/internal/ticktock/check_hub_state.go @@ -1,15 +1,15 @@ package ticktock import ( + "context" "fmt" "time" "gitlink.org.cn/cloudream/common/pkgs/logger" - "gitlink.org.cn/cloudream/common/pkgs/mq" "gitlink.org.cn/cloudream/common/utils/reflect2" "gitlink.org.cn/cloudream/jcs-pub/common/consts" stgglb "gitlink.org.cn/cloudream/jcs-pub/common/globals" - hubmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/hub" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" ) @@ -45,14 +45,19 @@ func (j *CheckHubState) Execute(t *TickTock) { func (j *CheckHubState) checkOne(t *TickTock, hub cortypes.Hub) error { log := logger.WithType[CheckHubState]("TickTock") - agtCli, err := stgglb.HubMQPool.Acquire(hub.HubID) - if err != nil { - return fmt.Errorf("new hub mq client: %w", err) + addr, ok := hub.Address.(*cortypes.GRPCAddressInfo) + if !ok { + return fmt.Errorf("hub has no grpc address") } - defer stgglb.HubMQPool.Release(agtCli) - _, err = agtCli.GetState(hubmq.NewGetState(), mq.RequestOption{Timeout: time.Second * 30}) - if err != nil { + agtCli := stgglb.HubRPCPool.Get(stgglb.SelectGRPCAddress(hub, *addr)) + defer agtCli.Release() + + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + _, cerr := agtCli.GetState(ctx, &hubrpc.GetState{}) + if cerr != nil { if hub.LastReportTime != nil { if time.Since(*hub.LastReportTime) > t.cfg.HubUnavailableTime { err := t.db.Hub().UpdateState(t.db.DefCtx(), hub.HubID, consts.HubStateUnavailable) @@ -68,11 +73,11 @@ func (j *CheckHubState) checkOne(t *TickTock, hub cortypes.Hub) error { } } - return fmt.Errorf("getting state: %w", err) + return fmt.Errorf("getting state: %w", cerr) } // TODO 如果以后还有其他的状态,要判断哪些状态下能设置Normal - err = t.db.Hub().UpdateState(t.db.DefCtx(), hub.HubID, consts.HubStateNormal) + err := t.db.Hub().UpdateState(t.db.DefCtx(), hub.HubID, consts.HubStateNormal) if err != nil { log.Warnf("set hub %v state: %s", hub, err.Error()) } diff --git a/hub/internal/cmd/serve.go b/hub/internal/cmd/serve.go index 8ec6246..98923d6 100644 --- a/hub/internal/cmd/serve.go +++ b/hub/internal/cmd/serve.go @@ -3,31 +3,25 @@ package cmd import ( "context" "fmt" - "net" "os" "github.com/go-co-op/gocron/v2" "github.com/spf13/cobra" + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/storage/pool" "gitlink.org.cn/cloudream/jcs-pub/hub/internal/http" + myrpc "gitlink.org.cn/cloudream/jcs-pub/hub/internal/rpc" "gitlink.org.cn/cloudream/common/pkgs/ioswitch/exec" "gitlink.org.cn/cloudream/common/pkgs/logger" - "gitlink.org.cn/cloudream/common/pkgs/mq" stgglb "gitlink.org.cn/cloudream/jcs-pub/common/globals" "gitlink.org.cn/cloudream/jcs-pub/common/models/datamap" - hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/grpc/hub" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/sysevent" cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" "gitlink.org.cn/cloudream/jcs-pub/hub/internal/config" - "google.golang.org/grpc" - coormq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/coordinator" - hubmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/hub" - - grpcsvc "gitlink.org.cn/cloudream/jcs-pub/hub/internal/grpc" - cmdsvc "gitlink.org.cn/cloudream/jcs-pub/hub/internal/mq" ) func init() { @@ -61,7 +55,7 @@ func serve(configPath string, httpAddr string) { stgglb.InitLocal(config.Cfg().Local) stgglb.InitMQPool(config.Cfg().RabbitMQ) - stgglb.InitHubRPCPool(&hubrpc.PoolConfig{}) + stgglb.InitHubRPCPool(hubrpc.PoolConfig{}) // stgglb.Stats.SetupHubStorageTransfer(*config.Cfg().Local.HubID) // stgglb.Stats.SetupHubTransfer(*config.Cfg().Local.HubID) // 获取Hub配置 @@ -145,29 +139,36 @@ func serve(configPath string, httpAddr string) { sch.Start() defer sch.Shutdown() - // 启动命令服务器 - // TODO 需要设计HubID持久化机制 - hubSvr, err := hubmq.NewServer(cmdsvc.NewService(stgPool), config.Cfg().ID, config.Cfg().RabbitMQ) - if err != nil { - logger.Fatalf("new hub server failed, err: %s", err.Error()) - } - hubSvr.OnError(func(err error) { - logger.Warnf("hub server err: %s", err.Error()) - }) - go serveHubServer(hubSvr) + // RPC服务 + rpcSvr := hubrpc.NewServer(config.Cfg().RPC, myrpc.NewService(&worker, stgPool)) + rpcSvrChan := rpcSvr.Start() + defer rpcSvr.Stop() - // 启动GRPC服务 - listenAddr := config.Cfg().GRPC.MakeListenAddress() - lis, err := net.Listen("tcp", listenAddr) - if err != nil { - logger.Fatalf("listen on %s failed, err: %s", listenAddr, err.Error()) + /// 开始监听各个模块的事件 + rpcEvt := rpcSvrChan.Receive() + +loop: + for { + select { + case e := <-rpcEvt.Chan(): + if e.Err != nil { + logger.Errorf("receive rpc event: %v", e.Err) + break loop + } + + switch e := e.Value.(type) { + case rpc.ExitEvent: + if e.Err != nil { + logger.Errorf("rpc server exited with error: %v", e.Err) + } else { + logger.Infof("rpc server exited") + } + break loop + } + rpcEvt = rpcSvrChan.Receive() + } } - s := grpc.NewServer() - hubrpc.RegisterHubServer(s, grpcsvc.NewService(&worker, stgPool)) - go serveGRPC(s, lis) - foever := make(chan struct{}) - <-foever } func downloadHubConfig() coormq.GetHubConfigResp { @@ -244,51 +245,6 @@ func setupTickTask(hubPool *pool.Pool, evtPub *sysevent.Publisher) gocron.Schedu return sch } -func serveHubServer(server *hubmq.Server) { - logger.Info("start serving command server") - - ch := server.Start() -loop: - for { - val, err := ch.Receive() - if err != nil { - logger.Errorf("command server stopped with error: %s", err.Error()) - break - } - - switch val := val.(type) { - case error: - logger.Errorf("rabbitmq connect with error: %v", val) - case mq.ServerExit: - if val.Error != nil { - logger.Errorf("rabbitmq server exit with error: %v", val.Error) - } else { - logger.Info("rabbitmq server exit") - } - break loop - } - } - logger.Info("command server stopped") - - // TODO 仅简单结束了程序 - os.Exit(1) -} - -func serveGRPC(s *grpc.Server, lis net.Listener) { - logger.Info("start serving grpc") - - err := s.Serve(lis) - - if err != nil { - logger.Errorf("grpc stopped with error: %s", err.Error()) - } - - logger.Info("grpc stopped") - - // TODO 仅简单结束了程序 - os.Exit(1) -} - func serveHTTP(server *http.Server) { logger.Info("start serving http") diff --git a/hub/internal/config/config.go b/hub/internal/config/config.go index f832ea3..0ffccb4 100644 --- a/hub/internal/config/config.go +++ b/hub/internal/config/config.go @@ -6,14 +6,14 @@ import ( c "gitlink.org.cn/cloudream/common/utils/config" stgglb "gitlink.org.cn/cloudream/jcs-pub/common/globals" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/connectivity" - "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/grpc" + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" cortypes "gitlink.org.cn/cloudream/jcs-pub/coordinator/types" ) type Config struct { ID cortypes.HubID `json:"id"` Local stgglb.LocalMachineInfo `json:"local"` - GRPC *grpc.Config `json:"grpc"` + RPC rpc.Config `json:"rpc"` Logger log.Config `json:"logger"` RabbitMQ mq.Config `json:"rabbitMQ"` Connectivity connectivity.Config `json:"connectivity"` diff --git a/hub/internal/grpc/io.go b/hub/internal/grpc/io.go deleted file mode 100644 index d0b8edf..0000000 --- a/hub/internal/grpc/io.go +++ /dev/null @@ -1,240 +0,0 @@ -package grpc - -import ( - "context" - "fmt" - "io" - "time" - - "github.com/inhies/go-bytesize" - "gitlink.org.cn/cloudream/common/pkgs/ioswitch/exec" - "gitlink.org.cn/cloudream/common/pkgs/logger" - "gitlink.org.cn/cloudream/common/utils/io2" - "gitlink.org.cn/cloudream/common/utils/serder" - hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/grpc/hub" -) - -func (s *Service) ExecuteIOPlan(ctx context.Context, req *hubrpc.ExecuteIOPlanReq) (*hubrpc.ExecuteIOPlanResp, error) { - plan, err := serder.JSONToObjectEx[exec.Plan]([]byte(req.Plan)) - if err != nil { - return nil, fmt.Errorf("deserializing plan: %w", err) - } - - log := logger.WithField("PlanID", plan.ID) - log.Infof("begin execute io plan") - - sw := exec.NewExecutor(plan) - - s.swWorker.Add(sw) - defer s.swWorker.Remove(sw) - - execCtx := exec.NewWithContext(ctx) - exec.SetValueByType(execCtx, s.stgPool) - _, err = sw.Run(execCtx) - if err != nil { - log.Warnf("running io plan: %v", err) - return nil, fmt.Errorf("running io plan: %w", err) - } - - log.Infof("plan finished") - return &hubrpc.ExecuteIOPlanResp{}, nil -} - -func (s *Service) SendStream(server hubrpc.Hub_SendStreamServer) error { - msg, err := server.Recv() - if err != nil { - return fmt.Errorf("recving stream id packet: %w", err) - } - if msg.Type != hubrpc.StreamDataPacketType_SendArgs { - return fmt.Errorf("first packet must be a SendArgs packet") - } - - logger. - WithField("PlanID", msg.PlanID). - WithField("VarID", msg.VarID). - Debugf("stream input") - - // 同一批Plan中每个节点的Plan的启动时间有先后,但最多不应该超过30秒 - ctx, cancel := context.WithTimeout(server.Context(), time.Second*30) - defer cancel() - - sw := s.swWorker.FindByIDContexted(ctx, exec.PlanID(msg.PlanID)) - if sw == nil { - return fmt.Errorf("plan not found") - } - - pr, pw := io.Pipe() - - varID := exec.VarID(msg.VarID) - sw.PutVar(varID, &exec.StreamValue{Stream: pr}) - - // 然后读取文件数据 - var recvSize int64 - for { - msg, err := server.Recv() - - // 读取客户端数据失败 - // 即使err是io.EOF,只要没有收到客户端包含EOF数据包就被断开了连接,就认为接收失败 - if err != nil { - // 关闭文件写入 - pw.CloseWithError(io.ErrClosedPipe) - logger.WithField("ReceiveSize", recvSize). - WithField("VarID", varID). - Warnf("recv message failed, err: %s", err.Error()) - return fmt.Errorf("recv message failed, err: %w", err) - } - - err = io2.WriteAll(pw, msg.Data) - if err != nil { - // 关闭文件写入 - pw.CloseWithError(io.ErrClosedPipe) - logger.Warnf("write data to file failed, err: %s", err.Error()) - return fmt.Errorf("write data to file failed, err: %w", err) - } - - recvSize += int64(len(msg.Data)) - - if msg.Type == hubrpc.StreamDataPacketType_EOF { - // 客户端明确说明文件传输已经结束,那么结束写入,获得文件Hash - err := pw.Close() - if err != nil { - logger.Warnf("finish writing failed, err: %s", err.Error()) - return fmt.Errorf("finish writing failed, err: %w", err) - } - - // 并将结果返回到客户端 - err = server.SendAndClose(&hubrpc.SendStreamResp{}) - if err != nil { - logger.Warnf("send response failed, err: %s", err.Error()) - return fmt.Errorf("send response failed, err: %w", err) - } - - return nil - } - } -} - -func (s *Service) GetStream(req *hubrpc.GetStreamReq, server hubrpc.Hub_GetStreamServer) error { - logger. - WithField("PlanID", req.PlanID). - WithField("VarID", req.VarID). - Debugf("stream output") - - // 同上 - ctx, cancel := context.WithTimeout(server.Context(), time.Second*30) - defer cancel() - - sw := s.swWorker.FindByIDContexted(ctx, exec.PlanID(req.PlanID)) - if sw == nil { - return fmt.Errorf("plan not found") - } - - signal, err := serder.JSONToObjectEx[exec.VarValue]([]byte(req.Signal)) - if err != nil { - return fmt.Errorf("deserializing var: %w", err) - } - - sw.PutVar(exec.VarID(req.SignalID), signal) - - strVar, err := exec.BindVar[*exec.StreamValue](sw, server.Context(), exec.VarID(req.VarID)) - if err != nil { - return fmt.Errorf("binding vars: %w", err) - } - - reader := strVar.Stream - defer reader.Close() - - buf := make([]byte, 1024*64) - readAllCnt := 0 - startTime := time.Now() - for { - readCnt, err := reader.Read(buf) - - if readCnt > 0 { - readAllCnt += readCnt - err = server.Send(&hubrpc.StreamDataPacket{ - Type: hubrpc.StreamDataPacketType_Data, - Data: buf[:readCnt], - }) - if err != nil { - logger. - WithField("PlanID", req.PlanID). - WithField("VarID", req.VarID). - Warnf("send stream data failed, err: %s", err.Error()) - return fmt.Errorf("send stream data failed, err: %w", err) - } - } - - // 文件读取完毕 - if err == io.EOF { - dt := time.Since(startTime) - logger. - WithField("PlanID", req.PlanID). - WithField("VarID", req.VarID). - Debugf("send data size %d in %v, speed %v/s", readAllCnt, dt, bytesize.New(float64(readAllCnt)/dt.Seconds())) - // 发送EOF消息 - server.Send(&hubrpc.StreamDataPacket{ - Type: hubrpc.StreamDataPacketType_EOF, - }) - return nil - } - - // io.ErrUnexpectedEOF没有读满整个buf就遇到了EOF,此时正常发送剩余数据即可。除了这两个错误之外,其他错误都中断操作 - if err != nil && err != io.ErrUnexpectedEOF { - logger. - WithField("PlanID", req.PlanID). - WithField("VarID", req.VarID). - Warnf("reading stream data: %s", err.Error()) - return fmt.Errorf("reading stream data: %w", err) - } - } -} - -func (s *Service) SendVar(ctx context.Context, req *hubrpc.SendVarReq) (*hubrpc.SendVarResp, error) { - ctx, cancel := context.WithTimeout(ctx, time.Second*30) - defer cancel() - - sw := s.swWorker.FindByIDContexted(ctx, exec.PlanID(req.PlanID)) - if sw == nil { - return nil, fmt.Errorf("plan not found") - } - - v, err := serder.JSONToObjectEx[exec.VarValue]([]byte(req.VarValue)) - if err != nil { - return nil, fmt.Errorf("deserializing var: %w", err) - } - - sw.PutVar(exec.VarID(req.VarID), v) - return &hubrpc.SendVarResp{}, nil -} - -func (s *Service) GetVar(ctx context.Context, req *hubrpc.GetVarReq) (*hubrpc.GetVarResp, error) { - ctx2, cancel := context.WithTimeout(ctx, time.Second*30) - defer cancel() - - sw := s.swWorker.FindByIDContexted(ctx2, exec.PlanID(req.PlanID)) - if sw == nil { - return nil, fmt.Errorf("plan not found") - } - - signal, err := serder.JSONToObjectEx[exec.VarValue]([]byte(req.Signal)) - if err != nil { - return nil, fmt.Errorf("deserializing var: %w", err) - } - - sw.PutVar(exec.VarID(req.SignalID), signal) - - v, err := sw.BindVar(ctx, exec.VarID(req.VarID)) - if err != nil { - return nil, fmt.Errorf("binding vars: %w", err) - } - - vd, err := serder.ObjectToJSONEx(v) - if err != nil { - return nil, fmt.Errorf("serializing var: %w", err) - } - - return &hubrpc.GetVarResp{ - Var: string(vd), - }, nil -} diff --git a/hub/internal/grpc/ping.go b/hub/internal/grpc/ping.go deleted file mode 100644 index 7103901..0000000 --- a/hub/internal/grpc/ping.go +++ /dev/null @@ -1,20 +0,0 @@ -package grpc - -import ( - "context" - - hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/grpc/hub" -) - -// Ping 是一个RPC方法,用于验证服务的可用性。 -// -// 参数: -// context.Context: 传递上下文信息,包括请求的元数据和取消信号。 -// *hubrpc.PingReq: 传递的Ping请求数据,当前实现中未使用。 -// -// 返回值: -// *hubrpc.PingResp: Ping响应数据,当前实现中始终返回空响应。 -// error: 如果处理过程中出现错误,则返回错误信息;否则返回nil。 -func (s *Service) Ping(context.Context, *hubrpc.PingReq) (*hubrpc.PingResp, error) { - return &hubrpc.PingResp{}, nil -} diff --git a/hub/internal/mq/cache.go b/hub/internal/mq/cache.go deleted file mode 100644 index 32fdd23..0000000 --- a/hub/internal/mq/cache.go +++ /dev/null @@ -1,43 +0,0 @@ -package mq - -import ( - "fmt" - - "gitlink.org.cn/cloudream/common/consts/errorcode" - "gitlink.org.cn/cloudream/common/pkgs/mq" - clitypes "gitlink.org.cn/cloudream/jcs-pub/client/types" - agtmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/hub" -) - -func (svc *Service) CheckCache(msg *agtmq.CheckCache) (*agtmq.CheckCacheResp, *mq.CodeMessage) { - store, err := svc.stgPool.GetShardStore(&msg.UserSpace) - if err != nil { - return nil, mq.Failed(errorcode.OperationFailed, fmt.Sprintf("get shard store of user space %v: %v", msg.UserSpace, err)) - } - - infos, err := store.ListAll() - if err != nil { - return nil, mq.Failed(errorcode.OperationFailed, fmt.Sprintf("listting file in shard store: %v", err)) - } - - var fileHashes []clitypes.FileHash - for _, info := range infos { - fileHashes = append(fileHashes, info.Hash) - } - - return mq.ReplyOK(agtmq.NewCheckCacheResp(fileHashes)) -} - -func (svc *Service) CacheGC(msg *agtmq.CacheGC) (*agtmq.CacheGCResp, *mq.CodeMessage) { - store, err := svc.stgPool.GetShardStore(&msg.UserSpace) - if err != nil { - return nil, mq.Failed(errorcode.OperationFailed, fmt.Sprintf("get shard store of user space %v: %v", msg.UserSpace, err)) - } - - err = store.GC(msg.Avaiables) - if err != nil { - return nil, mq.Failed(errorcode.OperationFailed, fmt.Sprintf("purging cache: %v", err)) - } - - return mq.ReplyOK(agtmq.RespCacheGC()) -} diff --git a/hub/internal/mq/hub.go b/hub/internal/mq/hub.go deleted file mode 100644 index 8abac81..0000000 --- a/hub/internal/mq/hub.go +++ /dev/null @@ -1,10 +0,0 @@ -package mq - -import ( - "gitlink.org.cn/cloudream/common/pkgs/mq" - hubmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/hub" -) - -func (svc *Service) GetState(msg *hubmq.GetState) (*hubmq.GetStateResp, *mq.CodeMessage) { - return mq.ReplyOK(hubmq.NewGetStateResp()) -} diff --git a/hub/internal/mq/service.go b/hub/internal/mq/service.go deleted file mode 100644 index dbb2c15..0000000 --- a/hub/internal/mq/service.go +++ /dev/null @@ -1,15 +0,0 @@ -package mq - -import ( - "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/storage/pool" -) - -type Service struct { - stgPool *pool.Pool -} - -func NewService(stgPool *pool.Pool) *Service { - return &Service{ - stgPool: stgPool, - } -} diff --git a/hub/internal/mq/user_space.go b/hub/internal/mq/user_space.go deleted file mode 100644 index 02a5619..0000000 --- a/hub/internal/mq/user_space.go +++ /dev/null @@ -1,45 +0,0 @@ -package mq - -import ( - "gitlink.org.cn/cloudream/common/consts/errorcode" - "gitlink.org.cn/cloudream/common/pkgs/logger" - "gitlink.org.cn/cloudream/common/pkgs/mq" - hubmq "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/mq/hub" -) - -func (svc *Service) PublicStoreListAll(msg *hubmq.PublicStoreListAll) (*hubmq.PublicStoreListAllResp, *mq.CodeMessage) { - pub, err := svc.stgPool.GetPublicStore(&msg.UserSpace) - if err != nil { - return nil, mq.Failed(errorcode.OperationFailed, err.Error()) - } - - es, err := pub.ListAll(msg.Path) - if err != nil { - return nil, mq.Failed(errorcode.OperationFailed, err.Error()) - } - - return mq.ReplyOK(&hubmq.PublicStoreListAllResp{ - Entries: es, - }) -} - -func (svc *Service) PublicStoreMkdirs(msg *hubmq.PublicStoreMkdirs) (*hubmq.PublicStoreMkdirsResp, *mq.CodeMessage) { - pub, err := svc.stgPool.GetPublicStore(&msg.UserSpace) - if err != nil { - return nil, mq.Failed(errorcode.OperationFailed, err.Error()) - } - - var suc []bool - for _, p := range msg.Pathes { - if err := pub.Mkdir(p); err != nil { - suc = append(suc, false) - logger.Warnf("userspace %v mkdir %s: %v", msg.UserSpace, p, err) - } else { - suc = append(suc, true) - } - } - - return mq.ReplyOK(&hubmq.PublicStoreMkdirsResp{ - Successes: suc, - }) -} diff --git a/hub/internal/rpc/cache.go b/hub/internal/rpc/cache.go new file mode 100644 index 0000000..e557f89 --- /dev/null +++ b/hub/internal/rpc/cache.go @@ -0,0 +1,44 @@ +package rpc + +import ( + "context" + "fmt" + + "gitlink.org.cn/cloudream/common/consts/errorcode" + clitypes "gitlink.org.cn/cloudream/jcs-pub/client/types" + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" +) + +func (svc *Service) CheckCache(context context.Context, msg *hubrpc.CheckCache) (*hubrpc.CheckCacheResp, *rpc.CodeError) { + store, err := svc.stgPool.GetShardStore(&msg.UserSpace) + if err != nil { + return nil, rpc.Failed(errorcode.OperationFailed, fmt.Sprintf("get shard store of user space %v: %v", msg.UserSpace, err)) + } + + infos, err := store.ListAll() + if err != nil { + return nil, rpc.Failed(errorcode.OperationFailed, fmt.Sprintf("listting file in shard store: %v", err)) + } + + var fileHashes []clitypes.FileHash + for _, info := range infos { + fileHashes = append(fileHashes, info.Hash) + } + + return &hubrpc.CheckCacheResp{FileHashes: fileHashes}, nil +} + +func (svc *Service) CacheGC(context context.Context, msg *hubrpc.CacheGC) (*hubrpc.CacheGCResp, *rpc.CodeError) { + store, err := svc.stgPool.GetShardStore(&msg.UserSpace) + if err != nil { + return nil, rpc.Failed(errorcode.OperationFailed, fmt.Sprintf("get shard store of user space %v: %v", msg.UserSpace, err)) + } + + err = store.GC(msg.Availables) + if err != nil { + return nil, rpc.Failed(errorcode.OperationFailed, fmt.Sprintf("purging cache: %v", err)) + } + + return &hubrpc.CacheGCResp{}, nil +} diff --git a/hub/internal/rpc/ioswitch.go b/hub/internal/rpc/ioswitch.go new file mode 100644 index 0000000..a97e7be --- /dev/null +++ b/hub/internal/rpc/ioswitch.go @@ -0,0 +1,127 @@ +package rpc + +import ( + "context" + "time" + + "gitlink.org.cn/cloudream/common/consts/errorcode" + "gitlink.org.cn/cloudream/common/pkgs/future" + "gitlink.org.cn/cloudream/common/pkgs/ioswitch/exec" + "gitlink.org.cn/cloudream/common/pkgs/logger" + "gitlink.org.cn/cloudream/common/utils/io2" + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" +) + +func (s *Service) ExecuteIOPlan(ctx context.Context, req *hubrpc.ExecuteIOPlan) (*hubrpc.ExecuteIOPlanResp, *rpc.CodeError) { + log := logger.WithField("PlanID", req.Plan.ID) + log.Infof("begin execute io plan") + + sw := exec.NewExecutor(req.Plan) + + s.swWorker.Add(sw) + defer s.swWorker.Remove(sw) + + execCtx := exec.NewWithContext(ctx) + exec.SetValueByType(execCtx, s.stgPool) + _, err := sw.Run(execCtx) + if err != nil { + log.Warnf("running io plan: %v", err) + return nil, rpc.Failed(errorcode.OperationFailed, "%v", err) + } + + log.Infof("plan finished") + return &hubrpc.ExecuteIOPlanResp{}, nil +} + +func (s *Service) SendIOStream(ctx context.Context, req *hubrpc.SendIOStream) (*hubrpc.SendIOStreamResp, *rpc.CodeError) { + logger. + WithField("PlanID", req.PlanID). + WithField("VarID", req.VarID). + Debugf("stream input") + + // 同一批Plan中每个节点的Plan的启动时间有先后,但最多不应该超过30秒 + ctx, cancel := context.WithTimeout(ctx, time.Second*30) + defer cancel() + + sw := s.swWorker.FindByIDContexted(ctx, exec.PlanID(req.PlanID)) + if sw == nil { + return nil, rpc.Failed(errorcode.DataNotFound, "plan not found") + } + + fut := future.NewSetVoid() + + varID := exec.VarID(req.VarID) + sw.PutVar(varID, &exec.StreamValue{Stream: io2.DelegateReadCloser(req.Stream, func() error { + fut.SetVoid() + return nil + })}) + + err := fut.Wait(ctx) + if err != nil { + return nil, rpc.Failed(errorcode.OperationFailed, "%v", err) + } + + return &hubrpc.SendIOStreamResp{}, nil +} + +func (s *Service) GetIOStream(ctx context.Context, req *hubrpc.GetIOStream) (*hubrpc.GetIOStreamResp, *rpc.CodeError) { + logger. + WithField("PlanID", req.PlanID). + WithField("VarID", req.VarID). + Debugf("stream output") + + // 同上 + ctx, cancel := context.WithTimeout(ctx, time.Second*30) + defer cancel() + + sw := s.swWorker.FindByIDContexted(ctx, exec.PlanID(req.PlanID)) + if sw == nil { + return nil, rpc.Failed(errorcode.DataNotFound, "plan not found") + } + + sw.PutVar(req.SignalID, req.Signal) + + strVar, err := exec.BindVar[*exec.StreamValue](sw, ctx, exec.VarID(req.VarID)) + if err != nil { + return nil, rpc.Failed(errorcode.OperationFailed, "bind var: %v", err) + } + + return &hubrpc.GetIOStreamResp{ + Stream: strVar.Stream, + }, nil +} + +func (s *Service) SendIOVar(ctx context.Context, req *hubrpc.SendIOVar) (*hubrpc.SendIOVarResp, *rpc.CodeError) { + ctx, cancel := context.WithTimeout(ctx, time.Second*30) + defer cancel() + + sw := s.swWorker.FindByIDContexted(ctx, exec.PlanID(req.PlanID)) + if sw == nil { + return nil, rpc.Failed(errorcode.DataNotFound, "plan not found") + } + + sw.PutVar(req.VarID, req.Value) + return &hubrpc.SendIOVarResp{}, nil +} + +func (s *Service) GetIOVar(ctx context.Context, req *hubrpc.GetIOVar) (*hubrpc.GetIOVarResp, *rpc.CodeError) { + ctx2, cancel := context.WithTimeout(ctx, time.Second*30) + defer cancel() + + sw := s.swWorker.FindByIDContexted(ctx2, exec.PlanID(req.PlanID)) + if sw == nil { + return nil, rpc.Failed(errorcode.DataNotFound, "plan not found") + } + + sw.PutVar(req.SignalID, req.Signal) + + v, err := sw.BindVar(ctx, exec.VarID(req.VarID)) + if err != nil { + return nil, rpc.Failed(errorcode.OperationFailed, "bind var: %v", err) + } + + return &hubrpc.GetIOVarResp{ + Value: v, + }, nil +} diff --git a/hub/internal/rpc/misc.go b/hub/internal/rpc/misc.go new file mode 100644 index 0000000..a77310d --- /dev/null +++ b/hub/internal/rpc/misc.go @@ -0,0 +1,16 @@ +package rpc + +import ( + "context" + + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" +) + +func (s *Service) Ping(context.Context, *hubrpc.Ping) (*hubrpc.PingResp, *rpc.CodeError) { + return &hubrpc.PingResp{}, nil +} + +func (svc *Service) GetState(context context.Context, msg *hubrpc.GetState) (*hubrpc.GetStateResp, *rpc.CodeError) { + return &hubrpc.GetStateResp{}, nil +} diff --git a/hub/internal/grpc/service.go b/hub/internal/rpc/rpc.go similarity index 74% rename from hub/internal/grpc/service.go rename to hub/internal/rpc/rpc.go index 64bf4a7..72ede2d 100644 --- a/hub/internal/grpc/service.go +++ b/hub/internal/rpc/rpc.go @@ -1,13 +1,12 @@ -package grpc +package rpc import ( "gitlink.org.cn/cloudream/common/pkgs/ioswitch/exec" - hubserver "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/grpc/hub" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/storage/pool" ) type Service struct { - hubserver.HubServer swWorker *exec.Worker stgPool *pool.Pool } @@ -18,3 +17,5 @@ func NewService(swWorker *exec.Worker, stgPool *pool.Pool) *Service { stgPool: stgPool, } } + +var _ hubrpc.HubAPI = (*Service)(nil) diff --git a/hub/internal/rpc/user_space.go b/hub/internal/rpc/user_space.go new file mode 100644 index 0000000..021a95c --- /dev/null +++ b/hub/internal/rpc/user_space.go @@ -0,0 +1,47 @@ +package rpc + +import ( + "context" + + "gitlink.org.cn/cloudream/common/consts/errorcode" + "gitlink.org.cn/cloudream/common/pkgs/logger" + "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc" + hubrpc "gitlink.org.cn/cloudream/jcs-pub/common/pkgs/rpc/hub" +) + +func (svc *Service) PublicStoreListAll(context context.Context, msg *hubrpc.PublicStoreListAll) (*hubrpc.PublicStoreListAllResp, *rpc.CodeError) { + pub, err := svc.stgPool.GetPublicStore(&msg.UserSpace) + if err != nil { + return nil, rpc.Failed(errorcode.OperationFailed, err.Error()) + } + + es, err := pub.ListAll(msg.Path) + if err != nil { + return nil, rpc.Failed(errorcode.OperationFailed, err.Error()) + } + + return &hubrpc.PublicStoreListAllResp{ + Entries: es, + }, nil +} + +func (svc *Service) PublicStoreMkdirs(context context.Context, msg *hubrpc.PublicStoreMkdirs) (*hubrpc.PublicStoreMkdirsResp, *rpc.CodeError) { + pub, err := svc.stgPool.GetPublicStore(&msg.UserSpace) + if err != nil { + return nil, rpc.Failed(errorcode.OperationFailed, err.Error()) + } + + var suc []bool + for _, p := range msg.Pathes { + if err := pub.Mkdir(p); err != nil { + suc = append(suc, false) + logger.Warnf("userspace %v mkdir %s: %v", msg.UserSpace, p, err) + } else { + suc = append(suc, true) + } + } + + return &hubrpc.PublicStoreMkdirsResp{ + Successes: suc, + }, nil +}