mirror of
https://github.com/crawlab-team/crawlab.git
synced 2026-01-21 17:21:09 +01:00
feat(sync): add gRPC file synchronization service and integrate end-to-end
- add proto/services/sync_service.proto and generate Go pb + grpc bindings - implement SyncServiceServer (streaming file scan + download) with: - request deduplication, in-memory cache (TTL), chunked streaming - concurrent-safe broadcast to waiters and server-side logging - register SyncSvr in gRPC server and expose sync client in GrpcClient: - add syncClient field, registration and safe getters with reconnection-aware timeouts - integrate gRPC sync into runner: - split syncFiles into syncFilesHTTP (legacy) and syncFilesGRPC - Runner now chooses implementation via config flag and performs streaming scan/download - controller improvements: - add semaphore-based rate limiting for sync scan requests with in-flight counters and logs - misc: - add utils.IsSyncGrpcEnabled() config helper - improve HTTP sync error diagnostics (Content-Type validation, response previews) - update/regenerate many protobuf and gRPC generated files (protoc/protoc-gen-go / protoc-gen-go-grpc version bumps)
This commit is contained in:
@@ -15,9 +15,29 @@ import (
|
||||
var (
|
||||
syncDownloadSemaphore = semaphore.NewWeighted(utils.GetSyncDownloadMaxConcurrency())
|
||||
syncDownloadInFlight int64
|
||||
syncScanSemaphore = semaphore.NewWeighted(10) // Limit concurrent scan requests
|
||||
syncScanInFlight int64
|
||||
)
|
||||
|
||||
func GetSyncScan(c *gin.Context) (response *Response[entity.FsFileInfoMap], err error) {
|
||||
ctx := c.Request.Context()
|
||||
if ctx == nil {
|
||||
ctx = context.Background()
|
||||
}
|
||||
|
||||
// Rate limiting for scan requests
|
||||
if err := syncScanSemaphore.Acquire(ctx, 1); err != nil {
|
||||
logger.Warnf("failed to acquire sync scan slot for id=%s path=%s: %v", c.Param("id"), c.Param("path"), err)
|
||||
return GetErrorResponse[entity.FsFileInfoMap](errors.Annotate(err, "server overloaded, please retry"))
|
||||
}
|
||||
current := atomic.AddInt64(&syncScanInFlight, 1)
|
||||
logger.Debugf("sync scan in-flight=%d id=%s path=%s", current, c.Param("id"), c.Param("path"))
|
||||
defer func() {
|
||||
newVal := atomic.AddInt64(&syncScanInFlight, -1)
|
||||
logger.Debugf("sync scan completed in-flight=%d id=%s path=%s", newVal, c.Param("id"), c.Param("path"))
|
||||
syncScanSemaphore.Release(1)
|
||||
}()
|
||||
|
||||
workspacePath := utils.GetWorkspace()
|
||||
dirPath := filepath.Join(workspacePath, c.Param("id"), c.Param("path"))
|
||||
files, err := utils.ScanDirectory(dirPath)
|
||||
|
||||
@@ -90,6 +90,7 @@ type GrpcClient struct {
|
||||
modelBaseServiceClient grpc2.ModelBaseServiceClient
|
||||
dependencyClient grpc2.DependencyServiceClient
|
||||
metricClient grpc2.MetricServiceClient
|
||||
syncClient grpc2.SyncServiceClient
|
||||
|
||||
// Add new fields for state management
|
||||
state connectivity.State
|
||||
@@ -218,6 +219,7 @@ func (c *GrpcClient) register() {
|
||||
c.taskClient = grpc2.NewTaskServiceClient(c.conn)
|
||||
c.dependencyClient = grpc2.NewDependencyServiceClient(c.conn)
|
||||
c.metricClient = grpc2.NewMetricServiceClient(c.conn)
|
||||
c.syncClient = grpc2.NewSyncServiceClient(c.conn)
|
||||
c.healthClient = grpc_health_v1.NewHealthClient(c.conn)
|
||||
|
||||
// Enable health checks by default for new connections
|
||||
@@ -498,6 +500,21 @@ func (c *GrpcClient) GetMetricClient() (grpc2.MetricServiceClient, error) {
|
||||
return c.GetMetricClientWithContext(ctx)
|
||||
}
|
||||
|
||||
func (c *GrpcClient) GetSyncClient() (grpc2.SyncServiceClient, error) {
|
||||
// Use longer timeout during reconnection scenarios
|
||||
timeout := defaultClientTimeout
|
||||
c.reconnectMux.Lock()
|
||||
if c.reconnecting {
|
||||
timeout = reconnectionClientTimeout
|
||||
}
|
||||
c.reconnectMux.Unlock()
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), timeout)
|
||||
defer cancel()
|
||||
|
||||
return c.GetSyncClientWithContext(ctx)
|
||||
}
|
||||
|
||||
// Safe client getters with timeout - these methods will wait up to the specified timeout
|
||||
// for registration to complete before returning an error
|
||||
|
||||
@@ -648,6 +665,14 @@ func (c *GrpcClient) GetMetricClientWithContext(ctx context.Context) (grpc2.Metr
|
||||
return client.(grpc2.MetricServiceClient), nil
|
||||
}
|
||||
|
||||
func (c *GrpcClient) GetSyncClientWithContext(ctx context.Context) (grpc2.SyncServiceClient, error) {
|
||||
client, err := c.getClientWithContext(ctx, func() interface{} { return c.syncClient }, "sync")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return client.(grpc2.SyncServiceClient), nil
|
||||
}
|
||||
|
||||
func (c *GrpcClient) getClientWithContext(ctx context.Context, getter func() interface{}, clientType string) (interface{}, error) {
|
||||
if c.stopped {
|
||||
return nil, fmt.Errorf("grpc client is stopped")
|
||||
|
||||
@@ -32,6 +32,7 @@ type GrpcServer struct {
|
||||
ModelBaseServiceSvr *ModelBaseServiceServer
|
||||
DependencySvr *DependencyServiceServer
|
||||
MetricSvr *MetricServiceServer
|
||||
SyncSvr *SyncServiceServer
|
||||
}
|
||||
|
||||
func (svr *GrpcServer) Init() {
|
||||
@@ -89,6 +90,7 @@ func (svr *GrpcServer) register() {
|
||||
grpc2.RegisterTaskServiceServer(svr.svr, svr.TaskSvr)
|
||||
grpc2.RegisterDependencyServiceServer(svr.svr, svr.DependencySvr)
|
||||
grpc2.RegisterMetricServiceServer(svr.svr, svr.MetricSvr)
|
||||
grpc2.RegisterSyncServiceServer(svr.svr, svr.SyncSvr)
|
||||
}
|
||||
|
||||
func (svr *GrpcServer) recoveryHandlerFunc(p interface{}) (err error) {
|
||||
@@ -109,6 +111,7 @@ func newGrpcServer() *GrpcServer {
|
||||
svr.TaskSvr = GetTaskServiceServer()
|
||||
svr.DependencySvr = GetDependencyServer()
|
||||
svr.MetricSvr = GetMetricsServer()
|
||||
svr.SyncSvr = GetSyncServiceServer()
|
||||
|
||||
// recovery options
|
||||
recoveryOpts := []grpcrecovery.Option{
|
||||
|
||||
296
core/grpc/server/sync_service_server.go
Normal file
296
core/grpc/server/sync_service_server.go
Normal file
@@ -0,0 +1,296 @@
|
||||
package server
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/crawlab-team/crawlab/core/utils"
|
||||
grpc2 "github.com/crawlab-team/crawlab/grpc"
|
||||
)
|
||||
|
||||
type SyncServiceServer struct {
|
||||
grpc2.UnimplementedSyncServiceServer
|
||||
*utils.Logger
|
||||
|
||||
// Request deduplication: key = spider_id:path
|
||||
activeScans map[string]*activeScanState
|
||||
activeScansMu sync.RWMutex
|
||||
|
||||
// Cache: avoid rescanning within TTL
|
||||
scanCache map[string]*cachedScanResult
|
||||
scanCacheMu sync.RWMutex
|
||||
scanCacheTTL time.Duration
|
||||
chunkSize int
|
||||
}
|
||||
|
||||
type activeScanState struct {
|
||||
inProgress bool
|
||||
waitChan chan *cachedScanResult // Broadcast to waiting requests
|
||||
subscribers int
|
||||
}
|
||||
|
||||
type cachedScanResult struct {
|
||||
files []*grpc2.FileInfo
|
||||
timestamp time.Time
|
||||
err error
|
||||
}
|
||||
|
||||
func NewSyncServiceServer() *SyncServiceServer {
|
||||
return &SyncServiceServer{
|
||||
Logger: utils.NewLogger("SyncServiceServer"),
|
||||
activeScans: make(map[string]*activeScanState),
|
||||
scanCache: make(map[string]*cachedScanResult),
|
||||
scanCacheTTL: 60 * time.Second, // Longer TTL for streaming
|
||||
chunkSize: 100, // Files per chunk
|
||||
}
|
||||
}
|
||||
|
||||
// StreamFileScan streams file information to worker
|
||||
func (s *SyncServiceServer) StreamFileScan(
|
||||
req *grpc2.FileSyncRequest,
|
||||
stream grpc2.SyncService_StreamFileScanServer,
|
||||
) error {
|
||||
cacheKey := req.SpiderId + ":" + req.Path
|
||||
|
||||
s.Debugf("file scan request from node %s for spider %s, path %s", req.NodeKey, req.SpiderId, req.Path)
|
||||
|
||||
// Check cache first
|
||||
if result := s.getCachedScan(cacheKey); result != nil {
|
||||
s.Debugf("returning cached scan for %s", cacheKey)
|
||||
return s.streamCachedResult(stream, result)
|
||||
}
|
||||
|
||||
// Deduplicate concurrent requests
|
||||
result, err := s.getOrWaitForScan(cacheKey, func() (*cachedScanResult, error) {
|
||||
return s.performScan(req)
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
s.Errorf("scan failed for %s: %v", cacheKey, err)
|
||||
return stream.Send(&grpc2.FileScanChunk{
|
||||
IsComplete: true,
|
||||
Error: err.Error(),
|
||||
})
|
||||
}
|
||||
|
||||
return s.streamCachedResult(stream, result)
|
||||
}
|
||||
|
||||
// performScan does the actual directory scan
|
||||
func (s *SyncServiceServer) performScan(req *grpc2.FileSyncRequest) (*cachedScanResult, error) {
|
||||
workspacePath := utils.GetWorkspace()
|
||||
dirPath := filepath.Join(workspacePath, req.SpiderId, req.Path)
|
||||
|
||||
s.Infof("performing directory scan for %s", dirPath)
|
||||
|
||||
// Use existing ScanDirectory which has singleflight and short-term cache
|
||||
fileMap, err := utils.ScanDirectory(dirPath)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to scan directory: %w", err)
|
||||
}
|
||||
|
||||
// Convert to protobuf format
|
||||
files := make([]*grpc2.FileInfo, 0, len(fileMap))
|
||||
for _, f := range fileMap {
|
||||
files = append(files, &grpc2.FileInfo{
|
||||
Name: f.Name,
|
||||
Path: f.Path,
|
||||
FullPath: f.FullPath,
|
||||
Extension: f.Extension,
|
||||
IsDir: f.IsDir,
|
||||
FileSize: f.FileSize,
|
||||
ModTime: f.ModTime.Unix(),
|
||||
Mode: uint32(f.Mode),
|
||||
Hash: f.Hash,
|
||||
})
|
||||
}
|
||||
|
||||
result := &cachedScanResult{
|
||||
files: files,
|
||||
timestamp: time.Now(),
|
||||
}
|
||||
|
||||
// Cache the result
|
||||
cacheKey := req.SpiderId + ":" + req.Path
|
||||
s.scanCacheMu.Lock()
|
||||
s.scanCache[cacheKey] = result
|
||||
s.scanCacheMu.Unlock()
|
||||
|
||||
s.Infof("scanned %d files from %s", len(files), dirPath)
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// streamCachedResult streams the cached result in chunks
|
||||
func (s *SyncServiceServer) streamCachedResult(
|
||||
stream grpc2.SyncService_StreamFileScanServer,
|
||||
result *cachedScanResult,
|
||||
) error {
|
||||
totalFiles := len(result.files)
|
||||
|
||||
for i := 0; i < totalFiles; i += s.chunkSize {
|
||||
end := i + s.chunkSize
|
||||
if end > totalFiles {
|
||||
end = totalFiles
|
||||
}
|
||||
|
||||
chunk := &grpc2.FileScanChunk{
|
||||
Files: result.files[i:end],
|
||||
IsComplete: end >= totalFiles,
|
||||
TotalFiles: int32(totalFiles),
|
||||
}
|
||||
|
||||
if err := stream.Send(chunk); err != nil {
|
||||
return fmt.Errorf("failed to send chunk: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// getOrWaitForScan implements request deduplication
|
||||
func (s *SyncServiceServer) getOrWaitForScan(
|
||||
key string,
|
||||
scanFunc func() (*cachedScanResult, error),
|
||||
) (*cachedScanResult, error) {
|
||||
s.activeScansMu.Lock()
|
||||
|
||||
state, exists := s.activeScans[key]
|
||||
if exists && state.inProgress {
|
||||
// Another request is already scanning, wait for it
|
||||
state.subscribers++
|
||||
waitChan := state.waitChan
|
||||
s.activeScansMu.Unlock()
|
||||
|
||||
s.Debugf("waiting for ongoing scan: %s", key)
|
||||
result := <-waitChan
|
||||
return result, result.err
|
||||
}
|
||||
|
||||
// We're the first request, start scanning
|
||||
state = &activeScanState{
|
||||
inProgress: true,
|
||||
waitChan: make(chan *cachedScanResult, 10),
|
||||
subscribers: 0,
|
||||
}
|
||||
s.activeScans[key] = state
|
||||
s.activeScansMu.Unlock()
|
||||
|
||||
s.Debugf("initiating new scan: %s", key)
|
||||
|
||||
// Perform scan
|
||||
result, err := scanFunc()
|
||||
if err != nil {
|
||||
result = &cachedScanResult{err: err}
|
||||
}
|
||||
|
||||
// Broadcast to waiting requests
|
||||
s.activeScansMu.Lock()
|
||||
for i := 0; i < state.subscribers; i++ {
|
||||
state.waitChan <- result
|
||||
}
|
||||
delete(s.activeScans, key)
|
||||
close(state.waitChan)
|
||||
s.activeScansMu.Unlock()
|
||||
|
||||
s.Debugf("scan complete for %s, notified %d subscribers", key, state.subscribers)
|
||||
|
||||
return result, err
|
||||
}
|
||||
|
||||
func (s *SyncServiceServer) getCachedScan(key string) *cachedScanResult {
|
||||
s.scanCacheMu.RLock()
|
||||
defer s.scanCacheMu.RUnlock()
|
||||
|
||||
result, exists := s.scanCache[key]
|
||||
if !exists {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Check if cache expired
|
||||
if time.Since(result.timestamp) > s.scanCacheTTL {
|
||||
return nil
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
// StreamFileDownload streams file content to worker
|
||||
func (s *SyncServiceServer) StreamFileDownload(
|
||||
req *grpc2.FileDownloadRequest,
|
||||
stream grpc2.SyncService_StreamFileDownloadServer,
|
||||
) error {
|
||||
workspacePath := utils.GetWorkspace()
|
||||
filePath := filepath.Join(workspacePath, req.SpiderId, req.Path)
|
||||
|
||||
s.Infof("streaming file download: %s", filePath)
|
||||
|
||||
// Open file
|
||||
file, err := os.Open(filePath)
|
||||
if err != nil {
|
||||
return stream.Send(&grpc2.FileDownloadChunk{
|
||||
IsComplete: true,
|
||||
Error: fmt.Sprintf("failed to open file: %v", err),
|
||||
})
|
||||
}
|
||||
defer file.Close()
|
||||
|
||||
// Get file size
|
||||
fileInfo, err := file.Stat()
|
||||
if err != nil {
|
||||
return stream.Send(&grpc2.FileDownloadChunk{
|
||||
IsComplete: true,
|
||||
Error: fmt.Sprintf("failed to stat file: %v", err),
|
||||
})
|
||||
}
|
||||
|
||||
// Stream file in chunks
|
||||
const bufferSize = 64 * 1024 // 64KB chunks
|
||||
buffer := make([]byte, bufferSize)
|
||||
totalBytes := fileInfo.Size()
|
||||
bytesSent := int64(0)
|
||||
|
||||
for {
|
||||
n, err := file.Read(buffer)
|
||||
if n > 0 {
|
||||
chunk := &grpc2.FileDownloadChunk{
|
||||
Data: buffer[:n],
|
||||
IsComplete: false,
|
||||
TotalBytes: totalBytes,
|
||||
}
|
||||
|
||||
if err := stream.Send(chunk); err != nil {
|
||||
return fmt.Errorf("failed to send chunk: %w", err)
|
||||
}
|
||||
|
||||
bytesSent += int64(n)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
if err.Error() == "EOF" {
|
||||
// Send final chunk
|
||||
return stream.Send(&grpc2.FileDownloadChunk{
|
||||
IsComplete: true,
|
||||
TotalBytes: totalBytes,
|
||||
})
|
||||
}
|
||||
return stream.Send(&grpc2.FileDownloadChunk{
|
||||
IsComplete: true,
|
||||
Error: fmt.Sprintf("read error: %v", err),
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
var _syncServiceServer *SyncServiceServer
|
||||
var _syncServiceServerOnce sync.Once
|
||||
|
||||
func GetSyncServiceServer() *SyncServiceServer {
|
||||
_syncServiceServerOnce.Do(func() {
|
||||
_syncServiceServer = NewSyncServiceServer()
|
||||
})
|
||||
return _syncServiceServer
|
||||
}
|
||||
@@ -31,13 +31,22 @@ var (
|
||||
jitterMutex sync.Mutex
|
||||
)
|
||||
|
||||
// syncFiles synchronizes files between master and worker nodes:
|
||||
// syncFiles synchronizes files between master and worker nodes.
|
||||
// It switches between gRPC streaming and HTTP based on the feature flag.
|
||||
func (r *Runner) syncFiles() (err error) {
|
||||
if utils.IsSyncGrpcEnabled() {
|
||||
return r.syncFilesGRPC()
|
||||
}
|
||||
return r.syncFilesHTTP()
|
||||
}
|
||||
|
||||
// syncFilesHTTP synchronizes files using HTTP/JSON (legacy implementation):
|
||||
// 1. Gets file list from master
|
||||
// 2. Compares with local files
|
||||
// 3. Downloads new/modified files
|
||||
// 4. Deletes files that no longer exist on master
|
||||
func (r *Runner) syncFiles() (err error) {
|
||||
r.Infof("starting file synchronization for spider: %s", r.s.Id.Hex())
|
||||
func (r *Runner) syncFilesHTTP() (err error) {
|
||||
r.Infof("starting HTTP file synchronization for spider: %s", r.s.Id.Hex())
|
||||
|
||||
workingDir := ""
|
||||
if !r.s.GitId.IsZero() {
|
||||
@@ -56,6 +65,21 @@ func (r *Runner) syncFiles() (err error) {
|
||||
return err
|
||||
}
|
||||
defer resp.Body.Close()
|
||||
|
||||
// Validate Content-Type to detect non-JSON responses early
|
||||
contentType := resp.Header.Get("Content-Type")
|
||||
if !strings.Contains(contentType, "application/json") {
|
||||
r.Errorf("unexpected Content-Type: %s (expected application/json)", contentType)
|
||||
r.Errorf("URL: %s, Status: %d", resp.Request.URL.String(), resp.StatusCode)
|
||||
body, _ := io.ReadAll(resp.Body)
|
||||
if len(body) > 500 {
|
||||
r.Errorf("Response preview: %s...", string(body[:500]))
|
||||
} else {
|
||||
r.Errorf("Response body: %s", string(body))
|
||||
}
|
||||
return fmt.Errorf("master returned non-JSON response (Content-Type: %s, Status: %d)", contentType, resp.StatusCode)
|
||||
}
|
||||
|
||||
body, err := io.ReadAll(resp.Body)
|
||||
if err != nil {
|
||||
r.Errorf("error reading response body: %v", err)
|
||||
@@ -68,6 +92,12 @@ func (r *Runner) syncFiles() (err error) {
|
||||
if err != nil {
|
||||
r.Errorf("error unmarshaling JSON for URL: %s", resp.Request.URL.String())
|
||||
r.Errorf("error details: %v", err)
|
||||
r.Errorf("response body length: %d bytes", len(body))
|
||||
if len(body) > 500 {
|
||||
r.Errorf("response preview: %s...", string(body[:500]))
|
||||
} else if len(body) > 0 {
|
||||
r.Errorf("response body: %s", string(body))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
|
||||
231
core/task/handler/runner_sync_grpc.go
Normal file
231
core/task/handler/runner_sync_grpc.go
Normal file
@@ -0,0 +1,231 @@
|
||||
package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"time"
|
||||
|
||||
"github.com/crawlab-team/crawlab/core/entity"
|
||||
client2 "github.com/crawlab-team/crawlab/core/grpc/client"
|
||||
"github.com/crawlab-team/crawlab/core/utils"
|
||||
grpc2 "github.com/crawlab-team/crawlab/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
// syncFilesGRPC replaces HTTP-based syncFiles() with gRPC streaming
|
||||
func (r *Runner) syncFilesGRPC() (err error) {
|
||||
r.Infof("starting gRPC file synchronization for spider: %s", r.s.Id.Hex())
|
||||
|
||||
workingDir := ""
|
||||
if !r.s.GitId.IsZero() {
|
||||
workingDir = r.s.GitRootPath
|
||||
r.Debugf("using git root path: %s", workingDir)
|
||||
}
|
||||
|
||||
// Get sync service client
|
||||
syncClient, err := client2.GetGrpcClient().GetSyncClient()
|
||||
if err != nil {
|
||||
r.Errorf("failed to get sync client: %v", err)
|
||||
return err
|
||||
}
|
||||
|
||||
// Prepare request
|
||||
req := &grpc2.FileSyncRequest{
|
||||
SpiderId: r.s.Id.Hex(),
|
||||
Path: workingDir,
|
||||
NodeKey: utils.GetNodeKey(),
|
||||
}
|
||||
|
||||
// Create context with timeout
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
// Stream file list from master
|
||||
r.Infof("fetching file list from master via gRPC")
|
||||
stream, err := syncClient.StreamFileScan(ctx, req)
|
||||
if err != nil {
|
||||
r.Errorf("failed to start file scan stream: %v", err)
|
||||
return err
|
||||
}
|
||||
|
||||
// Receive file list in chunks
|
||||
masterFilesMap := make(entity.FsFileInfoMap)
|
||||
totalFiles := 0
|
||||
|
||||
for {
|
||||
chunk, err := stream.Recv()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
// Check for gRPC-specific errors
|
||||
if st, ok := status.FromError(err); ok {
|
||||
if st.Code() == codes.Unavailable {
|
||||
r.Errorf("gRPC service unavailable, will retry: %v", err)
|
||||
return fmt.Errorf("gRPC service unavailable: %w", err)
|
||||
}
|
||||
}
|
||||
r.Errorf("error receiving file scan chunk: %v", err)
|
||||
return err
|
||||
}
|
||||
|
||||
// Check for error in chunk
|
||||
if chunk.Error != "" {
|
||||
r.Errorf("server error during file scan: %s", chunk.Error)
|
||||
return fmt.Errorf("server error: %s", chunk.Error)
|
||||
}
|
||||
|
||||
// Process files in chunk
|
||||
for _, fileInfo := range chunk.Files {
|
||||
fsFileInfo := entity.FsFileInfo{
|
||||
Name: fileInfo.Name,
|
||||
Path: fileInfo.Path,
|
||||
FullPath: fileInfo.FullPath,
|
||||
Extension: fileInfo.Extension,
|
||||
IsDir: fileInfo.IsDir,
|
||||
FileSize: fileInfo.FileSize,
|
||||
ModTime: time.Unix(fileInfo.ModTime, 0),
|
||||
Mode: os.FileMode(fileInfo.Mode),
|
||||
Hash: fileInfo.Hash,
|
||||
}
|
||||
masterFilesMap[fileInfo.Path] = fsFileInfo
|
||||
}
|
||||
|
||||
if chunk.IsComplete {
|
||||
totalFiles = int(chunk.TotalFiles)
|
||||
r.Infof("received complete file list: %d files", totalFiles)
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// Create working directory if not exists
|
||||
if _, err := os.Stat(r.cwd); os.IsNotExist(err) {
|
||||
if err := os.MkdirAll(r.cwd, os.ModePerm); err != nil {
|
||||
r.Errorf("error creating worker directory: %v", err)
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Get file list from worker
|
||||
workerFiles, err := utils.ScanDirectory(r.cwd)
|
||||
if err != nil {
|
||||
r.Errorf("error scanning worker directory: %v", err)
|
||||
return err
|
||||
}
|
||||
|
||||
// Delete files that are deleted on master node
|
||||
for path, workerFile := range workerFiles {
|
||||
if _, exists := masterFilesMap[path]; !exists {
|
||||
r.Infof("deleting file: %s", path)
|
||||
err := os.Remove(workerFile.FullPath)
|
||||
if err != nil {
|
||||
r.Errorf("error deleting file: %v", err)
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Download new or modified files
|
||||
downloadCount := 0
|
||||
for path, masterFile := range masterFilesMap {
|
||||
// Skip directories
|
||||
if masterFile.IsDir {
|
||||
continue
|
||||
}
|
||||
|
||||
workerFile, exists := workerFiles[path]
|
||||
needsDownload := false
|
||||
|
||||
if !exists {
|
||||
r.Debugf("file not found locally: %s", path)
|
||||
needsDownload = true
|
||||
} else if workerFile.Hash != masterFile.Hash {
|
||||
r.Debugf("file hash mismatch: %s (local: %s, master: %s)", path, workerFile.Hash, masterFile.Hash)
|
||||
needsDownload = true
|
||||
}
|
||||
|
||||
if needsDownload {
|
||||
if err := r.downloadFileGRPC(syncClient, r.s.Id.Hex(), path); err != nil {
|
||||
r.Errorf("error downloading file %s: %v", path, err)
|
||||
return err
|
||||
}
|
||||
downloadCount++
|
||||
}
|
||||
}
|
||||
|
||||
r.Infof("file synchronization complete: %d files downloaded", downloadCount)
|
||||
return nil
|
||||
}
|
||||
|
||||
// downloadFileGRPC downloads a single file from master via gRPC streaming
|
||||
func (r *Runner) downloadFileGRPC(client grpc2.SyncServiceClient, spiderId, path string) error {
|
||||
r.Debugf("downloading file via gRPC: %s", path)
|
||||
|
||||
// Prepare request
|
||||
req := &grpc2.FileDownloadRequest{
|
||||
SpiderId: spiderId,
|
||||
Path: path,
|
||||
NodeKey: utils.GetNodeKey(),
|
||||
}
|
||||
|
||||
// Create context with timeout
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
// Stream file download
|
||||
stream, err := client.StreamFileDownload(ctx, req)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to start download stream: %w", err)
|
||||
}
|
||||
|
||||
// Create target file
|
||||
targetPath := fmt.Sprintf("%s/%s", r.cwd, path)
|
||||
|
||||
// Create directory if not exists
|
||||
targetDir := targetPath[:len(targetPath)-len(path)]
|
||||
if err := os.MkdirAll(targetDir, os.ModePerm); err != nil {
|
||||
return fmt.Errorf("failed to create directory: %w", err)
|
||||
}
|
||||
|
||||
file, err := os.Create(targetPath)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create file: %w", err)
|
||||
}
|
||||
defer file.Close()
|
||||
|
||||
// Receive file content in chunks
|
||||
bytesReceived := int64(0)
|
||||
for {
|
||||
chunk, err := stream.Recv()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return fmt.Errorf("error receiving download chunk: %w", err)
|
||||
}
|
||||
|
||||
// Check for error in chunk
|
||||
if chunk.Error != "" {
|
||||
return fmt.Errorf("server error during download: %s", chunk.Error)
|
||||
}
|
||||
|
||||
// Write chunk data
|
||||
if len(chunk.Data) > 0 {
|
||||
n, err := file.Write(chunk.Data)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error writing file: %w", err)
|
||||
}
|
||||
bytesReceived += int64(n)
|
||||
}
|
||||
|
||||
if chunk.IsComplete {
|
||||
r.Debugf("download complete: %s (%d bytes)", path, bytesReceived)
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
@@ -348,3 +348,7 @@ func GetMinFileDescriptorLimit() uint64 {
|
||||
}
|
||||
return DefaultMinFileDescriptorLimit
|
||||
}
|
||||
|
||||
func IsSyncGrpcEnabled() bool {
|
||||
return viper.GetBool("sync.useGrpc")
|
||||
}
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.27.2
|
||||
// protoc-gen-go v1.36.1
|
||||
// protoc v5.29.2
|
||||
// source: services/dependency_service.proto
|
||||
|
||||
package grpc
|
||||
@@ -76,21 +76,18 @@ func (DependencyServiceCode) EnumDescriptor() ([]byte, []int) {
|
||||
}
|
||||
|
||||
type Dependency struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
|
||||
Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
|
||||
Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *Dependency) Reset() {
|
||||
*x = Dependency{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_dependency_service_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *Dependency) String() string {
|
||||
@@ -101,7 +98,7 @@ func (*Dependency) ProtoMessage() {}
|
||||
|
||||
func (x *Dependency) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[0]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -131,20 +128,17 @@ func (x *Dependency) GetVersion() string {
|
||||
}
|
||||
|
||||
type DependencyServiceConnectRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *DependencyServiceConnectRequest) Reset() {
|
||||
*x = DependencyServiceConnectRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_dependency_service_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *DependencyServiceConnectRequest) String() string {
|
||||
@@ -155,7 +149,7 @@ func (*DependencyServiceConnectRequest) ProtoMessage() {}
|
||||
|
||||
func (x *DependencyServiceConnectRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[1]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -178,24 +172,21 @@ func (x *DependencyServiceConnectRequest) GetNodeKey() string {
|
||||
}
|
||||
|
||||
type DependencyServiceConnectResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Code DependencyServiceCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.DependencyServiceCode" json:"code,omitempty"`
|
||||
Lang string `protobuf:"bytes,2,opt,name=lang,proto3" json:"lang,omitempty"`
|
||||
Proxy string `protobuf:"bytes,3,opt,name=proxy,proto3" json:"proxy,omitempty"`
|
||||
Dependency *Dependency `protobuf:"bytes,4,opt,name=dependency,proto3" json:"dependency,omitempty"`
|
||||
Version string `protobuf:"bytes,5,opt,name=version,proto3" json:"version,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Code DependencyServiceCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.DependencyServiceCode" json:"code,omitempty"`
|
||||
Lang string `protobuf:"bytes,2,opt,name=lang,proto3" json:"lang,omitempty"`
|
||||
Proxy string `protobuf:"bytes,3,opt,name=proxy,proto3" json:"proxy,omitempty"`
|
||||
Dependency *Dependency `protobuf:"bytes,4,opt,name=dependency,proto3" json:"dependency,omitempty"`
|
||||
Version string `protobuf:"bytes,5,opt,name=version,proto3" json:"version,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *DependencyServiceConnectResponse) Reset() {
|
||||
*x = DependencyServiceConnectResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_dependency_service_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *DependencyServiceConnectResponse) String() string {
|
||||
@@ -206,7 +197,7 @@ func (*DependencyServiceConnectResponse) ProtoMessage() {}
|
||||
|
||||
func (x *DependencyServiceConnectResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[2]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -257,22 +248,19 @@ func (x *DependencyServiceConnectResponse) GetVersion() string {
|
||||
}
|
||||
|
||||
type DependencyServiceSyncRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
Lang string `protobuf:"bytes,2,opt,name=lang,proto3" json:"lang,omitempty"`
|
||||
Dependencies []*Dependency `protobuf:"bytes,3,rep,name=dependencies,proto3" json:"dependencies,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
Lang string `protobuf:"bytes,2,opt,name=lang,proto3" json:"lang,omitempty"`
|
||||
Dependencies []*Dependency `protobuf:"bytes,3,rep,name=dependencies,proto3" json:"dependencies,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *DependencyServiceSyncRequest) Reset() {
|
||||
*x = DependencyServiceSyncRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[3]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_dependency_service_proto_msgTypes[3]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *DependencyServiceSyncRequest) String() string {
|
||||
@@ -283,7 +271,7 @@ func (*DependencyServiceSyncRequest) ProtoMessage() {}
|
||||
|
||||
func (x *DependencyServiceSyncRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[3]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -320,21 +308,18 @@ func (x *DependencyServiceSyncRequest) GetDependencies() []*Dependency {
|
||||
}
|
||||
|
||||
type DependencyServiceUpdateLogsRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
TargetId string `protobuf:"bytes,1,opt,name=target_id,json=targetId,proto3" json:"target_id,omitempty"`
|
||||
Logs []string `protobuf:"bytes,2,rep,name=logs,proto3" json:"logs,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
TargetId string `protobuf:"bytes,1,opt,name=target_id,json=targetId,proto3" json:"target_id,omitempty"`
|
||||
Logs []string `protobuf:"bytes,2,rep,name=logs,proto3" json:"logs,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *DependencyServiceUpdateLogsRequest) Reset() {
|
||||
*x = DependencyServiceUpdateLogsRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[4]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_dependency_service_proto_msgTypes[4]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *DependencyServiceUpdateLogsRequest) String() string {
|
||||
@@ -345,7 +330,7 @@ func (*DependencyServiceUpdateLogsRequest) ProtoMessage() {}
|
||||
|
||||
func (x *DependencyServiceUpdateLogsRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[4]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -375,21 +360,18 @@ func (x *DependencyServiceUpdateLogsRequest) GetLogs() []string {
|
||||
}
|
||||
|
||||
type DependencyDriver struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
|
||||
Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
|
||||
Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *DependencyDriver) Reset() {
|
||||
*x = DependencyDriver{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[5]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_dependency_service_proto_msgTypes[5]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *DependencyDriver) String() string {
|
||||
@@ -400,7 +382,7 @@ func (*DependencyDriver) ProtoMessage() {}
|
||||
|
||||
func (x *DependencyDriver) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[5]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -430,25 +412,22 @@ func (x *DependencyDriver) GetVersion() string {
|
||||
}
|
||||
|
||||
type DependencyServiceSyncConfigSetupRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
Lang string `protobuf:"bytes,2,opt,name=lang,proto3" json:"lang,omitempty"`
|
||||
Version string `protobuf:"bytes,3,opt,name=version,proto3" json:"version,omitempty"`
|
||||
Status string `protobuf:"bytes,4,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Error string `protobuf:"bytes,5,opt,name=error,proto3" json:"error,omitempty"`
|
||||
Drivers []*DependencyDriver `protobuf:"bytes,6,rep,name=drivers,proto3" json:"drivers,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
Lang string `protobuf:"bytes,2,opt,name=lang,proto3" json:"lang,omitempty"`
|
||||
Version string `protobuf:"bytes,3,opt,name=version,proto3" json:"version,omitempty"`
|
||||
Status string `protobuf:"bytes,4,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Error string `protobuf:"bytes,5,opt,name=error,proto3" json:"error,omitempty"`
|
||||
Drivers []*DependencyDriver `protobuf:"bytes,6,rep,name=drivers,proto3" json:"drivers,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *DependencyServiceSyncConfigSetupRequest) Reset() {
|
||||
*x = DependencyServiceSyncConfigSetupRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[6]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_dependency_service_proto_msgTypes[6]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *DependencyServiceSyncConfigSetupRequest) String() string {
|
||||
@@ -459,7 +438,7 @@ func (*DependencyServiceSyncConfigSetupRequest) ProtoMessage() {}
|
||||
|
||||
func (x *DependencyServiceSyncConfigSetupRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_dependency_service_proto_msgTypes[6]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -657,92 +636,6 @@ func file_services_dependency_service_proto_init() {
|
||||
return
|
||||
}
|
||||
file_entity_response_proto_init()
|
||||
if !protoimpl.UnsafeEnabled {
|
||||
file_services_dependency_service_proto_msgTypes[0].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*Dependency); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_dependency_service_proto_msgTypes[1].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*DependencyServiceConnectRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_dependency_service_proto_msgTypes[2].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*DependencyServiceConnectResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_dependency_service_proto_msgTypes[3].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*DependencyServiceSyncRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_dependency_service_proto_msgTypes[4].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*DependencyServiceUpdateLogsRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_dependency_service_proto_msgTypes[5].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*DependencyDriver); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_dependency_service_proto_msgTypes[6].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*DependencyServiceSyncConfigSetupRequest); 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{
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.4.0
|
||||
// - protoc v5.27.2
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.29.2
|
||||
// source: services/dependency_service.proto
|
||||
|
||||
package grpc
|
||||
@@ -15,8 +15,8 @@ import (
|
||||
|
||||
// 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.62.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion8
|
||||
// Requires gRPC-Go v1.64.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion9
|
||||
|
||||
const (
|
||||
DependencyService_Connect_FullMethodName = "/grpc.DependencyService/Connect"
|
||||
@@ -29,9 +29,9 @@ const (
|
||||
//
|
||||
// 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 DependencyServiceClient interface {
|
||||
Connect(ctx context.Context, in *DependencyServiceConnectRequest, opts ...grpc.CallOption) (DependencyService_ConnectClient, error)
|
||||
Connect(ctx context.Context, in *DependencyServiceConnectRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[DependencyServiceConnectResponse], error)
|
||||
Sync(ctx context.Context, in *DependencyServiceSyncRequest, opts ...grpc.CallOption) (*Response, error)
|
||||
UpdateLogs(ctx context.Context, opts ...grpc.CallOption) (DependencyService_UpdateLogsClient, error)
|
||||
UpdateLogs(ctx context.Context, opts ...grpc.CallOption) (grpc.ClientStreamingClient[DependencyServiceUpdateLogsRequest, Response], error)
|
||||
SyncConfigSetup(ctx context.Context, in *DependencyServiceSyncConfigSetupRequest, opts ...grpc.CallOption) (*Response, error)
|
||||
}
|
||||
|
||||
@@ -43,13 +43,13 @@ func NewDependencyServiceClient(cc grpc.ClientConnInterface) DependencyServiceCl
|
||||
return &dependencyServiceClient{cc}
|
||||
}
|
||||
|
||||
func (c *dependencyServiceClient) Connect(ctx context.Context, in *DependencyServiceConnectRequest, opts ...grpc.CallOption) (DependencyService_ConnectClient, error) {
|
||||
func (c *dependencyServiceClient) Connect(ctx context.Context, in *DependencyServiceConnectRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[DependencyServiceConnectResponse], error) {
|
||||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &DependencyService_ServiceDesc.Streams[0], DependencyService_Connect_FullMethodName, cOpts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &dependencyServiceConnectClient{ClientStream: stream}
|
||||
x := &grpc.GenericClientStream[DependencyServiceConnectRequest, DependencyServiceConnectResponse]{ClientStream: stream}
|
||||
if err := x.ClientStream.SendMsg(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -59,22 +59,8 @@ func (c *dependencyServiceClient) Connect(ctx context.Context, in *DependencySer
|
||||
return x, nil
|
||||
}
|
||||
|
||||
type DependencyService_ConnectClient interface {
|
||||
Recv() (*DependencyServiceConnectResponse, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
type dependencyServiceConnectClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *dependencyServiceConnectClient) Recv() (*DependencyServiceConnectResponse, error) {
|
||||
m := new(DependencyServiceConnectResponse)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type DependencyService_ConnectClient = grpc.ServerStreamingClient[DependencyServiceConnectResponse]
|
||||
|
||||
func (c *dependencyServiceClient) Sync(ctx context.Context, in *DependencyServiceSyncRequest, opts ...grpc.CallOption) (*Response, error) {
|
||||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||
@@ -86,40 +72,18 @@ func (c *dependencyServiceClient) Sync(ctx context.Context, in *DependencyServic
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dependencyServiceClient) UpdateLogs(ctx context.Context, opts ...grpc.CallOption) (DependencyService_UpdateLogsClient, error) {
|
||||
func (c *dependencyServiceClient) UpdateLogs(ctx context.Context, opts ...grpc.CallOption) (grpc.ClientStreamingClient[DependencyServiceUpdateLogsRequest, Response], error) {
|
||||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &DependencyService_ServiceDesc.Streams[1], DependencyService_UpdateLogs_FullMethodName, cOpts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &dependencyServiceUpdateLogsClient{ClientStream: stream}
|
||||
x := &grpc.GenericClientStream[DependencyServiceUpdateLogsRequest, Response]{ClientStream: stream}
|
||||
return x, nil
|
||||
}
|
||||
|
||||
type DependencyService_UpdateLogsClient interface {
|
||||
Send(*DependencyServiceUpdateLogsRequest) error
|
||||
CloseAndRecv() (*Response, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
type dependencyServiceUpdateLogsClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *dependencyServiceUpdateLogsClient) Send(m *DependencyServiceUpdateLogsRequest) error {
|
||||
return x.ClientStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *dependencyServiceUpdateLogsClient) CloseAndRecv() (*Response, error) {
|
||||
if err := x.ClientStream.CloseSend(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
m := new(Response)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type DependencyService_UpdateLogsClient = grpc.ClientStreamingClient[DependencyServiceUpdateLogsRequest, Response]
|
||||
|
||||
func (c *dependencyServiceClient) SyncConfigSetup(ctx context.Context, in *DependencyServiceSyncConfigSetupRequest, opts ...grpc.CallOption) (*Response, error) {
|
||||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||
@@ -133,32 +97,36 @@ func (c *dependencyServiceClient) SyncConfigSetup(ctx context.Context, in *Depen
|
||||
|
||||
// DependencyServiceServer is the server API for DependencyService service.
|
||||
// All implementations must embed UnimplementedDependencyServiceServer
|
||||
// for forward compatibility
|
||||
// for forward compatibility.
|
||||
type DependencyServiceServer interface {
|
||||
Connect(*DependencyServiceConnectRequest, DependencyService_ConnectServer) error
|
||||
Connect(*DependencyServiceConnectRequest, grpc.ServerStreamingServer[DependencyServiceConnectResponse]) error
|
||||
Sync(context.Context, *DependencyServiceSyncRequest) (*Response, error)
|
||||
UpdateLogs(DependencyService_UpdateLogsServer) error
|
||||
UpdateLogs(grpc.ClientStreamingServer[DependencyServiceUpdateLogsRequest, Response]) error
|
||||
SyncConfigSetup(context.Context, *DependencyServiceSyncConfigSetupRequest) (*Response, error)
|
||||
mustEmbedUnimplementedDependencyServiceServer()
|
||||
}
|
||||
|
||||
// UnimplementedDependencyServiceServer must be embedded to have forward compatible implementations.
|
||||
type UnimplementedDependencyServiceServer struct {
|
||||
}
|
||||
// UnimplementedDependencyServiceServer must be embedded to have
|
||||
// forward compatible implementations.
|
||||
//
|
||||
// NOTE: this should be embedded by value instead of pointer to avoid a nil
|
||||
// pointer dereference when methods are called.
|
||||
type UnimplementedDependencyServiceServer struct{}
|
||||
|
||||
func (UnimplementedDependencyServiceServer) Connect(*DependencyServiceConnectRequest, DependencyService_ConnectServer) error {
|
||||
func (UnimplementedDependencyServiceServer) Connect(*DependencyServiceConnectRequest, grpc.ServerStreamingServer[DependencyServiceConnectResponse]) error {
|
||||
return status.Errorf(codes.Unimplemented, "method Connect not implemented")
|
||||
}
|
||||
func (UnimplementedDependencyServiceServer) Sync(context.Context, *DependencyServiceSyncRequest) (*Response, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Sync not implemented")
|
||||
}
|
||||
func (UnimplementedDependencyServiceServer) UpdateLogs(DependencyService_UpdateLogsServer) error {
|
||||
func (UnimplementedDependencyServiceServer) UpdateLogs(grpc.ClientStreamingServer[DependencyServiceUpdateLogsRequest, Response]) error {
|
||||
return status.Errorf(codes.Unimplemented, "method UpdateLogs not implemented")
|
||||
}
|
||||
func (UnimplementedDependencyServiceServer) SyncConfigSetup(context.Context, *DependencyServiceSyncConfigSetupRequest) (*Response, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method SyncConfigSetup not implemented")
|
||||
}
|
||||
func (UnimplementedDependencyServiceServer) mustEmbedUnimplementedDependencyServiceServer() {}
|
||||
func (UnimplementedDependencyServiceServer) testEmbeddedByValue() {}
|
||||
|
||||
// UnsafeDependencyServiceServer may be embedded to opt out of forward compatibility for this service.
|
||||
// Use of this interface is not recommended, as added methods to DependencyServiceServer will
|
||||
@@ -168,6 +136,13 @@ type UnsafeDependencyServiceServer interface {
|
||||
}
|
||||
|
||||
func RegisterDependencyServiceServer(s grpc.ServiceRegistrar, srv DependencyServiceServer) {
|
||||
// If the following call pancis, it indicates UnimplementedDependencyServiceServer was
|
||||
// embedded by pointer and is nil. This will cause panics if an
|
||||
// unimplemented method is ever invoked, so we test this at initialization
|
||||
// time to prevent it from happening at runtime later due to I/O.
|
||||
if t, ok := srv.(interface{ testEmbeddedByValue() }); ok {
|
||||
t.testEmbeddedByValue()
|
||||
}
|
||||
s.RegisterService(&DependencyService_ServiceDesc, srv)
|
||||
}
|
||||
|
||||
@@ -176,21 +151,11 @@ func _DependencyService_Connect_Handler(srv interface{}, stream grpc.ServerStrea
|
||||
if err := stream.RecvMsg(m); err != nil {
|
||||
return err
|
||||
}
|
||||
return srv.(DependencyServiceServer).Connect(m, &dependencyServiceConnectServer{ServerStream: stream})
|
||||
return srv.(DependencyServiceServer).Connect(m, &grpc.GenericServerStream[DependencyServiceConnectRequest, DependencyServiceConnectResponse]{ServerStream: stream})
|
||||
}
|
||||
|
||||
type DependencyService_ConnectServer interface {
|
||||
Send(*DependencyServiceConnectResponse) error
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type dependencyServiceConnectServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *dependencyServiceConnectServer) Send(m *DependencyServiceConnectResponse) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type DependencyService_ConnectServer = grpc.ServerStreamingServer[DependencyServiceConnectResponse]
|
||||
|
||||
func _DependencyService_Sync_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(DependencyServiceSyncRequest)
|
||||
@@ -211,30 +176,11 @@ func _DependencyService_Sync_Handler(srv interface{}, ctx context.Context, dec f
|
||||
}
|
||||
|
||||
func _DependencyService_UpdateLogs_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
return srv.(DependencyServiceServer).UpdateLogs(&dependencyServiceUpdateLogsServer{ServerStream: stream})
|
||||
return srv.(DependencyServiceServer).UpdateLogs(&grpc.GenericServerStream[DependencyServiceUpdateLogsRequest, Response]{ServerStream: stream})
|
||||
}
|
||||
|
||||
type DependencyService_UpdateLogsServer interface {
|
||||
SendAndClose(*Response) error
|
||||
Recv() (*DependencyServiceUpdateLogsRequest, error)
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type dependencyServiceUpdateLogsServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *dependencyServiceUpdateLogsServer) SendAndClose(m *Response) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *dependencyServiceUpdateLogsServer) Recv() (*DependencyServiceUpdateLogsRequest, error) {
|
||||
m := new(DependencyServiceUpdateLogsRequest)
|
||||
if err := x.ServerStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type DependencyService_UpdateLogsServer = grpc.ClientStreamingServer[DependencyServiceUpdateLogsRequest, Response]
|
||||
|
||||
func _DependencyService_SyncConfigSetup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(DependencyServiceSyncConfigSetupRequest)
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.27.2
|
||||
// protoc-gen-go v1.36.1
|
||||
// protoc v5.29.2
|
||||
// source: services/metric_service.proto
|
||||
|
||||
package grpc
|
||||
@@ -21,36 +21,33 @@ const (
|
||||
)
|
||||
|
||||
type MetricServiceSendRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"`
|
||||
NodeKey string `protobuf:"bytes,2,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
Timestamp int64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
|
||||
CpuUsagePercent float32 `protobuf:"fixed32,4,opt,name=cpu_usage_percent,json=cpuUsagePercent,proto3" json:"cpu_usage_percent,omitempty"`
|
||||
TotalMemory uint64 `protobuf:"varint,5,opt,name=total_memory,json=totalMemory,proto3" json:"total_memory,omitempty"`
|
||||
AvailableMemory uint64 `protobuf:"varint,6,opt,name=available_memory,json=availableMemory,proto3" json:"available_memory,omitempty"`
|
||||
UsedMemory uint64 `protobuf:"varint,7,opt,name=used_memory,json=usedMemory,proto3" json:"used_memory,omitempty"`
|
||||
UsedMemoryPercent float32 `protobuf:"fixed32,8,opt,name=used_memory_percent,json=usedMemoryPercent,proto3" json:"used_memory_percent,omitempty"`
|
||||
TotalDisk uint64 `protobuf:"varint,9,opt,name=total_disk,json=totalDisk,proto3" json:"total_disk,omitempty"`
|
||||
AvailableDisk uint64 `protobuf:"varint,10,opt,name=available_disk,json=availableDisk,proto3" json:"available_disk,omitempty"`
|
||||
UsedDisk uint64 `protobuf:"varint,11,opt,name=used_disk,json=usedDisk,proto3" json:"used_disk,omitempty"`
|
||||
UsedDiskPercent float32 `protobuf:"fixed32,12,opt,name=used_disk_percent,json=usedDiskPercent,proto3" json:"used_disk_percent,omitempty"`
|
||||
DiskReadBytesRate float32 `protobuf:"fixed32,15,opt,name=disk_read_bytes_rate,json=diskReadBytesRate,proto3" json:"disk_read_bytes_rate,omitempty"`
|
||||
DiskWriteBytesRate float32 `protobuf:"fixed32,16,opt,name=disk_write_bytes_rate,json=diskWriteBytesRate,proto3" json:"disk_write_bytes_rate,omitempty"`
|
||||
NetworkBytesSentRate float32 `protobuf:"fixed32,17,opt,name=network_bytes_sent_rate,json=networkBytesSentRate,proto3" json:"network_bytes_sent_rate,omitempty"`
|
||||
NetworkBytesRecvRate float32 `protobuf:"fixed32,18,opt,name=network_bytes_recv_rate,json=networkBytesRecvRate,proto3" json:"network_bytes_recv_rate,omitempty"`
|
||||
GoroutineCount int32 `protobuf:"varint,19,opt,name=goroutine_count,json=goroutineCount,proto3" json:"goroutine_count,omitempty"`
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"`
|
||||
NodeKey string `protobuf:"bytes,2,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
Timestamp int64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
|
||||
CpuUsagePercent float32 `protobuf:"fixed32,4,opt,name=cpu_usage_percent,json=cpuUsagePercent,proto3" json:"cpu_usage_percent,omitempty"`
|
||||
TotalMemory uint64 `protobuf:"varint,5,opt,name=total_memory,json=totalMemory,proto3" json:"total_memory,omitempty"`
|
||||
AvailableMemory uint64 `protobuf:"varint,6,opt,name=available_memory,json=availableMemory,proto3" json:"available_memory,omitempty"`
|
||||
UsedMemory uint64 `protobuf:"varint,7,opt,name=used_memory,json=usedMemory,proto3" json:"used_memory,omitempty"`
|
||||
UsedMemoryPercent float32 `protobuf:"fixed32,8,opt,name=used_memory_percent,json=usedMemoryPercent,proto3" json:"used_memory_percent,omitempty"`
|
||||
TotalDisk uint64 `protobuf:"varint,9,opt,name=total_disk,json=totalDisk,proto3" json:"total_disk,omitempty"`
|
||||
AvailableDisk uint64 `protobuf:"varint,10,opt,name=available_disk,json=availableDisk,proto3" json:"available_disk,omitempty"`
|
||||
UsedDisk uint64 `protobuf:"varint,11,opt,name=used_disk,json=usedDisk,proto3" json:"used_disk,omitempty"`
|
||||
UsedDiskPercent float32 `protobuf:"fixed32,12,opt,name=used_disk_percent,json=usedDiskPercent,proto3" json:"used_disk_percent,omitempty"`
|
||||
DiskReadBytesRate float32 `protobuf:"fixed32,15,opt,name=disk_read_bytes_rate,json=diskReadBytesRate,proto3" json:"disk_read_bytes_rate,omitempty"`
|
||||
DiskWriteBytesRate float32 `protobuf:"fixed32,16,opt,name=disk_write_bytes_rate,json=diskWriteBytesRate,proto3" json:"disk_write_bytes_rate,omitempty"`
|
||||
NetworkBytesSentRate float32 `protobuf:"fixed32,17,opt,name=network_bytes_sent_rate,json=networkBytesSentRate,proto3" json:"network_bytes_sent_rate,omitempty"`
|
||||
NetworkBytesRecvRate float32 `protobuf:"fixed32,18,opt,name=network_bytes_recv_rate,json=networkBytesRecvRate,proto3" json:"network_bytes_recv_rate,omitempty"`
|
||||
GoroutineCount int32 `protobuf:"varint,19,opt,name=goroutine_count,json=goroutineCount,proto3" json:"goroutine_count,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *MetricServiceSendRequest) Reset() {
|
||||
*x = MetricServiceSendRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_metric_service_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_metric_service_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *MetricServiceSendRequest) String() string {
|
||||
@@ -61,7 +58,7 @@ func (*MetricServiceSendRequest) ProtoMessage() {}
|
||||
|
||||
func (x *MetricServiceSendRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_metric_service_proto_msgTypes[0]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -287,20 +284,6 @@ func file_services_metric_service_proto_init() {
|
||||
return
|
||||
}
|
||||
file_entity_response_proto_init()
|
||||
if !protoimpl.UnsafeEnabled {
|
||||
file_services_metric_service_proto_msgTypes[0].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*MetricServiceSendRequest); 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{
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.4.0
|
||||
// - protoc v5.27.2
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.29.2
|
||||
// source: services/metric_service.proto
|
||||
|
||||
package grpc
|
||||
@@ -15,8 +15,8 @@ import (
|
||||
|
||||
// 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.62.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion8
|
||||
// Requires gRPC-Go v1.64.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion9
|
||||
|
||||
const (
|
||||
MetricService_Send_FullMethodName = "/grpc.MetricService/Send"
|
||||
@@ -49,20 +49,24 @@ func (c *metricServiceClient) Send(ctx context.Context, in *MetricServiceSendReq
|
||||
|
||||
// MetricServiceServer is the server API for MetricService service.
|
||||
// All implementations must embed UnimplementedMetricServiceServer
|
||||
// for forward compatibility
|
||||
// for forward compatibility.
|
||||
type MetricServiceServer interface {
|
||||
Send(context.Context, *MetricServiceSendRequest) (*Response, error)
|
||||
mustEmbedUnimplementedMetricServiceServer()
|
||||
}
|
||||
|
||||
// UnimplementedMetricServiceServer must be embedded to have forward compatible implementations.
|
||||
type UnimplementedMetricServiceServer struct {
|
||||
}
|
||||
// UnimplementedMetricServiceServer must be embedded to have
|
||||
// forward compatible implementations.
|
||||
//
|
||||
// NOTE: this should be embedded by value instead of pointer to avoid a nil
|
||||
// pointer dereference when methods are called.
|
||||
type UnimplementedMetricServiceServer struct{}
|
||||
|
||||
func (UnimplementedMetricServiceServer) Send(context.Context, *MetricServiceSendRequest) (*Response, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Send not implemented")
|
||||
}
|
||||
func (UnimplementedMetricServiceServer) mustEmbedUnimplementedMetricServiceServer() {}
|
||||
func (UnimplementedMetricServiceServer) testEmbeddedByValue() {}
|
||||
|
||||
// UnsafeMetricServiceServer may be embedded to opt out of forward compatibility for this service.
|
||||
// Use of this interface is not recommended, as added methods to MetricServiceServer will
|
||||
@@ -72,6 +76,13 @@ type UnsafeMetricServiceServer interface {
|
||||
}
|
||||
|
||||
func RegisterMetricServiceServer(s grpc.ServiceRegistrar, srv MetricServiceServer) {
|
||||
// If the following call pancis, it indicates UnimplementedMetricServiceServer was
|
||||
// embedded by pointer and is nil. This will cause panics if an
|
||||
// unimplemented method is ever invoked, so we test this at initialization
|
||||
// time to prevent it from happening at runtime later due to I/O.
|
||||
if t, ok := srv.(interface{ testEmbeddedByValue() }); ok {
|
||||
t.testEmbeddedByValue()
|
||||
}
|
||||
s.RegisterService(&MetricService_ServiceDesc, srv)
|
||||
}
|
||||
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.27.2
|
||||
// protoc-gen-go v1.36.1
|
||||
// protoc v5.29.2
|
||||
// source: services/model_base_service.proto
|
||||
|
||||
package grpc
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.4.0
|
||||
// - protoc v5.27.2
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.29.2
|
||||
// source: services/model_base_service.proto
|
||||
|
||||
package grpc
|
||||
@@ -15,8 +15,8 @@ import (
|
||||
|
||||
// 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.62.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion8
|
||||
// Requires gRPC-Go v1.64.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion9
|
||||
|
||||
const (
|
||||
ModelBaseService_GetById_FullMethodName = "/grpc.ModelBaseService/GetById"
|
||||
@@ -217,7 +217,7 @@ func (c *modelBaseServiceClient) Count(ctx context.Context, in *ModelServiceCoun
|
||||
|
||||
// ModelBaseServiceServer is the server API for ModelBaseService service.
|
||||
// All implementations must embed UnimplementedModelBaseServiceServer
|
||||
// for forward compatibility
|
||||
// for forward compatibility.
|
||||
type ModelBaseServiceServer interface {
|
||||
GetById(context.Context, *ModelServiceGetByIdRequest) (*Response, error)
|
||||
GetOne(context.Context, *ModelServiceGetOneRequest) (*Response, error)
|
||||
@@ -237,9 +237,12 @@ type ModelBaseServiceServer interface {
|
||||
mustEmbedUnimplementedModelBaseServiceServer()
|
||||
}
|
||||
|
||||
// UnimplementedModelBaseServiceServer must be embedded to have forward compatible implementations.
|
||||
type UnimplementedModelBaseServiceServer struct {
|
||||
}
|
||||
// UnimplementedModelBaseServiceServer must be embedded to have
|
||||
// forward compatible implementations.
|
||||
//
|
||||
// NOTE: this should be embedded by value instead of pointer to avoid a nil
|
||||
// pointer dereference when methods are called.
|
||||
type UnimplementedModelBaseServiceServer struct{}
|
||||
|
||||
func (UnimplementedModelBaseServiceServer) GetById(context.Context, *ModelServiceGetByIdRequest) (*Response, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetById not implemented")
|
||||
@@ -287,6 +290,7 @@ func (UnimplementedModelBaseServiceServer) Count(context.Context, *ModelServiceC
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Count not implemented")
|
||||
}
|
||||
func (UnimplementedModelBaseServiceServer) mustEmbedUnimplementedModelBaseServiceServer() {}
|
||||
func (UnimplementedModelBaseServiceServer) testEmbeddedByValue() {}
|
||||
|
||||
// UnsafeModelBaseServiceServer may be embedded to opt out of forward compatibility for this service.
|
||||
// Use of this interface is not recommended, as added methods to ModelBaseServiceServer will
|
||||
@@ -296,6 +300,13 @@ type UnsafeModelBaseServiceServer interface {
|
||||
}
|
||||
|
||||
func RegisterModelBaseServiceServer(s grpc.ServiceRegistrar, srv ModelBaseServiceServer) {
|
||||
// If the following call pancis, it indicates UnimplementedModelBaseServiceServer was
|
||||
// embedded by pointer and is nil. This will cause panics if an
|
||||
// unimplemented method is ever invoked, so we test this at initialization
|
||||
// time to prevent it from happening at runtime later due to I/O.
|
||||
if t, ok := srv.(interface{ testEmbeddedByValue() }); ok {
|
||||
t.testEmbeddedByValue()
|
||||
}
|
||||
s.RegisterService(&ModelBaseService_ServiceDesc, srv)
|
||||
}
|
||||
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.27.2
|
||||
// protoc-gen-go v1.36.1
|
||||
// protoc v5.29.2
|
||||
// source: entity/model_service_request.proto
|
||||
|
||||
package grpc
|
||||
@@ -21,22 +21,19 @@ const (
|
||||
)
|
||||
|
||||
type ModelServiceGetByIdRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceGetByIdRequest) Reset() {
|
||||
*x = ModelServiceGetByIdRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceGetByIdRequest) String() string {
|
||||
@@ -47,7 +44,7 @@ func (*ModelServiceGetByIdRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceGetByIdRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[0]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -84,23 +81,20 @@ func (x *ModelServiceGetByIdRequest) GetId() string {
|
||||
}
|
||||
|
||||
type ModelServiceGetOneRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
FindOptions []byte `protobuf:"bytes,4,opt,name=find_options,json=findOptions,proto3" json:"find_options,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
FindOptions []byte `protobuf:"bytes,4,opt,name=find_options,json=findOptions,proto3" json:"find_options,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceGetOneRequest) Reset() {
|
||||
*x = ModelServiceGetOneRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceGetOneRequest) String() string {
|
||||
@@ -111,7 +105,7 @@ func (*ModelServiceGetOneRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceGetOneRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[1]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -155,23 +149,20 @@ func (x *ModelServiceGetOneRequest) GetFindOptions() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceGetManyRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
FindOptions []byte `protobuf:"bytes,4,opt,name=find_options,json=findOptions,proto3" json:"find_options,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
FindOptions []byte `protobuf:"bytes,4,opt,name=find_options,json=findOptions,proto3" json:"find_options,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceGetManyRequest) Reset() {
|
||||
*x = ModelServiceGetManyRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceGetManyRequest) String() string {
|
||||
@@ -182,7 +173,7 @@ func (*ModelServiceGetManyRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceGetManyRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[2]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -226,22 +217,19 @@ func (x *ModelServiceGetManyRequest) GetFindOptions() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceDeleteByIdRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceDeleteByIdRequest) Reset() {
|
||||
*x = ModelServiceDeleteByIdRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[3]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[3]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceDeleteByIdRequest) String() string {
|
||||
@@ -252,7 +240,7 @@ func (*ModelServiceDeleteByIdRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceDeleteByIdRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[3]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -289,22 +277,19 @@ func (x *ModelServiceDeleteByIdRequest) GetId() string {
|
||||
}
|
||||
|
||||
type ModelServiceDeleteOneRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceDeleteOneRequest) Reset() {
|
||||
*x = ModelServiceDeleteOneRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[4]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[4]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceDeleteOneRequest) String() string {
|
||||
@@ -315,7 +300,7 @@ func (*ModelServiceDeleteOneRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceDeleteOneRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[4]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -352,22 +337,19 @@ func (x *ModelServiceDeleteOneRequest) GetQuery() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceDeleteManyRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceDeleteManyRequest) Reset() {
|
||||
*x = ModelServiceDeleteManyRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[5]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[5]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceDeleteManyRequest) String() string {
|
||||
@@ -378,7 +360,7 @@ func (*ModelServiceDeleteManyRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceDeleteManyRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[5]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -415,23 +397,20 @@ func (x *ModelServiceDeleteManyRequest) GetQuery() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceUpdateByIdRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"`
|
||||
Update []byte `protobuf:"bytes,4,opt,name=update,proto3" json:"update,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"`
|
||||
Update []byte `protobuf:"bytes,4,opt,name=update,proto3" json:"update,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceUpdateByIdRequest) Reset() {
|
||||
*x = ModelServiceUpdateByIdRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[6]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[6]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceUpdateByIdRequest) String() string {
|
||||
@@ -442,7 +421,7 @@ func (*ModelServiceUpdateByIdRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceUpdateByIdRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[6]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -486,23 +465,20 @@ func (x *ModelServiceUpdateByIdRequest) GetUpdate() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceUpdateOneRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
Update []byte `protobuf:"bytes,4,opt,name=update,proto3" json:"update,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
Update []byte `protobuf:"bytes,4,opt,name=update,proto3" json:"update,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceUpdateOneRequest) Reset() {
|
||||
*x = ModelServiceUpdateOneRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[7]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[7]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceUpdateOneRequest) String() string {
|
||||
@@ -513,7 +489,7 @@ func (*ModelServiceUpdateOneRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceUpdateOneRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[7]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -557,23 +533,20 @@ func (x *ModelServiceUpdateOneRequest) GetUpdate() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceUpdateManyRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
Update []byte `protobuf:"bytes,4,opt,name=update,proto3" json:"update,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
Update []byte `protobuf:"bytes,4,opt,name=update,proto3" json:"update,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceUpdateManyRequest) Reset() {
|
||||
*x = ModelServiceUpdateManyRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[8]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[8]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceUpdateManyRequest) String() string {
|
||||
@@ -584,7 +557,7 @@ func (*ModelServiceUpdateManyRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceUpdateManyRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[8]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -628,23 +601,20 @@ func (x *ModelServiceUpdateManyRequest) GetUpdate() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceReplaceByIdRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"`
|
||||
Model []byte `protobuf:"bytes,4,opt,name=model,proto3" json:"model,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"`
|
||||
Model []byte `protobuf:"bytes,4,opt,name=model,proto3" json:"model,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceReplaceByIdRequest) Reset() {
|
||||
*x = ModelServiceReplaceByIdRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[9]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[9]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceReplaceByIdRequest) String() string {
|
||||
@@ -655,7 +625,7 @@ func (*ModelServiceReplaceByIdRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceReplaceByIdRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[9]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -699,23 +669,20 @@ func (x *ModelServiceReplaceByIdRequest) GetModel() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceReplaceOneRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
Model []byte `protobuf:"bytes,4,opt,name=model,proto3" json:"model,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
Model []byte `protobuf:"bytes,4,opt,name=model,proto3" json:"model,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceReplaceOneRequest) Reset() {
|
||||
*x = ModelServiceReplaceOneRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[10]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[10]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceReplaceOneRequest) String() string {
|
||||
@@ -726,7 +693,7 @@ func (*ModelServiceReplaceOneRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceReplaceOneRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[10]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -770,22 +737,19 @@ func (x *ModelServiceReplaceOneRequest) GetModel() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceInsertOneRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Model []byte `protobuf:"bytes,3,opt,name=model,proto3" json:"model,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Model []byte `protobuf:"bytes,3,opt,name=model,proto3" json:"model,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceInsertOneRequest) Reset() {
|
||||
*x = ModelServiceInsertOneRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[11]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[11]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceInsertOneRequest) String() string {
|
||||
@@ -796,7 +760,7 @@ func (*ModelServiceInsertOneRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceInsertOneRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[11]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -833,22 +797,19 @@ func (x *ModelServiceInsertOneRequest) GetModel() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceInsertManyRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Models []byte `protobuf:"bytes,3,opt,name=models,proto3" json:"models,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Models []byte `protobuf:"bytes,3,opt,name=models,proto3" json:"models,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceInsertManyRequest) Reset() {
|
||||
*x = ModelServiceInsertManyRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[12]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[12]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceInsertManyRequest) String() string {
|
||||
@@ -859,7 +820,7 @@ func (*ModelServiceInsertManyRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceInsertManyRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[12]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -896,23 +857,20 @@ func (x *ModelServiceInsertManyRequest) GetModels() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceUpsertOneRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
Model []byte `protobuf:"bytes,4,opt,name=model,proto3" json:"model,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
Model []byte `protobuf:"bytes,4,opt,name=model,proto3" json:"model,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceUpsertOneRequest) Reset() {
|
||||
*x = ModelServiceUpsertOneRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[13]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[13]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceUpsertOneRequest) String() string {
|
||||
@@ -923,7 +881,7 @@ func (*ModelServiceUpsertOneRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceUpsertOneRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[13]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -967,22 +925,19 @@ func (x *ModelServiceUpsertOneRequest) GetModel() []byte {
|
||||
}
|
||||
|
||||
type ModelServiceCountRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
ModelType string `protobuf:"bytes,2,opt,name=model_type,json=modelType,proto3" json:"model_type,omitempty"`
|
||||
Query []byte `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *ModelServiceCountRequest) Reset() {
|
||||
*x = ModelServiceCountRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[14]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[14]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *ModelServiceCountRequest) String() string {
|
||||
@@ -993,7 +948,7 @@ func (*ModelServiceCountRequest) ProtoMessage() {}
|
||||
|
||||
func (x *ModelServiceCountRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_model_service_request_proto_msgTypes[14]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -1198,188 +1153,6 @@ func file_entity_model_service_request_proto_init() {
|
||||
if File_entity_model_service_request_proto != nil {
|
||||
return
|
||||
}
|
||||
if !protoimpl.UnsafeEnabled {
|
||||
file_entity_model_service_request_proto_msgTypes[0].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceGetByIdRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[1].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceGetOneRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[2].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceGetManyRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[3].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceDeleteByIdRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[4].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceDeleteOneRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[5].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceDeleteManyRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[6].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceUpdateByIdRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[7].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceUpdateOneRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[8].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceUpdateManyRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[9].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceReplaceByIdRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[10].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceReplaceOneRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[11].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceInsertOneRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[12].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceInsertManyRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[13].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceUpsertOneRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_entity_model_service_request_proto_msgTypes[14].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*ModelServiceCountRequest); 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{
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.27.2
|
||||
// protoc-gen-go v1.36.1
|
||||
// protoc v5.29.2
|
||||
// source: services/node_service.proto
|
||||
|
||||
package grpc
|
||||
@@ -64,22 +64,19 @@ func (NodeServiceSubscribeCode) EnumDescriptor() ([]byte, []int) {
|
||||
}
|
||||
|
||||
type NodeServiceRegisterRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
NodeName string `protobuf:"bytes,2,opt,name=node_name,json=nodeName,proto3" json:"node_name,omitempty"`
|
||||
MaxRunners int32 `protobuf:"varint,3,opt,name=max_runners,json=maxRunners,proto3" json:"max_runners,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
NodeName string `protobuf:"bytes,2,opt,name=node_name,json=nodeName,proto3" json:"node_name,omitempty"`
|
||||
MaxRunners int32 `protobuf:"varint,3,opt,name=max_runners,json=maxRunners,proto3" json:"max_runners,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *NodeServiceRegisterRequest) Reset() {
|
||||
*x = NodeServiceRegisterRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_node_service_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_node_service_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *NodeServiceRegisterRequest) String() string {
|
||||
@@ -90,7 +87,7 @@ func (*NodeServiceRegisterRequest) ProtoMessage() {}
|
||||
|
||||
func (x *NodeServiceRegisterRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_node_service_proto_msgTypes[0]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -127,20 +124,17 @@ func (x *NodeServiceRegisterRequest) GetMaxRunners() int32 {
|
||||
}
|
||||
|
||||
type NodeServiceSendHeartbeatRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *NodeServiceSendHeartbeatRequest) Reset() {
|
||||
*x = NodeServiceSendHeartbeatRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_node_service_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_node_service_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *NodeServiceSendHeartbeatRequest) String() string {
|
||||
@@ -151,7 +145,7 @@ func (*NodeServiceSendHeartbeatRequest) ProtoMessage() {}
|
||||
|
||||
func (x *NodeServiceSendHeartbeatRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_node_service_proto_msgTypes[1]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -174,20 +168,17 @@ func (x *NodeServiceSendHeartbeatRequest) GetNodeKey() string {
|
||||
}
|
||||
|
||||
type NodeServiceSubscribeRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *NodeServiceSubscribeRequest) Reset() {
|
||||
*x = NodeServiceSubscribeRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_node_service_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_node_service_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *NodeServiceSubscribeRequest) String() string {
|
||||
@@ -198,7 +189,7 @@ func (*NodeServiceSubscribeRequest) ProtoMessage() {}
|
||||
|
||||
func (x *NodeServiceSubscribeRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_node_service_proto_msgTypes[2]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -221,20 +212,17 @@ func (x *NodeServiceSubscribeRequest) GetNodeKey() string {
|
||||
}
|
||||
|
||||
type NodeServiceSubscribeResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Code NodeServiceSubscribeCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.NodeServiceSubscribeCode" json:"code,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Code NodeServiceSubscribeCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.NodeServiceSubscribeCode" json:"code,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *NodeServiceSubscribeResponse) Reset() {
|
||||
*x = NodeServiceSubscribeResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_node_service_proto_msgTypes[3]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_node_service_proto_msgTypes[3]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *NodeServiceSubscribeResponse) String() string {
|
||||
@@ -245,7 +233,7 @@ func (*NodeServiceSubscribeResponse) ProtoMessage() {}
|
||||
|
||||
func (x *NodeServiceSubscribeResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_node_service_proto_msgTypes[3]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -358,56 +346,6 @@ func file_services_node_service_proto_init() {
|
||||
return
|
||||
}
|
||||
file_entity_response_proto_init()
|
||||
if !protoimpl.UnsafeEnabled {
|
||||
file_services_node_service_proto_msgTypes[0].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*NodeServiceRegisterRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_node_service_proto_msgTypes[1].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*NodeServiceSendHeartbeatRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_node_service_proto_msgTypes[2].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*NodeServiceSubscribeRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_node_service_proto_msgTypes[3].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*NodeServiceSubscribeResponse); 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{
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.4.0
|
||||
// - protoc v5.27.2
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.29.2
|
||||
// source: services/node_service.proto
|
||||
|
||||
package grpc
|
||||
@@ -15,8 +15,8 @@ import (
|
||||
|
||||
// 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.62.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion8
|
||||
// Requires gRPC-Go v1.64.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion9
|
||||
|
||||
const (
|
||||
NodeService_Register_FullMethodName = "/grpc.NodeService/Register"
|
||||
@@ -30,7 +30,7 @@ const (
|
||||
type NodeServiceClient interface {
|
||||
Register(ctx context.Context, in *NodeServiceRegisterRequest, opts ...grpc.CallOption) (*Response, error)
|
||||
SendHeartbeat(ctx context.Context, in *NodeServiceSendHeartbeatRequest, opts ...grpc.CallOption) (*Response, error)
|
||||
Subscribe(ctx context.Context, in *NodeServiceSubscribeRequest, opts ...grpc.CallOption) (NodeService_SubscribeClient, error)
|
||||
Subscribe(ctx context.Context, in *NodeServiceSubscribeRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[NodeServiceSubscribeResponse], error)
|
||||
}
|
||||
|
||||
type nodeServiceClient struct {
|
||||
@@ -61,13 +61,13 @@ func (c *nodeServiceClient) SendHeartbeat(ctx context.Context, in *NodeServiceSe
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *nodeServiceClient) Subscribe(ctx context.Context, in *NodeServiceSubscribeRequest, opts ...grpc.CallOption) (NodeService_SubscribeClient, error) {
|
||||
func (c *nodeServiceClient) Subscribe(ctx context.Context, in *NodeServiceSubscribeRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[NodeServiceSubscribeResponse], error) {
|
||||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &NodeService_ServiceDesc.Streams[0], NodeService_Subscribe_FullMethodName, cOpts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &nodeServiceSubscribeClient{ClientStream: stream}
|
||||
x := &grpc.GenericClientStream[NodeServiceSubscribeRequest, NodeServiceSubscribeResponse]{ClientStream: stream}
|
||||
if err := x.ClientStream.SendMsg(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -77,36 +77,25 @@ func (c *nodeServiceClient) Subscribe(ctx context.Context, in *NodeServiceSubscr
|
||||
return x, nil
|
||||
}
|
||||
|
||||
type NodeService_SubscribeClient interface {
|
||||
Recv() (*NodeServiceSubscribeResponse, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
type nodeServiceSubscribeClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *nodeServiceSubscribeClient) Recv() (*NodeServiceSubscribeResponse, error) {
|
||||
m := new(NodeServiceSubscribeResponse)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type NodeService_SubscribeClient = grpc.ServerStreamingClient[NodeServiceSubscribeResponse]
|
||||
|
||||
// NodeServiceServer is the server API for NodeService service.
|
||||
// All implementations must embed UnimplementedNodeServiceServer
|
||||
// for forward compatibility
|
||||
// for forward compatibility.
|
||||
type NodeServiceServer interface {
|
||||
Register(context.Context, *NodeServiceRegisterRequest) (*Response, error)
|
||||
SendHeartbeat(context.Context, *NodeServiceSendHeartbeatRequest) (*Response, error)
|
||||
Subscribe(*NodeServiceSubscribeRequest, NodeService_SubscribeServer) error
|
||||
Subscribe(*NodeServiceSubscribeRequest, grpc.ServerStreamingServer[NodeServiceSubscribeResponse]) error
|
||||
mustEmbedUnimplementedNodeServiceServer()
|
||||
}
|
||||
|
||||
// UnimplementedNodeServiceServer must be embedded to have forward compatible implementations.
|
||||
type UnimplementedNodeServiceServer struct {
|
||||
}
|
||||
// UnimplementedNodeServiceServer must be embedded to have
|
||||
// forward compatible implementations.
|
||||
//
|
||||
// NOTE: this should be embedded by value instead of pointer to avoid a nil
|
||||
// pointer dereference when methods are called.
|
||||
type UnimplementedNodeServiceServer struct{}
|
||||
|
||||
func (UnimplementedNodeServiceServer) Register(context.Context, *NodeServiceRegisterRequest) (*Response, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Register not implemented")
|
||||
@@ -114,10 +103,11 @@ func (UnimplementedNodeServiceServer) Register(context.Context, *NodeServiceRegi
|
||||
func (UnimplementedNodeServiceServer) SendHeartbeat(context.Context, *NodeServiceSendHeartbeatRequest) (*Response, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method SendHeartbeat not implemented")
|
||||
}
|
||||
func (UnimplementedNodeServiceServer) Subscribe(*NodeServiceSubscribeRequest, NodeService_SubscribeServer) error {
|
||||
func (UnimplementedNodeServiceServer) Subscribe(*NodeServiceSubscribeRequest, grpc.ServerStreamingServer[NodeServiceSubscribeResponse]) error {
|
||||
return status.Errorf(codes.Unimplemented, "method Subscribe not implemented")
|
||||
}
|
||||
func (UnimplementedNodeServiceServer) mustEmbedUnimplementedNodeServiceServer() {}
|
||||
func (UnimplementedNodeServiceServer) testEmbeddedByValue() {}
|
||||
|
||||
// UnsafeNodeServiceServer may be embedded to opt out of forward compatibility for this service.
|
||||
// Use of this interface is not recommended, as added methods to NodeServiceServer will
|
||||
@@ -127,6 +117,13 @@ type UnsafeNodeServiceServer interface {
|
||||
}
|
||||
|
||||
func RegisterNodeServiceServer(s grpc.ServiceRegistrar, srv NodeServiceServer) {
|
||||
// If the following call pancis, it indicates UnimplementedNodeServiceServer was
|
||||
// embedded by pointer and is nil. This will cause panics if an
|
||||
// unimplemented method is ever invoked, so we test this at initialization
|
||||
// time to prevent it from happening at runtime later due to I/O.
|
||||
if t, ok := srv.(interface{ testEmbeddedByValue() }); ok {
|
||||
t.testEmbeddedByValue()
|
||||
}
|
||||
s.RegisterService(&NodeService_ServiceDesc, srv)
|
||||
}
|
||||
|
||||
@@ -171,21 +168,11 @@ func _NodeService_Subscribe_Handler(srv interface{}, stream grpc.ServerStream) e
|
||||
if err := stream.RecvMsg(m); err != nil {
|
||||
return err
|
||||
}
|
||||
return srv.(NodeServiceServer).Subscribe(m, &nodeServiceSubscribeServer{ServerStream: stream})
|
||||
return srv.(NodeServiceServer).Subscribe(m, &grpc.GenericServerStream[NodeServiceSubscribeRequest, NodeServiceSubscribeResponse]{ServerStream: stream})
|
||||
}
|
||||
|
||||
type NodeService_SubscribeServer interface {
|
||||
Send(*NodeServiceSubscribeResponse) error
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type nodeServiceSubscribeServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *nodeServiceSubscribeServer) Send(m *NodeServiceSubscribeResponse) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type NodeService_SubscribeServer = grpc.ServerStreamingServer[NodeServiceSubscribeResponse]
|
||||
|
||||
// NodeService_ServiceDesc is the grpc.ServiceDesc for NodeService service.
|
||||
// It's only intended for direct use with grpc.RegisterService,
|
||||
|
||||
55
grpc/proto/services/sync_service.proto
Normal file
55
grpc/proto/services/sync_service.proto
Normal file
@@ -0,0 +1,55 @@
|
||||
syntax = "proto3";
|
||||
|
||||
package grpc;
|
||||
option go_package = ".;grpc";
|
||||
|
||||
// File synchronization request
|
||||
message FileSyncRequest {
|
||||
string spider_id = 1; // or git_id
|
||||
string path = 2; // working directory path
|
||||
string node_key = 3; // worker node key
|
||||
}
|
||||
|
||||
// File information message (streamable)
|
||||
message FileInfo {
|
||||
string name = 1;
|
||||
string path = 2;
|
||||
string full_path = 3;
|
||||
string extension = 4;
|
||||
bool is_dir = 5;
|
||||
int64 file_size = 6;
|
||||
int64 mod_time = 7; // Unix timestamp
|
||||
uint32 mode = 8; // File permissions
|
||||
string hash = 9; // File content hash
|
||||
}
|
||||
|
||||
// Stream response for file scan
|
||||
message FileScanChunk {
|
||||
repeated FileInfo files = 1; // Batch of files
|
||||
bool is_complete = 2; // Last chunk indicator
|
||||
string error = 3; // Error message if any
|
||||
int32 total_files = 4; // Total file count (in last chunk)
|
||||
}
|
||||
|
||||
// Download request
|
||||
message FileDownloadRequest {
|
||||
string spider_id = 1;
|
||||
string path = 2;
|
||||
string node_key = 3;
|
||||
}
|
||||
|
||||
// Download response (streamed in chunks)
|
||||
message FileDownloadChunk {
|
||||
bytes data = 1; // File data chunk
|
||||
bool is_complete = 2; // Last chunk indicator
|
||||
string error = 3; // Error if any
|
||||
int64 total_bytes = 4; // Total file size (in first chunk)
|
||||
}
|
||||
|
||||
service SyncService {
|
||||
// Stream file list for synchronization
|
||||
rpc StreamFileScan(FileSyncRequest) returns (stream FileScanChunk);
|
||||
|
||||
// Stream file download
|
||||
rpc StreamFileDownload(FileDownloadRequest) returns (stream FileDownloadChunk);
|
||||
}
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.27.2
|
||||
// protoc-gen-go v1.36.1
|
||||
// protoc v5.29.2
|
||||
// source: entity/response.proto
|
||||
|
||||
package grpc
|
||||
@@ -67,24 +67,21 @@ func (ResponseCode) EnumDescriptor() ([]byte, []int) {
|
||||
}
|
||||
|
||||
type Response struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Code ResponseCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.ResponseCode" json:"code,omitempty"`
|
||||
Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
|
||||
Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
|
||||
Error string `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"`
|
||||
Total int64 `protobuf:"varint,5,opt,name=total,proto3" json:"total,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Code ResponseCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.ResponseCode" json:"code,omitempty"`
|
||||
Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
|
||||
Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
|
||||
Error string `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"`
|
||||
Total int64 `protobuf:"varint,5,opt,name=total,proto3" json:"total,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *Response) Reset() {
|
||||
*x = Response{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_entity_response_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_entity_response_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *Response) String() string {
|
||||
@@ -95,7 +92,7 @@ func (*Response) ProtoMessage() {}
|
||||
|
||||
func (x *Response) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_entity_response_proto_msgTypes[0]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -197,20 +194,6 @@ func file_entity_response_proto_init() {
|
||||
if File_entity_response_proto != nil {
|
||||
return
|
||||
}
|
||||
if !protoimpl.UnsafeEnabled {
|
||||
file_entity_response_proto_msgTypes[0].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
|
||||
}
|
||||
}
|
||||
}
|
||||
type x struct{}
|
||||
out := protoimpl.TypeBuilder{
|
||||
File: protoimpl.DescBuilder{
|
||||
|
||||
510
grpc/sync_service.pb.go
Normal file
510
grpc/sync_service.pb.go
Normal file
@@ -0,0 +1,510 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.36.1
|
||||
// protoc v5.29.2
|
||||
// source: services/sync_service.proto
|
||||
|
||||
package grpc
|
||||
|
||||
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)
|
||||
)
|
||||
|
||||
// File synchronization request
|
||||
type FileSyncRequest struct {
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
SpiderId string `protobuf:"bytes,1,opt,name=spider_id,json=spiderId,proto3" json:"spider_id,omitempty"` // or git_id
|
||||
Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` // working directory path
|
||||
NodeKey string `protobuf:"bytes,3,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"` // worker node key
|
||||
unknownFields protoimpl.UnknownFields
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *FileSyncRequest) Reset() {
|
||||
*x = FileSyncRequest{}
|
||||
mi := &file_services_sync_service_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *FileSyncRequest) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*FileSyncRequest) ProtoMessage() {}
|
||||
|
||||
func (x *FileSyncRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_sync_service_proto_msgTypes[0]
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use FileSyncRequest.ProtoReflect.Descriptor instead.
|
||||
func (*FileSyncRequest) Descriptor() ([]byte, []int) {
|
||||
return file_services_sync_service_proto_rawDescGZIP(), []int{0}
|
||||
}
|
||||
|
||||
func (x *FileSyncRequest) GetSpiderId() string {
|
||||
if x != nil {
|
||||
return x.SpiderId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FileSyncRequest) GetPath() string {
|
||||
if x != nil {
|
||||
return x.Path
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FileSyncRequest) GetNodeKey() string {
|
||||
if x != nil {
|
||||
return x.NodeKey
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// File information message (streamable)
|
||||
type FileInfo struct {
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
|
||||
Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"`
|
||||
FullPath string `protobuf:"bytes,3,opt,name=full_path,json=fullPath,proto3" json:"full_path,omitempty"`
|
||||
Extension string `protobuf:"bytes,4,opt,name=extension,proto3" json:"extension,omitempty"`
|
||||
IsDir bool `protobuf:"varint,5,opt,name=is_dir,json=isDir,proto3" json:"is_dir,omitempty"`
|
||||
FileSize int64 `protobuf:"varint,6,opt,name=file_size,json=fileSize,proto3" json:"file_size,omitempty"`
|
||||
ModTime int64 `protobuf:"varint,7,opt,name=mod_time,json=modTime,proto3" json:"mod_time,omitempty"` // Unix timestamp
|
||||
Mode uint32 `protobuf:"varint,8,opt,name=mode,proto3" json:"mode,omitempty"` // File permissions
|
||||
Hash string `protobuf:"bytes,9,opt,name=hash,proto3" json:"hash,omitempty"` // File content hash
|
||||
unknownFields protoimpl.UnknownFields
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *FileInfo) Reset() {
|
||||
*x = FileInfo{}
|
||||
mi := &file_services_sync_service_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *FileInfo) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*FileInfo) ProtoMessage() {}
|
||||
|
||||
func (x *FileInfo) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_sync_service_proto_msgTypes[1]
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use FileInfo.ProtoReflect.Descriptor instead.
|
||||
func (*FileInfo) Descriptor() ([]byte, []int) {
|
||||
return file_services_sync_service_proto_rawDescGZIP(), []int{1}
|
||||
}
|
||||
|
||||
func (x *FileInfo) GetName() string {
|
||||
if x != nil {
|
||||
return x.Name
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FileInfo) GetPath() string {
|
||||
if x != nil {
|
||||
return x.Path
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FileInfo) GetFullPath() string {
|
||||
if x != nil {
|
||||
return x.FullPath
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FileInfo) GetExtension() string {
|
||||
if x != nil {
|
||||
return x.Extension
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FileInfo) GetIsDir() bool {
|
||||
if x != nil {
|
||||
return x.IsDir
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (x *FileInfo) GetFileSize() int64 {
|
||||
if x != nil {
|
||||
return x.FileSize
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *FileInfo) GetModTime() int64 {
|
||||
if x != nil {
|
||||
return x.ModTime
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *FileInfo) GetMode() uint32 {
|
||||
if x != nil {
|
||||
return x.Mode
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *FileInfo) GetHash() string {
|
||||
if x != nil {
|
||||
return x.Hash
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// Stream response for file scan
|
||||
type FileScanChunk struct {
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Files []*FileInfo `protobuf:"bytes,1,rep,name=files,proto3" json:"files,omitempty"` // Batch of files
|
||||
IsComplete bool `protobuf:"varint,2,opt,name=is_complete,json=isComplete,proto3" json:"is_complete,omitempty"` // Last chunk indicator
|
||||
Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` // Error message if any
|
||||
TotalFiles int32 `protobuf:"varint,4,opt,name=total_files,json=totalFiles,proto3" json:"total_files,omitempty"` // Total file count (in last chunk)
|
||||
unknownFields protoimpl.UnknownFields
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *FileScanChunk) Reset() {
|
||||
*x = FileScanChunk{}
|
||||
mi := &file_services_sync_service_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *FileScanChunk) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*FileScanChunk) ProtoMessage() {}
|
||||
|
||||
func (x *FileScanChunk) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_sync_service_proto_msgTypes[2]
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use FileScanChunk.ProtoReflect.Descriptor instead.
|
||||
func (*FileScanChunk) Descriptor() ([]byte, []int) {
|
||||
return file_services_sync_service_proto_rawDescGZIP(), []int{2}
|
||||
}
|
||||
|
||||
func (x *FileScanChunk) GetFiles() []*FileInfo {
|
||||
if x != nil {
|
||||
return x.Files
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *FileScanChunk) GetIsComplete() bool {
|
||||
if x != nil {
|
||||
return x.IsComplete
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (x *FileScanChunk) GetError() string {
|
||||
if x != nil {
|
||||
return x.Error
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FileScanChunk) GetTotalFiles() int32 {
|
||||
if x != nil {
|
||||
return x.TotalFiles
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
// Download request
|
||||
type FileDownloadRequest struct {
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
SpiderId string `protobuf:"bytes,1,opt,name=spider_id,json=spiderId,proto3" json:"spider_id,omitempty"`
|
||||
Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"`
|
||||
NodeKey string `protobuf:"bytes,3,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *FileDownloadRequest) Reset() {
|
||||
*x = FileDownloadRequest{}
|
||||
mi := &file_services_sync_service_proto_msgTypes[3]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *FileDownloadRequest) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*FileDownloadRequest) ProtoMessage() {}
|
||||
|
||||
func (x *FileDownloadRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_sync_service_proto_msgTypes[3]
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use FileDownloadRequest.ProtoReflect.Descriptor instead.
|
||||
func (*FileDownloadRequest) Descriptor() ([]byte, []int) {
|
||||
return file_services_sync_service_proto_rawDescGZIP(), []int{3}
|
||||
}
|
||||
|
||||
func (x *FileDownloadRequest) GetSpiderId() string {
|
||||
if x != nil {
|
||||
return x.SpiderId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FileDownloadRequest) GetPath() string {
|
||||
if x != nil {
|
||||
return x.Path
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FileDownloadRequest) GetNodeKey() string {
|
||||
if x != nil {
|
||||
return x.NodeKey
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// Download response (streamed in chunks)
|
||||
type FileDownloadChunk struct {
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` // File data chunk
|
||||
IsComplete bool `protobuf:"varint,2,opt,name=is_complete,json=isComplete,proto3" json:"is_complete,omitempty"` // Last chunk indicator
|
||||
Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` // Error if any
|
||||
TotalBytes int64 `protobuf:"varint,4,opt,name=total_bytes,json=totalBytes,proto3" json:"total_bytes,omitempty"` // Total file size (in first chunk)
|
||||
unknownFields protoimpl.UnknownFields
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *FileDownloadChunk) Reset() {
|
||||
*x = FileDownloadChunk{}
|
||||
mi := &file_services_sync_service_proto_msgTypes[4]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *FileDownloadChunk) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*FileDownloadChunk) ProtoMessage() {}
|
||||
|
||||
func (x *FileDownloadChunk) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_sync_service_proto_msgTypes[4]
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use FileDownloadChunk.ProtoReflect.Descriptor instead.
|
||||
func (*FileDownloadChunk) Descriptor() ([]byte, []int) {
|
||||
return file_services_sync_service_proto_rawDescGZIP(), []int{4}
|
||||
}
|
||||
|
||||
func (x *FileDownloadChunk) GetData() []byte {
|
||||
if x != nil {
|
||||
return x.Data
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *FileDownloadChunk) GetIsComplete() bool {
|
||||
if x != nil {
|
||||
return x.IsComplete
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (x *FileDownloadChunk) GetError() string {
|
||||
if x != nil {
|
||||
return x.Error
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FileDownloadChunk) GetTotalBytes() int64 {
|
||||
if x != nil {
|
||||
return x.TotalBytes
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
var File_services_sync_service_proto protoreflect.FileDescriptor
|
||||
|
||||
var file_services_sync_service_proto_rawDesc = []byte{
|
||||
0x0a, 0x1b, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x2f, 0x73, 0x79, 0x6e, 0x63, 0x5f,
|
||||
0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x04, 0x67,
|
||||
0x72, 0x70, 0x63, 0x22, 0x5d, 0x0a, 0x0f, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x79, 0x6e, 0x63, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x70, 0x69, 0x64, 0x65, 0x72,
|
||||
0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x70, 0x69, 0x64, 0x65,
|
||||
0x72, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x5f,
|
||||
0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6e, 0x6f, 0x64, 0x65, 0x4b,
|
||||
0x65, 0x79, 0x22, 0xe4, 0x01, 0x0a, 0x08, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12,
|
||||
0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e,
|
||||
0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x75, 0x6c, 0x6c, 0x5f,
|
||||
0x70, 0x61, 0x74, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x75, 0x6c, 0x6c,
|
||||
0x50, 0x61, 0x74, 0x68, 0x12, 0x1c, 0x0a, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f,
|
||||
0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69,
|
||||
0x6f, 0x6e, 0x12, 0x15, 0x0a, 0x06, 0x69, 0x73, 0x5f, 0x64, 0x69, 0x72, 0x18, 0x05, 0x20, 0x01,
|
||||
0x28, 0x08, 0x52, 0x05, 0x69, 0x73, 0x44, 0x69, 0x72, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c,
|
||||
0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x66, 0x69,
|
||||
0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x6f, 0x64, 0x5f, 0x74, 0x69,
|
||||
0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6d, 0x6f, 0x64, 0x54, 0x69, 0x6d,
|
||||
0x65, 0x12, 0x12, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0d, 0x52,
|
||||
0x04, 0x6d, 0x6f, 0x64, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x09, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x22, 0x8d, 0x01, 0x0a, 0x0d, 0x46, 0x69,
|
||||
0x6c, 0x65, 0x53, 0x63, 0x61, 0x6e, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x12, 0x24, 0x0a, 0x05, 0x66,
|
||||
0x69, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x67, 0x72, 0x70,
|
||||
0x63, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65,
|
||||
0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65,
|
||||
0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x74, 0x61,
|
||||
0x6c, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x74,
|
||||
0x6f, 0x74, 0x61, 0x6c, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x22, 0x61, 0x0a, 0x13, 0x46, 0x69, 0x6c,
|
||||
0x65, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x12, 0x1b, 0x0a, 0x09, 0x73, 0x70, 0x69, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x70, 0x69, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x12, 0x0a,
|
||||
0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74,
|
||||
0x68, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x07, 0x6e, 0x6f, 0x64, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x7f, 0x0a, 0x11,
|
||||
0x46, 0x69, 0x6c, 0x65, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x43, 0x68, 0x75, 0x6e,
|
||||
0x6b, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52,
|
||||
0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x70,
|
||||
0x6c, 0x65, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x43, 0x6f,
|
||||
0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18,
|
||||
0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1f, 0x0a, 0x0b,
|
||||
0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28,
|
||||
0x03, 0x52, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x42, 0x79, 0x74, 0x65, 0x73, 0x32, 0x99, 0x01,
|
||||
0x0a, 0x0b, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x3e, 0x0a,
|
||||
0x0e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x63, 0x61, 0x6e, 0x12,
|
||||
0x15, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x79, 0x6e, 0x63, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x69,
|
||||
0x6c, 0x65, 0x53, 0x63, 0x61, 0x6e, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x30, 0x01, 0x12, 0x4a, 0x0a,
|
||||
0x12, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x46, 0x69, 0x6c, 0x65, 0x44, 0x6f, 0x77, 0x6e, 0x6c,
|
||||
0x6f, 0x61, 0x64, 0x12, 0x19, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x44,
|
||||
0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17,
|
||||
0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f,
|
||||
0x61, 0x64, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x30, 0x01, 0x42, 0x08, 0x5a, 0x06, 0x2e, 0x3b, 0x67,
|
||||
0x72, 0x70, 0x63, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
file_services_sync_service_proto_rawDescOnce sync.Once
|
||||
file_services_sync_service_proto_rawDescData = file_services_sync_service_proto_rawDesc
|
||||
)
|
||||
|
||||
func file_services_sync_service_proto_rawDescGZIP() []byte {
|
||||
file_services_sync_service_proto_rawDescOnce.Do(func() {
|
||||
file_services_sync_service_proto_rawDescData = protoimpl.X.CompressGZIP(file_services_sync_service_proto_rawDescData)
|
||||
})
|
||||
return file_services_sync_service_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_services_sync_service_proto_msgTypes = make([]protoimpl.MessageInfo, 5)
|
||||
var file_services_sync_service_proto_goTypes = []any{
|
||||
(*FileSyncRequest)(nil), // 0: grpc.FileSyncRequest
|
||||
(*FileInfo)(nil), // 1: grpc.FileInfo
|
||||
(*FileScanChunk)(nil), // 2: grpc.FileScanChunk
|
||||
(*FileDownloadRequest)(nil), // 3: grpc.FileDownloadRequest
|
||||
(*FileDownloadChunk)(nil), // 4: grpc.FileDownloadChunk
|
||||
}
|
||||
var file_services_sync_service_proto_depIdxs = []int32{
|
||||
1, // 0: grpc.FileScanChunk.files:type_name -> grpc.FileInfo
|
||||
0, // 1: grpc.SyncService.StreamFileScan:input_type -> grpc.FileSyncRequest
|
||||
3, // 2: grpc.SyncService.StreamFileDownload:input_type -> grpc.FileDownloadRequest
|
||||
2, // 3: grpc.SyncService.StreamFileScan:output_type -> grpc.FileScanChunk
|
||||
4, // 4: grpc.SyncService.StreamFileDownload:output_type -> grpc.FileDownloadChunk
|
||||
3, // [3:5] is the sub-list for method output_type
|
||||
1, // [1:3] 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_services_sync_service_proto_init() }
|
||||
func file_services_sync_service_proto_init() {
|
||||
if File_services_sync_service_proto != nil {
|
||||
return
|
||||
}
|
||||
type x struct{}
|
||||
out := protoimpl.TypeBuilder{
|
||||
File: protoimpl.DescBuilder{
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_services_sync_service_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 5,
|
||||
NumExtensions: 0,
|
||||
NumServices: 1,
|
||||
},
|
||||
GoTypes: file_services_sync_service_proto_goTypes,
|
||||
DependencyIndexes: file_services_sync_service_proto_depIdxs,
|
||||
MessageInfos: file_services_sync_service_proto_msgTypes,
|
||||
}.Build()
|
||||
File_services_sync_service_proto = out.File
|
||||
file_services_sync_service_proto_rawDesc = nil
|
||||
file_services_sync_service_proto_goTypes = nil
|
||||
file_services_sync_service_proto_depIdxs = nil
|
||||
}
|
||||
169
grpc/sync_service_grpc.pb.go
Normal file
169
grpc/sync_service_grpc.pb.go
Normal file
@@ -0,0 +1,169 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.29.2
|
||||
// source: services/sync_service.proto
|
||||
|
||||
package grpc
|
||||
|
||||
import (
|
||||
context "context"
|
||||
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.64.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion9
|
||||
|
||||
const (
|
||||
SyncService_StreamFileScan_FullMethodName = "/grpc.SyncService/StreamFileScan"
|
||||
SyncService_StreamFileDownload_FullMethodName = "/grpc.SyncService/StreamFileDownload"
|
||||
)
|
||||
|
||||
// SyncServiceClient is the client API for SyncService 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 SyncServiceClient interface {
|
||||
// Stream file list for synchronization
|
||||
StreamFileScan(ctx context.Context, in *FileSyncRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[FileScanChunk], error)
|
||||
// Stream file download
|
||||
StreamFileDownload(ctx context.Context, in *FileDownloadRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[FileDownloadChunk], error)
|
||||
}
|
||||
|
||||
type syncServiceClient struct {
|
||||
cc grpc.ClientConnInterface
|
||||
}
|
||||
|
||||
func NewSyncServiceClient(cc grpc.ClientConnInterface) SyncServiceClient {
|
||||
return &syncServiceClient{cc}
|
||||
}
|
||||
|
||||
func (c *syncServiceClient) StreamFileScan(ctx context.Context, in *FileSyncRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[FileScanChunk], error) {
|
||||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &SyncService_ServiceDesc.Streams[0], SyncService_StreamFileScan_FullMethodName, cOpts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &grpc.GenericClientStream[FileSyncRequest, FileScanChunk]{ClientStream: 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
|
||||
}
|
||||
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type SyncService_StreamFileScanClient = grpc.ServerStreamingClient[FileScanChunk]
|
||||
|
||||
func (c *syncServiceClient) StreamFileDownload(ctx context.Context, in *FileDownloadRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[FileDownloadChunk], error) {
|
||||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &SyncService_ServiceDesc.Streams[1], SyncService_StreamFileDownload_FullMethodName, cOpts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &grpc.GenericClientStream[FileDownloadRequest, FileDownloadChunk]{ClientStream: 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
|
||||
}
|
||||
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type SyncService_StreamFileDownloadClient = grpc.ServerStreamingClient[FileDownloadChunk]
|
||||
|
||||
// SyncServiceServer is the server API for SyncService service.
|
||||
// All implementations must embed UnimplementedSyncServiceServer
|
||||
// for forward compatibility.
|
||||
type SyncServiceServer interface {
|
||||
// Stream file list for synchronization
|
||||
StreamFileScan(*FileSyncRequest, grpc.ServerStreamingServer[FileScanChunk]) error
|
||||
// Stream file download
|
||||
StreamFileDownload(*FileDownloadRequest, grpc.ServerStreamingServer[FileDownloadChunk]) error
|
||||
mustEmbedUnimplementedSyncServiceServer()
|
||||
}
|
||||
|
||||
// UnimplementedSyncServiceServer must be embedded to have
|
||||
// forward compatible implementations.
|
||||
//
|
||||
// NOTE: this should be embedded by value instead of pointer to avoid a nil
|
||||
// pointer dereference when methods are called.
|
||||
type UnimplementedSyncServiceServer struct{}
|
||||
|
||||
func (UnimplementedSyncServiceServer) StreamFileScan(*FileSyncRequest, grpc.ServerStreamingServer[FileScanChunk]) error {
|
||||
return status.Errorf(codes.Unimplemented, "method StreamFileScan not implemented")
|
||||
}
|
||||
func (UnimplementedSyncServiceServer) StreamFileDownload(*FileDownloadRequest, grpc.ServerStreamingServer[FileDownloadChunk]) error {
|
||||
return status.Errorf(codes.Unimplemented, "method StreamFileDownload not implemented")
|
||||
}
|
||||
func (UnimplementedSyncServiceServer) mustEmbedUnimplementedSyncServiceServer() {}
|
||||
func (UnimplementedSyncServiceServer) testEmbeddedByValue() {}
|
||||
|
||||
// UnsafeSyncServiceServer may be embedded to opt out of forward compatibility for this service.
|
||||
// Use of this interface is not recommended, as added methods to SyncServiceServer will
|
||||
// result in compilation errors.
|
||||
type UnsafeSyncServiceServer interface {
|
||||
mustEmbedUnimplementedSyncServiceServer()
|
||||
}
|
||||
|
||||
func RegisterSyncServiceServer(s grpc.ServiceRegistrar, srv SyncServiceServer) {
|
||||
// If the following call pancis, it indicates UnimplementedSyncServiceServer was
|
||||
// embedded by pointer and is nil. This will cause panics if an
|
||||
// unimplemented method is ever invoked, so we test this at initialization
|
||||
// time to prevent it from happening at runtime later due to I/O.
|
||||
if t, ok := srv.(interface{ testEmbeddedByValue() }); ok {
|
||||
t.testEmbeddedByValue()
|
||||
}
|
||||
s.RegisterService(&SyncService_ServiceDesc, srv)
|
||||
}
|
||||
|
||||
func _SyncService_StreamFileScan_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
m := new(FileSyncRequest)
|
||||
if err := stream.RecvMsg(m); err != nil {
|
||||
return err
|
||||
}
|
||||
return srv.(SyncServiceServer).StreamFileScan(m, &grpc.GenericServerStream[FileSyncRequest, FileScanChunk]{ServerStream: stream})
|
||||
}
|
||||
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type SyncService_StreamFileScanServer = grpc.ServerStreamingServer[FileScanChunk]
|
||||
|
||||
func _SyncService_StreamFileDownload_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
m := new(FileDownloadRequest)
|
||||
if err := stream.RecvMsg(m); err != nil {
|
||||
return err
|
||||
}
|
||||
return srv.(SyncServiceServer).StreamFileDownload(m, &grpc.GenericServerStream[FileDownloadRequest, FileDownloadChunk]{ServerStream: stream})
|
||||
}
|
||||
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type SyncService_StreamFileDownloadServer = grpc.ServerStreamingServer[FileDownloadChunk]
|
||||
|
||||
// SyncService_ServiceDesc is the grpc.ServiceDesc for SyncService service.
|
||||
// It's only intended for direct use with grpc.RegisterService,
|
||||
// and not to be introspected or modified (even as a copy)
|
||||
var SyncService_ServiceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "grpc.SyncService",
|
||||
HandlerType: (*SyncServiceServer)(nil),
|
||||
Methods: []grpc.MethodDesc{},
|
||||
Streams: []grpc.StreamDesc{
|
||||
{
|
||||
StreamName: "StreamFileScan",
|
||||
Handler: _SyncService_StreamFileScan_Handler,
|
||||
ServerStreams: true,
|
||||
},
|
||||
{
|
||||
StreamName: "StreamFileDownload",
|
||||
Handler: _SyncService_StreamFileDownload_Handler,
|
||||
ServerStreams: true,
|
||||
},
|
||||
},
|
||||
Metadata: "services/sync_service.proto",
|
||||
}
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.27.2
|
||||
// protoc-gen-go v1.36.1
|
||||
// protoc v5.29.2
|
||||
// source: services/task_service.proto
|
||||
|
||||
package grpc
|
||||
@@ -272,20 +272,17 @@ func (ProcessStatus) EnumDescriptor() ([]byte, []int) {
|
||||
}
|
||||
|
||||
type TaskServiceSubscribeRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *TaskServiceSubscribeRequest) Reset() {
|
||||
*x = TaskServiceSubscribeRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_task_service_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_task_service_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *TaskServiceSubscribeRequest) String() string {
|
||||
@@ -296,7 +293,7 @@ func (*TaskServiceSubscribeRequest) ProtoMessage() {}
|
||||
|
||||
func (x *TaskServiceSubscribeRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_task_service_proto_msgTypes[0]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -319,22 +316,19 @@ func (x *TaskServiceSubscribeRequest) GetTaskId() string {
|
||||
}
|
||||
|
||||
type TaskServiceSubscribeResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Code TaskServiceSubscribeCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.TaskServiceSubscribeCode" json:"code,omitempty"`
|
||||
TaskId string `protobuf:"bytes,2,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Code TaskServiceSubscribeCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.TaskServiceSubscribeCode" json:"code,omitempty"`
|
||||
TaskId string `protobuf:"bytes,2,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *TaskServiceSubscribeResponse) Reset() {
|
||||
*x = TaskServiceSubscribeResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_task_service_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_task_service_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *TaskServiceSubscribeResponse) String() string {
|
||||
@@ -345,7 +339,7 @@ func (*TaskServiceSubscribeResponse) ProtoMessage() {}
|
||||
|
||||
func (x *TaskServiceSubscribeResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_task_service_proto_msgTypes[1]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -382,22 +376,19 @@ func (x *TaskServiceSubscribeResponse) GetForce() bool {
|
||||
}
|
||||
|
||||
type TaskServiceConnectRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Code TaskServiceConnectCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.TaskServiceConnectCode" json:"code,omitempty"`
|
||||
TaskId string `protobuf:"bytes,2,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Code TaskServiceConnectCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.TaskServiceConnectCode" json:"code,omitempty"`
|
||||
TaskId string `protobuf:"bytes,2,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *TaskServiceConnectRequest) Reset() {
|
||||
*x = TaskServiceConnectRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_task_service_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_task_service_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *TaskServiceConnectRequest) String() string {
|
||||
@@ -408,7 +399,7 @@ func (*TaskServiceConnectRequest) ProtoMessage() {}
|
||||
|
||||
func (x *TaskServiceConnectRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_task_service_proto_msgTypes[2]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -445,22 +436,19 @@ func (x *TaskServiceConnectRequest) GetData() []byte {
|
||||
}
|
||||
|
||||
type TaskServiceConnectResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Code TaskServiceConnectResponseCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.TaskServiceConnectResponseCode" json:"code,omitempty"`
|
||||
Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
|
||||
Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Code TaskServiceConnectResponseCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.TaskServiceConnectResponseCode" json:"code,omitempty"`
|
||||
Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
|
||||
Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *TaskServiceConnectResponse) Reset() {
|
||||
*x = TaskServiceConnectResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_task_service_proto_msgTypes[3]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_task_service_proto_msgTypes[3]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *TaskServiceConnectResponse) String() string {
|
||||
@@ -471,7 +459,7 @@ func (*TaskServiceConnectResponse) ProtoMessage() {}
|
||||
|
||||
func (x *TaskServiceConnectResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_task_service_proto_msgTypes[3]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -508,20 +496,17 @@ func (x *TaskServiceConnectResponse) GetError() string {
|
||||
}
|
||||
|
||||
type TaskServiceFetchTaskRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *TaskServiceFetchTaskRequest) Reset() {
|
||||
*x = TaskServiceFetchTaskRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_task_service_proto_msgTypes[4]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_task_service_proto_msgTypes[4]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *TaskServiceFetchTaskRequest) String() string {
|
||||
@@ -532,7 +517,7 @@ func (*TaskServiceFetchTaskRequest) ProtoMessage() {}
|
||||
|
||||
func (x *TaskServiceFetchTaskRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_task_service_proto_msgTypes[4]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -555,20 +540,17 @@ func (x *TaskServiceFetchTaskRequest) GetNodeKey() string {
|
||||
}
|
||||
|
||||
type TaskServiceFetchTaskResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
TaskId string `protobuf:"bytes,2,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
TaskId string `protobuf:"bytes,2,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *TaskServiceFetchTaskResponse) Reset() {
|
||||
*x = TaskServiceFetchTaskResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_task_service_proto_msgTypes[5]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_task_service_proto_msgTypes[5]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *TaskServiceFetchTaskResponse) String() string {
|
||||
@@ -579,7 +561,7 @@ func (*TaskServiceFetchTaskResponse) ProtoMessage() {}
|
||||
|
||||
func (x *TaskServiceFetchTaskResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_task_service_proto_msgTypes[5]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -602,21 +584,18 @@ func (x *TaskServiceFetchTaskResponse) GetTaskId() string {
|
||||
}
|
||||
|
||||
type TaskServiceSendNotificationRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
TaskId string `protobuf:"bytes,2,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
NodeKey string `protobuf:"bytes,1,opt,name=node_key,json=nodeKey,proto3" json:"node_key,omitempty"`
|
||||
TaskId string `protobuf:"bytes,2,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *TaskServiceSendNotificationRequest) Reset() {
|
||||
*x = TaskServiceSendNotificationRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_task_service_proto_msgTypes[6]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_task_service_proto_msgTypes[6]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *TaskServiceSendNotificationRequest) String() string {
|
||||
@@ -627,7 +606,7 @@ func (*TaskServiceSendNotificationRequest) ProtoMessage() {}
|
||||
|
||||
func (x *TaskServiceSendNotificationRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_task_service_proto_msgTypes[6]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -657,22 +636,19 @@ func (x *TaskServiceSendNotificationRequest) GetTaskId() string {
|
||||
}
|
||||
|
||||
type TaskServiceSendNotificationResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
Code TaskServiceSendNotificationResponseCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.TaskServiceSendNotificationResponseCode" json:"code,omitempty"`
|
||||
Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
|
||||
Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Code TaskServiceSendNotificationResponseCode `protobuf:"varint,1,opt,name=code,proto3,enum=grpc.TaskServiceSendNotificationResponseCode" json:"code,omitempty"`
|
||||
Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
|
||||
Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *TaskServiceSendNotificationResponse) Reset() {
|
||||
*x = TaskServiceSendNotificationResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_task_service_proto_msgTypes[7]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_task_service_proto_msgTypes[7]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *TaskServiceSendNotificationResponse) String() string {
|
||||
@@ -683,7 +659,7 @@ func (*TaskServiceSendNotificationResponse) ProtoMessage() {}
|
||||
|
||||
func (x *TaskServiceSendNotificationResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_task_service_proto_msgTypes[7]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -720,21 +696,18 @@ func (x *TaskServiceSendNotificationResponse) GetError() string {
|
||||
}
|
||||
|
||||
type TaskServiceCheckProcessRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
Pid int32 `protobuf:"varint,2,opt,name=pid,proto3" json:"pid,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
Pid int32 `protobuf:"varint,2,opt,name=pid,proto3" json:"pid,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *TaskServiceCheckProcessRequest) Reset() {
|
||||
*x = TaskServiceCheckProcessRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_task_service_proto_msgTypes[8]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_task_service_proto_msgTypes[8]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *TaskServiceCheckProcessRequest) String() string {
|
||||
@@ -745,7 +718,7 @@ func (*TaskServiceCheckProcessRequest) ProtoMessage() {}
|
||||
|
||||
func (x *TaskServiceCheckProcessRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_task_service_proto_msgTypes[8]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -775,24 +748,21 @@ func (x *TaskServiceCheckProcessRequest) GetPid() int32 {
|
||||
}
|
||||
|
||||
type TaskServiceCheckProcessResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
state protoimpl.MessageState `protogen:"open.v1"`
|
||||
TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
Pid int32 `protobuf:"varint,2,opt,name=pid,proto3" json:"pid,omitempty"`
|
||||
Status ProcessStatus `protobuf:"varint,3,opt,name=status,proto3,enum=grpc.ProcessStatus" json:"status,omitempty"`
|
||||
ExitCode int32 `protobuf:"varint,4,opt,name=exit_code,json=exitCode,proto3" json:"exit_code,omitempty"`
|
||||
ErrorMessage string `protobuf:"bytes,5,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"`
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
|
||||
Pid int32 `protobuf:"varint,2,opt,name=pid,proto3" json:"pid,omitempty"`
|
||||
Status ProcessStatus `protobuf:"varint,3,opt,name=status,proto3,enum=grpc.ProcessStatus" json:"status,omitempty"`
|
||||
ExitCode int32 `protobuf:"varint,4,opt,name=exit_code,json=exitCode,proto3" json:"exit_code,omitempty"`
|
||||
ErrorMessage string `protobuf:"bytes,5,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"`
|
||||
sizeCache protoimpl.SizeCache
|
||||
}
|
||||
|
||||
func (x *TaskServiceCheckProcessResponse) Reset() {
|
||||
*x = TaskServiceCheckProcessResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_services_task_service_proto_msgTypes[9]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
mi := &file_services_task_service_proto_msgTypes[9]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
||||
func (x *TaskServiceCheckProcessResponse) String() string {
|
||||
@@ -803,7 +773,7 @@ func (*TaskServiceCheckProcessResponse) ProtoMessage() {}
|
||||
|
||||
func (x *TaskServiceCheckProcessResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_services_task_service_proto_msgTypes[9]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
if x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
@@ -1049,128 +1019,6 @@ func file_services_task_service_proto_init() {
|
||||
if File_services_task_service_proto != nil {
|
||||
return
|
||||
}
|
||||
if !protoimpl.UnsafeEnabled {
|
||||
file_services_task_service_proto_msgTypes[0].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*TaskServiceSubscribeRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_task_service_proto_msgTypes[1].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*TaskServiceSubscribeResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_task_service_proto_msgTypes[2].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*TaskServiceConnectRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_task_service_proto_msgTypes[3].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*TaskServiceConnectResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_task_service_proto_msgTypes[4].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*TaskServiceFetchTaskRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_task_service_proto_msgTypes[5].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*TaskServiceFetchTaskResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_task_service_proto_msgTypes[6].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*TaskServiceSendNotificationRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_task_service_proto_msgTypes[7].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*TaskServiceSendNotificationResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_task_service_proto_msgTypes[8].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*TaskServiceCheckProcessRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_services_task_service_proto_msgTypes[9].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*TaskServiceCheckProcessResponse); 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{
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.4.0
|
||||
// - protoc v5.27.2
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.29.2
|
||||
// source: services/task_service.proto
|
||||
|
||||
package grpc
|
||||
@@ -15,8 +15,8 @@ import (
|
||||
|
||||
// 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.62.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion8
|
||||
// Requires gRPC-Go v1.64.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion9
|
||||
|
||||
const (
|
||||
TaskService_Subscribe_FullMethodName = "/grpc.TaskService/Subscribe"
|
||||
@@ -30,8 +30,8 @@ const (
|
||||
//
|
||||
// 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 TaskServiceClient interface {
|
||||
Subscribe(ctx context.Context, in *TaskServiceSubscribeRequest, opts ...grpc.CallOption) (TaskService_SubscribeClient, error)
|
||||
Connect(ctx context.Context, opts ...grpc.CallOption) (TaskService_ConnectClient, error)
|
||||
Subscribe(ctx context.Context, in *TaskServiceSubscribeRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[TaskServiceSubscribeResponse], error)
|
||||
Connect(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[TaskServiceConnectRequest, TaskServiceConnectResponse], error)
|
||||
FetchTask(ctx context.Context, in *TaskServiceFetchTaskRequest, opts ...grpc.CallOption) (*TaskServiceFetchTaskResponse, error)
|
||||
SendNotification(ctx context.Context, in *TaskServiceSendNotificationRequest, opts ...grpc.CallOption) (*TaskServiceSendNotificationResponse, error)
|
||||
CheckProcess(ctx context.Context, in *TaskServiceCheckProcessRequest, opts ...grpc.CallOption) (*TaskServiceCheckProcessResponse, error)
|
||||
@@ -45,13 +45,13 @@ func NewTaskServiceClient(cc grpc.ClientConnInterface) TaskServiceClient {
|
||||
return &taskServiceClient{cc}
|
||||
}
|
||||
|
||||
func (c *taskServiceClient) Subscribe(ctx context.Context, in *TaskServiceSubscribeRequest, opts ...grpc.CallOption) (TaskService_SubscribeClient, error) {
|
||||
func (c *taskServiceClient) Subscribe(ctx context.Context, in *TaskServiceSubscribeRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[TaskServiceSubscribeResponse], error) {
|
||||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &TaskService_ServiceDesc.Streams[0], TaskService_Subscribe_FullMethodName, cOpts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &taskServiceSubscribeClient{ClientStream: stream}
|
||||
x := &grpc.GenericClientStream[TaskServiceSubscribeRequest, TaskServiceSubscribeResponse]{ClientStream: stream}
|
||||
if err := x.ClientStream.SendMsg(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -61,54 +61,21 @@ func (c *taskServiceClient) Subscribe(ctx context.Context, in *TaskServiceSubscr
|
||||
return x, nil
|
||||
}
|
||||
|
||||
type TaskService_SubscribeClient interface {
|
||||
Recv() (*TaskServiceSubscribeResponse, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type TaskService_SubscribeClient = grpc.ServerStreamingClient[TaskServiceSubscribeResponse]
|
||||
|
||||
type taskServiceSubscribeClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *taskServiceSubscribeClient) Recv() (*TaskServiceSubscribeResponse, error) {
|
||||
m := new(TaskServiceSubscribeResponse)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func (c *taskServiceClient) Connect(ctx context.Context, opts ...grpc.CallOption) (TaskService_ConnectClient, error) {
|
||||
func (c *taskServiceClient) Connect(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[TaskServiceConnectRequest, TaskServiceConnectResponse], error) {
|
||||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &TaskService_ServiceDesc.Streams[1], TaskService_Connect_FullMethodName, cOpts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &taskServiceConnectClient{ClientStream: stream}
|
||||
x := &grpc.GenericClientStream[TaskServiceConnectRequest, TaskServiceConnectResponse]{ClientStream: stream}
|
||||
return x, nil
|
||||
}
|
||||
|
||||
type TaskService_ConnectClient interface {
|
||||
Send(*TaskServiceConnectRequest) error
|
||||
Recv() (*TaskServiceConnectResponse, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
type taskServiceConnectClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *taskServiceConnectClient) Send(m *TaskServiceConnectRequest) error {
|
||||
return x.ClientStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *taskServiceConnectClient) Recv() (*TaskServiceConnectResponse, error) {
|
||||
m := new(TaskServiceConnectResponse)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type TaskService_ConnectClient = grpc.BidiStreamingClient[TaskServiceConnectRequest, TaskServiceConnectResponse]
|
||||
|
||||
func (c *taskServiceClient) FetchTask(ctx context.Context, in *TaskServiceFetchTaskRequest, opts ...grpc.CallOption) (*TaskServiceFetchTaskResponse, error) {
|
||||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||
@@ -142,24 +109,27 @@ func (c *taskServiceClient) CheckProcess(ctx context.Context, in *TaskServiceChe
|
||||
|
||||
// TaskServiceServer is the server API for TaskService service.
|
||||
// All implementations must embed UnimplementedTaskServiceServer
|
||||
// for forward compatibility
|
||||
// for forward compatibility.
|
||||
type TaskServiceServer interface {
|
||||
Subscribe(*TaskServiceSubscribeRequest, TaskService_SubscribeServer) error
|
||||
Connect(TaskService_ConnectServer) error
|
||||
Subscribe(*TaskServiceSubscribeRequest, grpc.ServerStreamingServer[TaskServiceSubscribeResponse]) error
|
||||
Connect(grpc.BidiStreamingServer[TaskServiceConnectRequest, TaskServiceConnectResponse]) error
|
||||
FetchTask(context.Context, *TaskServiceFetchTaskRequest) (*TaskServiceFetchTaskResponse, error)
|
||||
SendNotification(context.Context, *TaskServiceSendNotificationRequest) (*TaskServiceSendNotificationResponse, error)
|
||||
CheckProcess(context.Context, *TaskServiceCheckProcessRequest) (*TaskServiceCheckProcessResponse, error)
|
||||
mustEmbedUnimplementedTaskServiceServer()
|
||||
}
|
||||
|
||||
// UnimplementedTaskServiceServer must be embedded to have forward compatible implementations.
|
||||
type UnimplementedTaskServiceServer struct {
|
||||
}
|
||||
// UnimplementedTaskServiceServer must be embedded to have
|
||||
// forward compatible implementations.
|
||||
//
|
||||
// NOTE: this should be embedded by value instead of pointer to avoid a nil
|
||||
// pointer dereference when methods are called.
|
||||
type UnimplementedTaskServiceServer struct{}
|
||||
|
||||
func (UnimplementedTaskServiceServer) Subscribe(*TaskServiceSubscribeRequest, TaskService_SubscribeServer) error {
|
||||
func (UnimplementedTaskServiceServer) Subscribe(*TaskServiceSubscribeRequest, grpc.ServerStreamingServer[TaskServiceSubscribeResponse]) error {
|
||||
return status.Errorf(codes.Unimplemented, "method Subscribe not implemented")
|
||||
}
|
||||
func (UnimplementedTaskServiceServer) Connect(TaskService_ConnectServer) error {
|
||||
func (UnimplementedTaskServiceServer) Connect(grpc.BidiStreamingServer[TaskServiceConnectRequest, TaskServiceConnectResponse]) error {
|
||||
return status.Errorf(codes.Unimplemented, "method Connect not implemented")
|
||||
}
|
||||
func (UnimplementedTaskServiceServer) FetchTask(context.Context, *TaskServiceFetchTaskRequest) (*TaskServiceFetchTaskResponse, error) {
|
||||
@@ -172,6 +142,7 @@ func (UnimplementedTaskServiceServer) CheckProcess(context.Context, *TaskService
|
||||
return nil, status.Errorf(codes.Unimplemented, "method CheckProcess not implemented")
|
||||
}
|
||||
func (UnimplementedTaskServiceServer) mustEmbedUnimplementedTaskServiceServer() {}
|
||||
func (UnimplementedTaskServiceServer) testEmbeddedByValue() {}
|
||||
|
||||
// UnsafeTaskServiceServer may be embedded to opt out of forward compatibility for this service.
|
||||
// Use of this interface is not recommended, as added methods to TaskServiceServer will
|
||||
@@ -181,6 +152,13 @@ type UnsafeTaskServiceServer interface {
|
||||
}
|
||||
|
||||
func RegisterTaskServiceServer(s grpc.ServiceRegistrar, srv TaskServiceServer) {
|
||||
// If the following call pancis, it indicates UnimplementedTaskServiceServer was
|
||||
// embedded by pointer and is nil. This will cause panics if an
|
||||
// unimplemented method is ever invoked, so we test this at initialization
|
||||
// time to prevent it from happening at runtime later due to I/O.
|
||||
if t, ok := srv.(interface{ testEmbeddedByValue() }); ok {
|
||||
t.testEmbeddedByValue()
|
||||
}
|
||||
s.RegisterService(&TaskService_ServiceDesc, srv)
|
||||
}
|
||||
|
||||
@@ -189,47 +167,18 @@ func _TaskService_Subscribe_Handler(srv interface{}, stream grpc.ServerStream) e
|
||||
if err := stream.RecvMsg(m); err != nil {
|
||||
return err
|
||||
}
|
||||
return srv.(TaskServiceServer).Subscribe(m, &taskServiceSubscribeServer{ServerStream: stream})
|
||||
return srv.(TaskServiceServer).Subscribe(m, &grpc.GenericServerStream[TaskServiceSubscribeRequest, TaskServiceSubscribeResponse]{ServerStream: stream})
|
||||
}
|
||||
|
||||
type TaskService_SubscribeServer interface {
|
||||
Send(*TaskServiceSubscribeResponse) error
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type taskServiceSubscribeServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *taskServiceSubscribeServer) Send(m *TaskServiceSubscribeResponse) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type TaskService_SubscribeServer = grpc.ServerStreamingServer[TaskServiceSubscribeResponse]
|
||||
|
||||
func _TaskService_Connect_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
return srv.(TaskServiceServer).Connect(&taskServiceConnectServer{ServerStream: stream})
|
||||
return srv.(TaskServiceServer).Connect(&grpc.GenericServerStream[TaskServiceConnectRequest, TaskServiceConnectResponse]{ServerStream: stream})
|
||||
}
|
||||
|
||||
type TaskService_ConnectServer interface {
|
||||
Send(*TaskServiceConnectResponse) error
|
||||
Recv() (*TaskServiceConnectRequest, error)
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type taskServiceConnectServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *taskServiceConnectServer) Send(m *TaskServiceConnectResponse) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *taskServiceConnectServer) Recv() (*TaskServiceConnectRequest, error) {
|
||||
m := new(TaskServiceConnectRequest)
|
||||
if err := x.ServerStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||
type TaskService_ConnectServer = grpc.BidiStreamingServer[TaskServiceConnectRequest, TaskServiceConnectResponse]
|
||||
|
||||
func _TaskService_FetchTask_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(TaskServiceFetchTaskRequest)
|
||||
|
||||
Reference in New Issue
Block a user