diff --git a/agent/rpc/auth_client_grpc.go b/agent/rpc/auth_client_grpc.go
new file mode 100644
index 000000000..92bc64f3d
--- /dev/null
+++ b/agent/rpc/auth_client_grpc.go
@@ -0,0 +1,45 @@
+package rpc
+
+import (
+ "context"
+ "time"
+
+ "github.com/woodpecker-ci/woodpecker/pipeline/rpc/proto"
+
+ "google.golang.org/grpc"
+)
+
+type AuthClient struct {
+ client proto.WoodpeckerAuthClient
+ conn *grpc.ClientConn
+ agentToken string
+ agentID int64
+}
+
+func NewAuthGrpcClient(conn *grpc.ClientConn, agentToken string, agentID int64) *AuthClient {
+ client := new(AuthClient)
+ client.client = proto.NewWoodpeckerAuthClient(conn)
+ client.conn = conn
+ client.agentToken = agentToken
+ client.agentID = agentID
+ return client
+}
+
+func (c *AuthClient) Auth() (string, int64, error) {
+ ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
+ defer cancel()
+
+ req := &proto.AuthRequest{
+ AgentToken: c.agentToken,
+ AgentId: c.agentID,
+ }
+
+ res, err := c.client.Auth(ctx, req)
+ if err != nil {
+ return "", -1, err
+ }
+
+ c.agentID = res.GetAgentId()
+
+ return res.GetAccessToken(), c.agentID, nil
+}
diff --git a/agent/rpc/auth_interceptor.go b/agent/rpc/auth_interceptor.go
new file mode 100644
index 000000000..8ee40c985
--- /dev/null
+++ b/agent/rpc/auth_interceptor.go
@@ -0,0 +1,99 @@
+package rpc
+
+import (
+ "context"
+ "log"
+ "time"
+
+ "google.golang.org/grpc"
+ "google.golang.org/grpc/metadata"
+)
+
+// AuthInterceptor is a client interceptor for authentication
+type AuthInterceptor struct {
+ authClient *AuthClient
+ accessToken string
+}
+
+// NewAuthInterceptor returns a new auth interceptor
+func NewAuthInterceptor(
+ authClient *AuthClient,
+ refreshDuration time.Duration,
+) (*AuthInterceptor, error) {
+ interceptor := &AuthInterceptor{
+ authClient: authClient,
+ }
+
+ err := interceptor.scheduleRefreshToken(refreshDuration)
+ if err != nil {
+ return nil, err
+ }
+
+ return interceptor, nil
+}
+
+// Unary returns a client interceptor to authenticate unary RPC
+func (interceptor *AuthInterceptor) Unary() grpc.UnaryClientInterceptor {
+ return func(
+ ctx context.Context,
+ method string,
+ req, reply interface{},
+ cc *grpc.ClientConn,
+ invoker grpc.UnaryInvoker,
+ opts ...grpc.CallOption,
+ ) error {
+ return invoker(interceptor.attachToken(ctx), method, req, reply, cc, opts...)
+ }
+}
+
+// Stream returns a client interceptor to authenticate stream RPC
+func (interceptor *AuthInterceptor) Stream() grpc.StreamClientInterceptor {
+ return func(
+ ctx context.Context,
+ desc *grpc.StreamDesc,
+ cc *grpc.ClientConn,
+ method string,
+ streamer grpc.Streamer,
+ opts ...grpc.CallOption,
+ ) (grpc.ClientStream, error) {
+ return streamer(interceptor.attachToken(ctx), desc, cc, method, opts...)
+ }
+}
+
+func (interceptor *AuthInterceptor) attachToken(ctx context.Context) context.Context {
+ return metadata.AppendToOutgoingContext(ctx, "token", interceptor.accessToken)
+}
+
+func (interceptor *AuthInterceptor) scheduleRefreshToken(refreshDuration time.Duration) error {
+ err := interceptor.refreshToken()
+ if err != nil {
+ return err
+ }
+
+ go func() {
+ wait := refreshDuration
+ for {
+ time.Sleep(wait)
+ err := interceptor.refreshToken()
+ if err != nil {
+ wait = time.Second
+ } else {
+ wait = refreshDuration
+ }
+ }
+ }()
+
+ return nil
+}
+
+func (interceptor *AuthInterceptor) refreshToken() error {
+ accessToken, _, err := interceptor.authClient.Auth()
+ if err != nil {
+ return err
+ }
+
+ interceptor.accessToken = accessToken
+ log.Printf("Token refreshed: %v", accessToken)
+
+ return nil
+}
diff --git a/pipeline/rpc/client_grpc.go b/agent/rpc/client_grpc.go
similarity index 82%
rename from pipeline/rpc/client_grpc.go
rename to agent/rpc/client_grpc.go
index 39f4db600..3cba654de 100644
--- a/pipeline/rpc/client_grpc.go
+++ b/agent/rpc/client_grpc.go
@@ -12,6 +12,7 @@ import (
"google.golang.org/grpc/status"
backend "github.com/woodpecker-ci/woodpecker/pipeline/backend/types"
+ "github.com/woodpecker-ci/woodpecker/pipeline/rpc"
"github.com/woodpecker-ci/woodpecker/pipeline/rpc/proto"
)
@@ -23,7 +24,7 @@ type client struct {
}
// NewGrpcClient returns a new grpc Client.
-func NewGrpcClient(conn *grpc.ClientConn) Peer {
+func NewGrpcClient(conn *grpc.ClientConn) rpc.Peer {
client := new(client)
client.client = proto.NewWoodpeckerClient(conn)
client.conn = conn
@@ -35,7 +36,7 @@ func (c *client) Close() error {
}
// Next returns the next pipeline in the queue.
-func (c *client) Next(ctx context.Context, f Filter) (*Pipeline, error) {
+func (c *client) Next(ctx context.Context, f rpc.Filter) (*rpc.Pipeline, error) {
var res *proto.NextReply
var err error
req := new(proto.NextRequest)
@@ -75,7 +76,7 @@ func (c *client) Next(ctx context.Context, f Filter) (*Pipeline, error) {
return nil, nil
}
- p := new(Pipeline)
+ p := new(rpc.Pipeline)
p.ID = res.GetPipeline().GetId()
p.Timeout = res.GetPipeline().GetTimeout()
p.Config = new(backend.Config)
@@ -113,7 +114,7 @@ func (c *client) Wait(ctx context.Context, id string) (err error) {
}
// Init signals the pipeline is initialized.
-func (c *client) Init(ctx context.Context, id string, state State) (err error) {
+func (c *client) Init(ctx context.Context, id string, state rpc.State) (err error) {
req := new(proto.InitRequest)
req.Id = id
req.State = new(proto.State)
@@ -147,7 +148,7 @@ func (c *client) Init(ctx context.Context, id string, state State) (err error) {
}
// Done signals the pipeline is complete.
-func (c *client) Done(ctx context.Context, id string, state State) (err error) {
+func (c *client) Done(ctx context.Context, id string, state rpc.State) (err error) {
req := new(proto.DoneRequest)
req.Id = id
req.State = new(proto.State)
@@ -208,7 +209,7 @@ func (c *client) Extend(ctx context.Context, id string) (err error) {
}
// Update updates the pipeline state.
-func (c *client) Update(ctx context.Context, id string, state State) (err error) {
+func (c *client) Update(ctx context.Context, id string, state rpc.State) (err error) {
req := new(proto.UpdateRequest)
req.Id = id
req.State = new(proto.State)
@@ -242,7 +243,7 @@ func (c *client) Update(ctx context.Context, id string, state State) (err error)
}
// Upload uploads the pipeline artifact.
-func (c *client) Upload(ctx context.Context, id string, file *File) (err error) {
+func (c *client) Upload(ctx context.Context, id string, file *rpc.File) (err error) {
req := new(proto.UploadRequest)
req.Id = id
req.File = new(proto.File)
@@ -277,7 +278,7 @@ func (c *client) Upload(ctx context.Context, id string, file *File) (err error)
}
// Log writes the pipeline log entry.
-func (c *client) Log(ctx context.Context, id string, line *Line) (err error) {
+func (c *client) Log(ctx context.Context, id string, line *rpc.Line) (err error) {
req := new(proto.LogRequest)
req.Id = id
req.Line = new(proto.Line)
@@ -307,3 +308,38 @@ func (c *client) Log(ctx context.Context, id string, line *Line) (err error) {
}
return nil
}
+
+func (c *client) RegisterAgent(ctx context.Context, platform, backend, version string, capacity int) (int64, error) {
+ req := new(proto.RegisterAgentRequest)
+ req.Platform = platform
+ req.Backend = backend
+ req.Version = version
+ req.Capacity = int32(capacity)
+
+ res, err := c.client.RegisterAgent(ctx, req)
+ return res.GetAgentId(), err
+}
+
+func (c *client) ReportHealth(ctx context.Context) (err error) {
+ req := new(proto.ReportHealthRequest)
+ req.Status = "I am alive!"
+
+ for {
+ _, err = c.client.ReportHealth(ctx, req)
+ if err == nil {
+ return nil
+ }
+ switch status.Code(err) {
+ case
+ codes.Aborted,
+ codes.DataLoss,
+ codes.DeadlineExceeded,
+ codes.Internal,
+ codes.Unavailable:
+ // non-fatal errors
+ default:
+ return err
+ }
+ <-time.After(backoff)
+ }
+}
diff --git a/cmd/agent/agent.go b/cmd/agent/agent.go
index 225db73e0..36b119ecb 100644
--- a/cmd/agent/agent.go
+++ b/cmd/agent/agent.go
@@ -22,6 +22,7 @@ import (
"runtime"
"strings"
"sync"
+ "time"
"github.com/rs/zerolog"
"github.com/rs/zerolog/log"
@@ -34,6 +35,7 @@ import (
"google.golang.org/grpc/metadata"
"github.com/woodpecker-ci/woodpecker/agent"
+ agentRpc "github.com/woodpecker-ci/woodpecker/agent/rpc"
"github.com/woodpecker-ci/woodpecker/pipeline/backend"
"github.com/woodpecker-ci/woodpecker/pipeline/backend/types"
"github.com/woodpecker-ci/woodpecker/pipeline/rpc"
@@ -47,9 +49,11 @@ func loop(c *cli.Context) error {
hostname, _ = os.Hostname()
}
+ platform := runtime.GOOS + "/" + runtime.GOARCH
+
labels := map[string]string{
"hostname": hostname,
- "platform": runtime.GOOS + "/" + runtime.GOARCH,
+ "platform": platform,
"repo": "*", // allow all repos by default
}
@@ -95,10 +99,6 @@ func loop(c *cli.Context) error {
}()
}
- // TODO pass version information to grpc server
- // TODO authenticate to grpc server
-
- // grpc.Dial(target, ))
var transport grpc.DialOption
if c.Bool("grpc-secure") {
transport = grpc.WithTransportCredentials(grpccredentials.NewTLS(&tls.Config{InsecureSkipVerify: c.Bool("skip-insecure-grpc")}))
@@ -106,13 +106,9 @@ func loop(c *cli.Context) error {
transport = grpc.WithTransportCredentials(insecure.NewCredentials())
}
- conn, err := grpc.Dial(
+ authConn, err := grpc.Dial(
c.String("server"),
transport,
- grpc.WithPerRPCCredentials(&credentials{
- username: c.String("grpc-username"),
- password: c.String("grpc-password"),
- }),
grpc.WithKeepaliveParams(keepalive.ClientParameters{
Time: c.Duration("grpc-keepalive-time"),
Timeout: c.Duration("grpc-keepalive-timeout"),
@@ -121,9 +117,32 @@ func loop(c *cli.Context) error {
if err != nil {
return err
}
+ defer authConn.Close()
+
+ agentID := int64(-1) // TODO: store agent id in a file
+ agentToken := c.String("grpc-token")
+ authClient := agentRpc.NewAuthGrpcClient(authConn, agentToken, agentID)
+ authInterceptor, err := agentRpc.NewAuthInterceptor(authClient, 30*time.Minute)
+ if err != nil {
+ return err
+ }
+
+ conn, err := grpc.Dial(
+ c.String("server"),
+ transport,
+ grpc.WithKeepaliveParams(keepalive.ClientParameters{
+ Time: c.Duration("grpc-keepalive-time"),
+ Timeout: c.Duration("grpc-keepalive-timeout"),
+ }),
+ grpc.WithUnaryInterceptor(authInterceptor.Unary()),
+ grpc.WithStreamInterceptor(authInterceptor.Stream()),
+ )
+ if err != nil {
+ return err
+ }
defer conn.Close()
- client := rpc.NewGrpcClient(conn)
+ client := agentRpc.NewGrpcClient(conn)
sigterm := abool.New()
ctx := metadata.NewOutgoingContext(
@@ -148,6 +167,29 @@ func loop(c *cli.Context) error {
return err
}
+ agentID, err = client.RegisterAgent(ctx, platform, engine.Name(), version.String(), parallel)
+ if err != nil {
+ return err
+ }
+
+ log.Debug().Msgf("Agent registered with ID %d", agentID)
+
+ go func() {
+ for {
+ if sigterm.IsSet() {
+ return
+ }
+
+ err := client.ReportHealth(ctx)
+ if err != nil {
+ log.Err(err).Msgf("Failed to report health")
+ return
+ }
+
+ <-time.After(time.Second * 10)
+ }
+ }()
+
for i := 0; i < parallel; i++ {
go func() {
defer wg.Done()
@@ -178,25 +220,9 @@ func loop(c *cli.Context) error {
}
log.Info().Msgf(
- "Starting Woodpecker agent with version '%s' and backend '%s' running up to %d pipelines in parallel",
- version.String(), engine.Name(), parallel)
+ "Starting Woodpecker agent with version '%s' and backend '%s' using platform '%s' running up to %d pipelines in parallel",
+ version.String(), engine.Name(), platform, parallel)
wg.Wait()
return nil
}
-
-type credentials struct {
- username string
- password string
-}
-
-func (c *credentials) GetRequestMetadata(context.Context, ...string) (map[string]string, error) {
- return map[string]string{
- "username": c.username,
- "password": c.password,
- }, nil
-}
-
-func (c *credentials) RequireTransportSecurity() bool {
- return false
-}
diff --git a/cmd/agent/flags.go b/cmd/agent/flags.go
index fbb956955..9f9b6bf97 100644
--- a/cmd/agent/flags.go
+++ b/cmd/agent/flags.go
@@ -29,16 +29,10 @@ var flags = []cli.Flag{
Usage: "server address",
Value: "localhost:9000",
},
- &cli.StringFlag{
- EnvVars: []string{"WOODPECKER_USERNAME"},
- Name: "grpc-username",
- Usage: "auth username",
- Value: "x-oauth-basic",
- },
&cli.StringFlag{
EnvVars: []string{"WOODPECKER_AGENT_SECRET"},
- Name: "grpc-password",
- Usage: "server-agent shared password",
+ Name: "grpc-token",
+ Usage: "server-agent shared token",
FilePath: os.Getenv("WOODPECKER_AGENT_SECRET_FILE"),
},
&cli.BoolFlag{
diff --git a/cmd/server/server.go b/cmd/server/server.go
index 08aa70789..7a26b734d 100644
--- a/cmd/server/server.go
+++ b/cmd/server/server.go
@@ -17,7 +17,6 @@ package main
import (
"context"
"crypto/tls"
- "errors"
"net"
"net/http"
"net/http/httputil"
@@ -34,7 +33,6 @@ import (
"golang.org/x/sync/errgroup"
"google.golang.org/grpc"
"google.golang.org/grpc/keepalive"
- "google.golang.org/grpc/metadata"
"github.com/woodpecker-ci/woodpecker/pipeline/rpc/proto"
"github.com/woodpecker-ci/woodpecker/server"
@@ -135,16 +133,19 @@ func run(c *cli.Context) error {
log.Err(err).Msg("")
return err
}
- authorizer := &authorizer{
- password: c.String("agent-secret"),
- }
+
+ jwtSecret := "secret" // TODO: make configurable
+ jwtManager := woodpeckerGrpcServer.NewJWTManager(jwtSecret)
+
+ authorizer := woodpeckerGrpcServer.NewAuthorizer(jwtManager)
grpcServer := grpc.NewServer(
- grpc.StreamInterceptor(authorizer.streamInterceptor),
- grpc.UnaryInterceptor(authorizer.unaryInterceptor),
+ grpc.StreamInterceptor(authorizer.StreamInterceptor),
+ grpc.UnaryInterceptor(authorizer.UnaryInterceptor),
grpc.KeepaliveEnforcementPolicy(keepalive.EnforcementPolicy{
MinTime: c.Duration("keepalive-min-time"),
}),
)
+
woodpeckerServer := woodpeckerGrpcServer.NewWoodpeckerServer(
_forge,
server.Config.Services.Queue,
@@ -155,6 +156,13 @@ func run(c *cli.Context) error {
)
proto.RegisterWoodpeckerServer(grpcServer, woodpeckerServer)
+ woodpeckerAuthServer := woodpeckerGrpcServer.NewWoodpeckerAuthServer(
+ jwtManager,
+ server.Config.Server.AgentToken,
+ _store,
+ )
+ proto.RegisterWoodpeckerAuthServer(grpcServer, woodpeckerAuthServer)
+
err = grpcServer.Serve(lis)
if err != nil {
log.Err(err).Msg("")
@@ -315,7 +323,7 @@ func setupEvilGlobals(c *cli.Context, v store.Store, f forge.Forge) {
// server configuration
server.Config.Server.Cert = c.String("server-cert")
server.Config.Server.Key = c.String("server-key")
- server.Config.Server.Pass = c.String("agent-secret")
+ server.Config.Server.AgentToken = c.String("agent-secret")
server.Config.Server.Host = c.String("server-host")
if c.IsSet("server-dev-oauth-host") {
server.Config.Server.OAuthHost = c.String("server-dev-oauth-host")
@@ -337,31 +345,3 @@ func setupEvilGlobals(c *cli.Context, v store.Store, f forge.Forge) {
// TODO(485) temporary workaround to not hit api rate limits
server.Config.FlatPermissions = c.Bool("flat-permissions")
}
-
-type authorizer struct {
- password string
-}
-
-func (a *authorizer) streamInterceptor(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
- if err := a.authorize(stream.Context()); err != nil {
- return err
- }
- return handler(srv, stream)
-}
-
-func (a *authorizer) unaryInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) {
- if err := a.authorize(ctx); err != nil {
- return nil, err
- }
- return handler(ctx, req)
-}
-
-func (a *authorizer) authorize(ctx context.Context) error {
- if md, ok := metadata.FromIncomingContext(ctx); ok {
- if len(md["password"]) > 0 && md["password"][0] == a.password {
- return nil
- }
- return errors.New("invalid agent token")
- }
- return errors.New("missing agent token")
-}
diff --git a/pipeline/rpc/client_grpc_health.go b/pipeline/rpc/client_grpc_health.go
deleted file mode 100644
index b14eeb250..000000000
--- a/pipeline/rpc/client_grpc_health.go
+++ /dev/null
@@ -1,60 +0,0 @@
-package rpc
-
-import (
- "context"
- "time"
-
- "google.golang.org/grpc"
- "google.golang.org/grpc/codes"
- "google.golang.org/grpc/status"
-
- "github.com/woodpecker-ci/woodpecker/pipeline/rpc/proto"
-)
-
-// generate protobuffs
-// protoc --go_out=plugins=grpc,import_path=proto:. *.proto
-
-type healthClient struct {
- client proto.HealthClient
- conn *grpc.ClientConn
-}
-
-// NewGrpcHealthClient returns a new grpc Client.
-func NewGrpcHealthClient(conn *grpc.ClientConn) Health {
- client := new(healthClient)
- client.client = proto.NewHealthClient(conn)
- client.conn = conn
- return client
-}
-
-func (c *healthClient) Close() error {
- return c.conn.Close()
-}
-
-func (c *healthClient) Check(ctx context.Context) (bool, error) {
- var res *proto.HealthCheckResponse
- var err error
- req := new(proto.HealthCheckRequest)
-
- for {
- res, err = c.client.Check(ctx, req)
- if err == nil {
- if res.GetStatus() == proto.HealthCheckResponse_SERVING {
- return true, nil
- }
- return false, nil
- }
- switch status.Code(err) {
- case
- codes.Aborted,
- codes.DataLoss,
- codes.DeadlineExceeded,
- codes.Internal,
- codes.Unavailable:
- // non-fatal errors
- default:
- return false, err
- }
- <-time.After(backoff)
- }
-}
diff --git a/pipeline/rpc/health.go b/pipeline/rpc/health.go
deleted file mode 100644
index c074d7546..000000000
--- a/pipeline/rpc/health.go
+++ /dev/null
@@ -1,11 +0,0 @@
-package rpc
-
-import (
- "context"
-)
-
-// Health defines a health-check connection.
-type Health interface {
- // Check returns if server is healthy or not
- Check(c context.Context) (bool, error)
-}
diff --git a/pipeline/rpc/peer.go b/pipeline/rpc/peer.go
index 8ee6d4ef5..4739a5ee1 100644
--- a/pipeline/rpc/peer.go
+++ b/pipeline/rpc/peer.go
@@ -66,4 +66,10 @@ type Peer interface {
// Log writes the pipeline log entry.
Log(c context.Context, id string, line *Line) error
+
+ // RegisterAgent register our agent to the server
+ RegisterAgent(ctx context.Context, platform, backend, version string, capacity int) (int64, error)
+
+ // ReportHealth reports health status of the agent to the server
+ ReportHealth(c context.Context) error
}
diff --git a/pipeline/rpc/proto/woodpecker.pb.go b/pipeline/rpc/proto/woodpecker.pb.go
index b2a53295a..3d1761c55 100644
--- a/pipeline/rpc/proto/woodpecker.pb.go
+++ b/pipeline/rpc/proto/woodpecker.pb.go
@@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
-// protoc-gen-go v1.28.1
-// protoc v3.21.7
+// protoc-gen-go v1.28.0
+// protoc v3.12.4
// source: woodpecker.proto
package proto
@@ -20,55 +20,6 @@ const (
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
)
-type HealthCheckResponse_ServingStatus int32
-
-const (
- HealthCheckResponse_UNKNOWN HealthCheckResponse_ServingStatus = 0
- HealthCheckResponse_SERVING HealthCheckResponse_ServingStatus = 1
- HealthCheckResponse_NOT_SERVING HealthCheckResponse_ServingStatus = 2
-)
-
-// Enum value maps for HealthCheckResponse_ServingStatus.
-var (
- HealthCheckResponse_ServingStatus_name = map[int32]string{
- 0: "UNKNOWN",
- 1: "SERVING",
- 2: "NOT_SERVING",
- }
- HealthCheckResponse_ServingStatus_value = map[string]int32{
- "UNKNOWN": 0,
- "SERVING": 1,
- "NOT_SERVING": 2,
- }
-)
-
-func (x HealthCheckResponse_ServingStatus) Enum() *HealthCheckResponse_ServingStatus {
- p := new(HealthCheckResponse_ServingStatus)
- *p = x
- return p
-}
-
-func (x HealthCheckResponse_ServingStatus) String() string {
- return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
-}
-
-func (HealthCheckResponse_ServingStatus) Descriptor() protoreflect.EnumDescriptor {
- return file_woodpecker_proto_enumTypes[0].Descriptor()
-}
-
-func (HealthCheckResponse_ServingStatus) Type() protoreflect.EnumType {
- return &file_woodpecker_proto_enumTypes[0]
-}
-
-func (x HealthCheckResponse_ServingStatus) Number() protoreflect.EnumNumber {
- return protoreflect.EnumNumber(x)
-}
-
-// Deprecated: Use HealthCheckResponse_ServingStatus.Descriptor instead.
-func (HealthCheckResponse_ServingStatus) EnumDescriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{6, 0}
-}
-
type File struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -432,100 +383,6 @@ func (x *Pipeline) GetPayload() []byte {
return nil
}
-type HealthCheckRequest struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- Service string `protobuf:"bytes,1,opt,name=service,proto3" json:"service,omitempty"`
-}
-
-func (x *HealthCheckRequest) Reset() {
- *x = HealthCheckRequest{}
- if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[5]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
-}
-
-func (x *HealthCheckRequest) String() string {
- return protoimpl.X.MessageStringOf(x)
-}
-
-func (*HealthCheckRequest) ProtoMessage() {}
-
-func (x *HealthCheckRequest) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[5]
- if protoimpl.UnsafeEnabled && x != nil {
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- if ms.LoadMessageInfo() == nil {
- ms.StoreMessageInfo(mi)
- }
- return ms
- }
- return mi.MessageOf(x)
-}
-
-// Deprecated: Use HealthCheckRequest.ProtoReflect.Descriptor instead.
-func (*HealthCheckRequest) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{5}
-}
-
-func (x *HealthCheckRequest) GetService() string {
- if x != nil {
- return x.Service
- }
- return ""
-}
-
-type HealthCheckResponse struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- Status HealthCheckResponse_ServingStatus `protobuf:"varint,1,opt,name=status,proto3,enum=proto.HealthCheckResponse_ServingStatus" json:"status,omitempty"`
-}
-
-func (x *HealthCheckResponse) Reset() {
- *x = HealthCheckResponse{}
- if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[6]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
-}
-
-func (x *HealthCheckResponse) String() string {
- return protoimpl.X.MessageStringOf(x)
-}
-
-func (*HealthCheckResponse) ProtoMessage() {}
-
-func (x *HealthCheckResponse) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[6]
- if protoimpl.UnsafeEnabled && x != nil {
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- if ms.LoadMessageInfo() == nil {
- ms.StoreMessageInfo(mi)
- }
- return ms
- }
- return mi.MessageOf(x)
-}
-
-// Deprecated: Use HealthCheckResponse.ProtoReflect.Descriptor instead.
-func (*HealthCheckResponse) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{6}
-}
-
-func (x *HealthCheckResponse) GetStatus() HealthCheckResponse_ServingStatus {
- if x != nil {
- return x.Status
- }
- return HealthCheckResponse_UNKNOWN
-}
-
type NextRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -537,7 +394,7 @@ type NextRequest struct {
func (x *NextRequest) Reset() {
*x = NextRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[7]
+ mi := &file_woodpecker_proto_msgTypes[5]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -550,7 +407,7 @@ func (x *NextRequest) String() string {
func (*NextRequest) ProtoMessage() {}
func (x *NextRequest) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[7]
+ mi := &file_woodpecker_proto_msgTypes[5]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -563,7 +420,7 @@ func (x *NextRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use NextRequest.ProtoReflect.Descriptor instead.
func (*NextRequest) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{7}
+ return file_woodpecker_proto_rawDescGZIP(), []int{5}
}
func (x *NextRequest) GetFilter() *Filter {
@@ -584,7 +441,7 @@ type NextReply struct {
func (x *NextReply) Reset() {
*x = NextReply{}
if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[8]
+ mi := &file_woodpecker_proto_msgTypes[6]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -597,7 +454,7 @@ func (x *NextReply) String() string {
func (*NextReply) ProtoMessage() {}
func (x *NextReply) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[8]
+ mi := &file_woodpecker_proto_msgTypes[6]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -610,7 +467,7 @@ func (x *NextReply) ProtoReflect() protoreflect.Message {
// Deprecated: Use NextReply.ProtoReflect.Descriptor instead.
func (*NextReply) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{8}
+ return file_woodpecker_proto_rawDescGZIP(), []int{6}
}
func (x *NextReply) GetPipeline() *Pipeline {
@@ -632,7 +489,7 @@ type InitRequest struct {
func (x *InitRequest) Reset() {
*x = InitRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[9]
+ mi := &file_woodpecker_proto_msgTypes[7]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -645,7 +502,7 @@ func (x *InitRequest) String() string {
func (*InitRequest) ProtoMessage() {}
func (x *InitRequest) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[9]
+ mi := &file_woodpecker_proto_msgTypes[7]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -658,7 +515,7 @@ func (x *InitRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use InitRequest.ProtoReflect.Descriptor instead.
func (*InitRequest) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{9}
+ return file_woodpecker_proto_rawDescGZIP(), []int{7}
}
func (x *InitRequest) GetId() string {
@@ -686,7 +543,7 @@ type WaitRequest struct {
func (x *WaitRequest) Reset() {
*x = WaitRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[10]
+ mi := &file_woodpecker_proto_msgTypes[8]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -699,7 +556,7 @@ func (x *WaitRequest) String() string {
func (*WaitRequest) ProtoMessage() {}
func (x *WaitRequest) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[10]
+ mi := &file_woodpecker_proto_msgTypes[8]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -712,7 +569,7 @@ func (x *WaitRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use WaitRequest.ProtoReflect.Descriptor instead.
func (*WaitRequest) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{10}
+ return file_woodpecker_proto_rawDescGZIP(), []int{8}
}
func (x *WaitRequest) GetId() string {
@@ -734,7 +591,7 @@ type DoneRequest struct {
func (x *DoneRequest) Reset() {
*x = DoneRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[11]
+ mi := &file_woodpecker_proto_msgTypes[9]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -747,7 +604,7 @@ func (x *DoneRequest) String() string {
func (*DoneRequest) ProtoMessage() {}
func (x *DoneRequest) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[11]
+ mi := &file_woodpecker_proto_msgTypes[9]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -760,7 +617,7 @@ func (x *DoneRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use DoneRequest.ProtoReflect.Descriptor instead.
func (*DoneRequest) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{11}
+ return file_woodpecker_proto_rawDescGZIP(), []int{9}
}
func (x *DoneRequest) GetId() string {
@@ -788,7 +645,7 @@ type ExtendRequest struct {
func (x *ExtendRequest) Reset() {
*x = ExtendRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[12]
+ mi := &file_woodpecker_proto_msgTypes[10]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -801,7 +658,7 @@ func (x *ExtendRequest) String() string {
func (*ExtendRequest) ProtoMessage() {}
func (x *ExtendRequest) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[12]
+ mi := &file_woodpecker_proto_msgTypes[10]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -814,7 +671,7 @@ func (x *ExtendRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use ExtendRequest.ProtoReflect.Descriptor instead.
func (*ExtendRequest) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{12}
+ return file_woodpecker_proto_rawDescGZIP(), []int{10}
}
func (x *ExtendRequest) GetId() string {
@@ -836,7 +693,7 @@ type UploadRequest struct {
func (x *UploadRequest) Reset() {
*x = UploadRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[13]
+ mi := &file_woodpecker_proto_msgTypes[11]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -849,7 +706,7 @@ func (x *UploadRequest) String() string {
func (*UploadRequest) ProtoMessage() {}
func (x *UploadRequest) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[13]
+ mi := &file_woodpecker_proto_msgTypes[11]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -862,7 +719,7 @@ func (x *UploadRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use UploadRequest.ProtoReflect.Descriptor instead.
func (*UploadRequest) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{13}
+ return file_woodpecker_proto_rawDescGZIP(), []int{11}
}
func (x *UploadRequest) GetId() string {
@@ -891,7 +748,7 @@ type UpdateRequest struct {
func (x *UpdateRequest) Reset() {
*x = UpdateRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[14]
+ mi := &file_woodpecker_proto_msgTypes[12]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -904,7 +761,7 @@ func (x *UpdateRequest) String() string {
func (*UpdateRequest) ProtoMessage() {}
func (x *UpdateRequest) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[14]
+ mi := &file_woodpecker_proto_msgTypes[12]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -917,7 +774,7 @@ func (x *UpdateRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use UpdateRequest.ProtoReflect.Descriptor instead.
func (*UpdateRequest) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{14}
+ return file_woodpecker_proto_rawDescGZIP(), []int{12}
}
func (x *UpdateRequest) GetId() string {
@@ -946,7 +803,7 @@ type LogRequest struct {
func (x *LogRequest) Reset() {
*x = LogRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[15]
+ mi := &file_woodpecker_proto_msgTypes[13]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -959,7 +816,7 @@ func (x *LogRequest) String() string {
func (*LogRequest) ProtoMessage() {}
func (x *LogRequest) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[15]
+ mi := &file_woodpecker_proto_msgTypes[13]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -972,7 +829,7 @@ func (x *LogRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use LogRequest.ProtoReflect.Descriptor instead.
func (*LogRequest) Descriptor() ([]byte, []int) {
- return file_woodpecker_proto_rawDescGZIP(), []int{15}
+ return file_woodpecker_proto_rawDescGZIP(), []int{13}
}
func (x *LogRequest) GetId() string {
@@ -998,7 +855,7 @@ type Empty struct {
func (x *Empty) Reset() {
*x = Empty{}
if protoimpl.UnsafeEnabled {
- mi := &file_woodpecker_proto_msgTypes[16]
+ mi := &file_woodpecker_proto_msgTypes[14]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1011,7 +868,7 @@ func (x *Empty) String() string {
func (*Empty) ProtoMessage() {}
func (x *Empty) ProtoReflect() protoreflect.Message {
- mi := &file_woodpecker_proto_msgTypes[16]
+ mi := &file_woodpecker_proto_msgTypes[14]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1024,9 +881,292 @@ func (x *Empty) ProtoReflect() protoreflect.Message {
// Deprecated: Use Empty.ProtoReflect.Descriptor instead.
func (*Empty) Descriptor() ([]byte, []int) {
+ return file_woodpecker_proto_rawDescGZIP(), []int{14}
+}
+
+type ReportHealthRequest struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ Status string `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
+}
+
+func (x *ReportHealthRequest) Reset() {
+ *x = ReportHealthRequest{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_woodpecker_proto_msgTypes[15]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *ReportHealthRequest) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ReportHealthRequest) ProtoMessage() {}
+
+func (x *ReportHealthRequest) ProtoReflect() protoreflect.Message {
+ mi := &file_woodpecker_proto_msgTypes[15]
+ if protoimpl.UnsafeEnabled && x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use ReportHealthRequest.ProtoReflect.Descriptor instead.
+func (*ReportHealthRequest) Descriptor() ([]byte, []int) {
+ return file_woodpecker_proto_rawDescGZIP(), []int{15}
+}
+
+func (x *ReportHealthRequest) GetStatus() string {
+ if x != nil {
+ return x.Status
+ }
+ return ""
+}
+
+type RegisterAgentRequest struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ Platform string `protobuf:"bytes,1,opt,name=platform,proto3" json:"platform,omitempty"`
+ Capacity int32 `protobuf:"varint,2,opt,name=capacity,proto3" json:"capacity,omitempty"`
+ Backend string `protobuf:"bytes,3,opt,name=backend,proto3" json:"backend,omitempty"`
+ Version string `protobuf:"bytes,4,opt,name=version,proto3" json:"version,omitempty"`
+}
+
+func (x *RegisterAgentRequest) Reset() {
+ *x = RegisterAgentRequest{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_woodpecker_proto_msgTypes[16]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *RegisterAgentRequest) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*RegisterAgentRequest) ProtoMessage() {}
+
+func (x *RegisterAgentRequest) ProtoReflect() protoreflect.Message {
+ mi := &file_woodpecker_proto_msgTypes[16]
+ if protoimpl.UnsafeEnabled && x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use RegisterAgentRequest.ProtoReflect.Descriptor instead.
+func (*RegisterAgentRequest) Descriptor() ([]byte, []int) {
return file_woodpecker_proto_rawDescGZIP(), []int{16}
}
+func (x *RegisterAgentRequest) GetPlatform() string {
+ if x != nil {
+ return x.Platform
+ }
+ return ""
+}
+
+func (x *RegisterAgentRequest) GetCapacity() int32 {
+ if x != nil {
+ return x.Capacity
+ }
+ return 0
+}
+
+func (x *RegisterAgentRequest) GetBackend() string {
+ if x != nil {
+ return x.Backend
+ }
+ return ""
+}
+
+func (x *RegisterAgentRequest) GetVersion() string {
+ if x != nil {
+ return x.Version
+ }
+ return ""
+}
+
+type RegisterAgentResponse struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ AgentId int64 `protobuf:"varint,1,opt,name=agent_id,json=agentId,proto3" json:"agent_id,omitempty"`
+}
+
+func (x *RegisterAgentResponse) Reset() {
+ *x = RegisterAgentResponse{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_woodpecker_proto_msgTypes[17]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *RegisterAgentResponse) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*RegisterAgentResponse) ProtoMessage() {}
+
+func (x *RegisterAgentResponse) ProtoReflect() protoreflect.Message {
+ mi := &file_woodpecker_proto_msgTypes[17]
+ if protoimpl.UnsafeEnabled && x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use RegisterAgentResponse.ProtoReflect.Descriptor instead.
+func (*RegisterAgentResponse) Descriptor() ([]byte, []int) {
+ return file_woodpecker_proto_rawDescGZIP(), []int{17}
+}
+
+func (x *RegisterAgentResponse) GetAgentId() int64 {
+ if x != nil {
+ return x.AgentId
+ }
+ return 0
+}
+
+type AuthRequest struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ AgentToken string `protobuf:"bytes,1,opt,name=agent_token,json=agentToken,proto3" json:"agent_token,omitempty"`
+ AgentId int64 `protobuf:"varint,2,opt,name=agent_id,json=agentId,proto3" json:"agent_id,omitempty"`
+}
+
+func (x *AuthRequest) Reset() {
+ *x = AuthRequest{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_woodpecker_proto_msgTypes[18]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *AuthRequest) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*AuthRequest) ProtoMessage() {}
+
+func (x *AuthRequest) ProtoReflect() protoreflect.Message {
+ mi := &file_woodpecker_proto_msgTypes[18]
+ if protoimpl.UnsafeEnabled && x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use AuthRequest.ProtoReflect.Descriptor instead.
+func (*AuthRequest) Descriptor() ([]byte, []int) {
+ return file_woodpecker_proto_rawDescGZIP(), []int{18}
+}
+
+func (x *AuthRequest) GetAgentToken() string {
+ if x != nil {
+ return x.AgentToken
+ }
+ return ""
+}
+
+func (x *AuthRequest) GetAgentId() int64 {
+ if x != nil {
+ return x.AgentId
+ }
+ return 0
+}
+
+type AuthReply struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ Status string `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
+ AgentId int64 `protobuf:"varint,2,opt,name=agent_id,json=agentId,proto3" json:"agent_id,omitempty"`
+ AccessToken string `protobuf:"bytes,3,opt,name=access_token,json=accessToken,proto3" json:"access_token,omitempty"`
+}
+
+func (x *AuthReply) Reset() {
+ *x = AuthReply{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_woodpecker_proto_msgTypes[19]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *AuthReply) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*AuthReply) ProtoMessage() {}
+
+func (x *AuthReply) ProtoReflect() protoreflect.Message {
+ mi := &file_woodpecker_proto_msgTypes[19]
+ if protoimpl.UnsafeEnabled && x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use AuthReply.ProtoReflect.Descriptor instead.
+func (*AuthReply) Descriptor() ([]byte, []int) {
+ return file_woodpecker_proto_rawDescGZIP(), []int{19}
+}
+
+func (x *AuthReply) GetStatus() string {
+ if x != nil {
+ return x.Status
+ }
+ return ""
+}
+
+func (x *AuthReply) GetAgentId() int64 {
+ if x != nil {
+ return x.AgentId
+ }
+ return 0
+}
+
+func (x *AuthReply) GetAccessToken() string {
+ if x != nil {
+ return x.AccessToken
+ }
+ return ""
+}
+
var File_woodpecker_proto protoreflect.FileDescriptor
var file_woodpecker_proto_rawDesc = []byte{
@@ -1073,52 +1213,65 @@ var file_woodpecker_proto_rawDesc = []byte{
0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74,
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12,
0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c,
- 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x2e, 0x0a, 0x12, 0x48, 0x65, 0x61,
- 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
- 0x18, 0x0a, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x22, 0x93, 0x01, 0x0a, 0x13, 0x48, 0x65,
- 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
- 0x65, 0x12, 0x40, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28,
- 0x0e, 0x32, 0x28, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68,
- 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x65,
- 0x72, 0x76, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61,
- 0x74, 0x75, 0x73, 0x22, 0x3a, 0x0a, 0x0d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x53, 0x74,
- 0x61, 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10,
- 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x45, 0x52, 0x56, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0f,
- 0x0a, 0x0b, 0x4e, 0x4f, 0x54, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x22,
- 0x34, 0x0a, 0x0b, 0x4e, 0x65, 0x78, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25,
- 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d,
- 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x06, 0x66,
- 0x69, 0x6c, 0x74, 0x65, 0x72, 0x22, 0x38, 0x0a, 0x09, 0x4e, 0x65, 0x78, 0x74, 0x52, 0x65, 0x70,
- 0x6c, 0x79, 0x12, 0x2b, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x50, 0x69, 0x70,
- 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22,
- 0x41, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e,
- 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x22,
- 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e,
- 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61,
- 0x74, 0x65, 0x22, 0x1d, 0x0a, 0x0b, 0x57, 0x61, 0x69, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
- 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69,
- 0x64, 0x22, 0x41, 0x0a, 0x0b, 0x44, 0x6f, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
- 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64,
- 0x12, 0x22, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
- 0x0c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73,
- 0x74, 0x61, 0x74, 0x65, 0x22, 0x1f, 0x0a, 0x0d, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x52, 0x65,
- 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
- 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x40, 0x0a, 0x0d, 0x55, 0x70, 0x6c, 0x6f, 0x61, 0x64, 0x52,
- 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x1f, 0x0a, 0x04, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x02,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x46, 0x69, 0x6c,
- 0x65, 0x52, 0x04, 0x66, 0x69, 0x6c, 0x65, 0x22, 0x43, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74,
- 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01,
+ 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x34, 0x0a, 0x0b, 0x4e, 0x65, 0x78,
+ 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74,
+ 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+ 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x22,
+ 0x38, 0x0a, 0x09, 0x4e, 0x65, 0x78, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x12, 0x2b, 0x0a, 0x08,
+ 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f,
+ 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52,
+ 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x41, 0x0a, 0x0b, 0x49, 0x6e, 0x69,
+ 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01,
0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x22, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74,
0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
- 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x3d, 0x0a, 0x0a,
- 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64,
- 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x1f, 0x0a, 0x04, 0x6c, 0x69,
- 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
- 0x2e, 0x4c, 0x69, 0x6e, 0x65, 0x52, 0x04, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x07, 0x0a, 0x05, 0x45,
- 0x6d, 0x70, 0x74, 0x79, 0x32, 0xfa, 0x02, 0x0a, 0x0a, 0x57, 0x6f, 0x6f, 0x64, 0x70, 0x65, 0x63,
+ 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x1d, 0x0a, 0x0b,
+ 0x57, 0x61, 0x69, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69,
+ 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x41, 0x0a, 0x0b, 0x44,
+ 0x6f, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x22, 0x0a, 0x05, 0x73, 0x74,
+ 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+ 0x6f, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x1f,
+ 0x0a, 0x0d, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
+ 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22,
+ 0x40, 0x0a, 0x0d, 0x55, 0x70, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64,
+ 0x12, 0x1f, 0x0a, 0x04, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b,
+ 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x04, 0x66, 0x69, 0x6c,
+ 0x65, 0x22, 0x43, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
+ 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02,
+ 0x69, 0x64, 0x12, 0x22, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,
+ 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52,
+ 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x3d, 0x0a, 0x0a, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
+ 0x52, 0x02, 0x69, 0x64, 0x12, 0x1f, 0x0a, 0x04, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01,
+ 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x6e, 0x65, 0x52,
+ 0x04, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x07, 0x0a, 0x05, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x2d,
+ 0x0a, 0x13, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18,
+ 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x82, 0x01,
+ 0x0a, 0x14, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x52,
+ 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f,
+ 0x72, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f,
+ 0x72, 0x6d, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x18, 0x02,
+ 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x12, 0x18,
+ 0x0a, 0x07, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52,
+ 0x07, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73,
+ 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69,
+ 0x6f, 0x6e, 0x22, 0x32, 0x0a, 0x15, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x41, 0x67,
+ 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x61,
+ 0x67, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x61,
+ 0x67, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x49, 0x0a, 0x0b, 0x41, 0x75, 0x74, 0x68, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x5f, 0x74,
+ 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x67, 0x65, 0x6e,
+ 0x74, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x5f,
+ 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x49,
+ 0x64, 0x22, 0x61, 0x0a, 0x09, 0x41, 0x75, 0x74, 0x68, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x12, 0x16,
+ 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06,
+ 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x5f,
+ 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x49,
+ 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x74, 0x6f, 0x6b, 0x65,
+ 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x54,
+ 0x6f, 0x6b, 0x65, 0x6e, 0x32, 0x84, 0x04, 0x0a, 0x0a, 0x57, 0x6f, 0x6f, 0x64, 0x70, 0x65, 0x63,
0x6b, 0x65, 0x72, 0x12, 0x2e, 0x0a, 0x04, 0x4e, 0x65, 0x78, 0x74, 0x12, 0x12, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x4e, 0x65, 0x78, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x10, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4e, 0x65, 0x78, 0x74, 0x52, 0x65, 0x70, 0x6c,
@@ -1142,15 +1295,23 @@ var file_woodpecker_proto_rawDesc = []byte{
0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, 0x12, 0x28, 0x0a, 0x03, 0x4c, 0x6f, 0x67, 0x12, 0x11,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x1a, 0x0c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22,
- 0x00, 0x32, 0x48, 0x0a, 0x06, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x3e, 0x0a, 0x05, 0x43,
- 0x68, 0x65, 0x63, 0x6b, 0x12, 0x19, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61,
- 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
- 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68,
- 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x38, 0x5a, 0x36, 0x67,
- 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x77, 0x6f, 0x6f, 0x64, 0x70, 0x65,
- 0x63, 0x6b, 0x65, 0x72, 0x2d, 0x63, 0x69, 0x2f, 0x77, 0x6f, 0x6f, 0x64, 0x70, 0x65, 0x63, 0x6b,
- 0x65, 0x72, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x72, 0x70, 0x63, 0x2f,
- 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+ 0x00, 0x12, 0x4c, 0x0a, 0x0d, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x41, 0x67, 0x65,
+ 0x6e, 0x74, 0x12, 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73,
+ 0x74, 0x65, 0x72, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
+ 0x1c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72,
+ 0x41, 0x67, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
+ 0x3a, 0x0a, 0x0c, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12,
+ 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x48, 0x65,
+ 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0c, 0x2e, 0x70, 0x72,
+ 0x6f, 0x74, 0x6f, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, 0x32, 0x40, 0x0a, 0x0e, 0x57,
+ 0x6f, 0x6f, 0x64, 0x70, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x41, 0x75, 0x74, 0x68, 0x12, 0x2e, 0x0a,
+ 0x04, 0x41, 0x75, 0x74, 0x68, 0x12, 0x12, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x41, 0x75,
+ 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+ 0x6f, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x42, 0x38, 0x5a,
+ 0x36, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x77, 0x6f, 0x6f, 0x64,
+ 0x70, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x2d, 0x63, 0x69, 0x2f, 0x77, 0x6f, 0x6f, 0x64, 0x70, 0x65,
+ 0x63, 0x6b, 0x65, 0x72, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x72, 0x70,
+ 0x63, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@@ -1165,64 +1326,68 @@ func file_woodpecker_proto_rawDescGZIP() []byte {
return file_woodpecker_proto_rawDescData
}
-var file_woodpecker_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
-var file_woodpecker_proto_msgTypes = make([]protoimpl.MessageInfo, 19)
+var file_woodpecker_proto_msgTypes = make([]protoimpl.MessageInfo, 22)
var file_woodpecker_proto_goTypes = []interface{}{
- (HealthCheckResponse_ServingStatus)(0), // 0: proto.HealthCheckResponse.ServingStatus
- (*File)(nil), // 1: proto.File
- (*State)(nil), // 2: proto.State
- (*Line)(nil), // 3: proto.Line
- (*Filter)(nil), // 4: proto.Filter
- (*Pipeline)(nil), // 5: proto.Pipeline
- (*HealthCheckRequest)(nil), // 6: proto.HealthCheckRequest
- (*HealthCheckResponse)(nil), // 7: proto.HealthCheckResponse
- (*NextRequest)(nil), // 8: proto.NextRequest
- (*NextReply)(nil), // 9: proto.NextReply
- (*InitRequest)(nil), // 10: proto.InitRequest
- (*WaitRequest)(nil), // 11: proto.WaitRequest
- (*DoneRequest)(nil), // 12: proto.DoneRequest
- (*ExtendRequest)(nil), // 13: proto.ExtendRequest
- (*UploadRequest)(nil), // 14: proto.UploadRequest
- (*UpdateRequest)(nil), // 15: proto.UpdateRequest
- (*LogRequest)(nil), // 16: proto.LogRequest
- (*Empty)(nil), // 17: proto.Empty
- nil, // 18: proto.File.MetaEntry
- nil, // 19: proto.Filter.LabelsEntry
+ (*File)(nil), // 0: proto.File
+ (*State)(nil), // 1: proto.State
+ (*Line)(nil), // 2: proto.Line
+ (*Filter)(nil), // 3: proto.Filter
+ (*Pipeline)(nil), // 4: proto.Pipeline
+ (*NextRequest)(nil), // 5: proto.NextRequest
+ (*NextReply)(nil), // 6: proto.NextReply
+ (*InitRequest)(nil), // 7: proto.InitRequest
+ (*WaitRequest)(nil), // 8: proto.WaitRequest
+ (*DoneRequest)(nil), // 9: proto.DoneRequest
+ (*ExtendRequest)(nil), // 10: proto.ExtendRequest
+ (*UploadRequest)(nil), // 11: proto.UploadRequest
+ (*UpdateRequest)(nil), // 12: proto.UpdateRequest
+ (*LogRequest)(nil), // 13: proto.LogRequest
+ (*Empty)(nil), // 14: proto.Empty
+ (*ReportHealthRequest)(nil), // 15: proto.ReportHealthRequest
+ (*RegisterAgentRequest)(nil), // 16: proto.RegisterAgentRequest
+ (*RegisterAgentResponse)(nil), // 17: proto.RegisterAgentResponse
+ (*AuthRequest)(nil), // 18: proto.AuthRequest
+ (*AuthReply)(nil), // 19: proto.AuthReply
+ nil, // 20: proto.File.MetaEntry
+ nil, // 21: proto.Filter.LabelsEntry
}
var file_woodpecker_proto_depIdxs = []int32{
- 18, // 0: proto.File.meta:type_name -> proto.File.MetaEntry
- 19, // 1: proto.Filter.labels:type_name -> proto.Filter.LabelsEntry
- 0, // 2: proto.HealthCheckResponse.status:type_name -> proto.HealthCheckResponse.ServingStatus
- 4, // 3: proto.NextRequest.filter:type_name -> proto.Filter
- 5, // 4: proto.NextReply.pipeline:type_name -> proto.Pipeline
- 2, // 5: proto.InitRequest.state:type_name -> proto.State
- 2, // 6: proto.DoneRequest.state:type_name -> proto.State
- 1, // 7: proto.UploadRequest.file:type_name -> proto.File
- 2, // 8: proto.UpdateRequest.state:type_name -> proto.State
- 3, // 9: proto.LogRequest.line:type_name -> proto.Line
- 8, // 10: proto.Woodpecker.Next:input_type -> proto.NextRequest
- 10, // 11: proto.Woodpecker.Init:input_type -> proto.InitRequest
- 11, // 12: proto.Woodpecker.Wait:input_type -> proto.WaitRequest
- 12, // 13: proto.Woodpecker.Done:input_type -> proto.DoneRequest
- 13, // 14: proto.Woodpecker.Extend:input_type -> proto.ExtendRequest
- 15, // 15: proto.Woodpecker.Update:input_type -> proto.UpdateRequest
- 14, // 16: proto.Woodpecker.Upload:input_type -> proto.UploadRequest
- 16, // 17: proto.Woodpecker.Log:input_type -> proto.LogRequest
- 6, // 18: proto.Health.Check:input_type -> proto.HealthCheckRequest
- 9, // 19: proto.Woodpecker.Next:output_type -> proto.NextReply
- 17, // 20: proto.Woodpecker.Init:output_type -> proto.Empty
- 17, // 21: proto.Woodpecker.Wait:output_type -> proto.Empty
- 17, // 22: proto.Woodpecker.Done:output_type -> proto.Empty
- 17, // 23: proto.Woodpecker.Extend:output_type -> proto.Empty
- 17, // 24: proto.Woodpecker.Update:output_type -> proto.Empty
- 17, // 25: proto.Woodpecker.Upload:output_type -> proto.Empty
- 17, // 26: proto.Woodpecker.Log:output_type -> proto.Empty
- 7, // 27: proto.Health.Check:output_type -> proto.HealthCheckResponse
- 19, // [19:28] is the sub-list for method output_type
- 10, // [10:19] is the sub-list for method input_type
- 10, // [10:10] is the sub-list for extension type_name
- 10, // [10:10] is the sub-list for extension extendee
- 0, // [0:10] is the sub-list for field type_name
+ 20, // 0: proto.File.meta:type_name -> proto.File.MetaEntry
+ 21, // 1: proto.Filter.labels:type_name -> proto.Filter.LabelsEntry
+ 3, // 2: proto.NextRequest.filter:type_name -> proto.Filter
+ 4, // 3: proto.NextReply.pipeline:type_name -> proto.Pipeline
+ 1, // 4: proto.InitRequest.state:type_name -> proto.State
+ 1, // 5: proto.DoneRequest.state:type_name -> proto.State
+ 0, // 6: proto.UploadRequest.file:type_name -> proto.File
+ 1, // 7: proto.UpdateRequest.state:type_name -> proto.State
+ 2, // 8: proto.LogRequest.line:type_name -> proto.Line
+ 5, // 9: proto.Woodpecker.Next:input_type -> proto.NextRequest
+ 7, // 10: proto.Woodpecker.Init:input_type -> proto.InitRequest
+ 8, // 11: proto.Woodpecker.Wait:input_type -> proto.WaitRequest
+ 9, // 12: proto.Woodpecker.Done:input_type -> proto.DoneRequest
+ 10, // 13: proto.Woodpecker.Extend:input_type -> proto.ExtendRequest
+ 12, // 14: proto.Woodpecker.Update:input_type -> proto.UpdateRequest
+ 11, // 15: proto.Woodpecker.Upload:input_type -> proto.UploadRequest
+ 13, // 16: proto.Woodpecker.Log:input_type -> proto.LogRequest
+ 16, // 17: proto.Woodpecker.RegisterAgent:input_type -> proto.RegisterAgentRequest
+ 15, // 18: proto.Woodpecker.ReportHealth:input_type -> proto.ReportHealthRequest
+ 18, // 19: proto.WoodpeckerAuth.Auth:input_type -> proto.AuthRequest
+ 6, // 20: proto.Woodpecker.Next:output_type -> proto.NextReply
+ 14, // 21: proto.Woodpecker.Init:output_type -> proto.Empty
+ 14, // 22: proto.Woodpecker.Wait:output_type -> proto.Empty
+ 14, // 23: proto.Woodpecker.Done:output_type -> proto.Empty
+ 14, // 24: proto.Woodpecker.Extend:output_type -> proto.Empty
+ 14, // 25: proto.Woodpecker.Update:output_type -> proto.Empty
+ 14, // 26: proto.Woodpecker.Upload:output_type -> proto.Empty
+ 14, // 27: proto.Woodpecker.Log:output_type -> proto.Empty
+ 17, // 28: proto.Woodpecker.RegisterAgent:output_type -> proto.RegisterAgentResponse
+ 14, // 29: proto.Woodpecker.ReportHealth:output_type -> proto.Empty
+ 19, // 30: proto.WoodpeckerAuth.Auth:output_type -> proto.AuthReply
+ 20, // [20:31] is the sub-list for method output_type
+ 9, // [9:20] is the sub-list for method input_type
+ 9, // [9:9] is the sub-list for extension type_name
+ 9, // [9:9] is the sub-list for extension extendee
+ 0, // [0:9] is the sub-list for field type_name
}
func init() { file_woodpecker_proto_init() }
@@ -1292,30 +1457,6 @@ func file_woodpecker_proto_init() {
}
}
file_woodpecker_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*HealthCheckRequest); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_woodpecker_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*HealthCheckResponse); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_woodpecker_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*NextRequest); i {
case 0:
return &v.state
@@ -1327,7 +1468,7 @@ func file_woodpecker_proto_init() {
return nil
}
}
- file_woodpecker_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
+ file_woodpecker_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*NextReply); i {
case 0:
return &v.state
@@ -1339,7 +1480,7 @@ func file_woodpecker_proto_init() {
return nil
}
}
- file_woodpecker_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
+ file_woodpecker_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*InitRequest); i {
case 0:
return &v.state
@@ -1351,7 +1492,7 @@ func file_woodpecker_proto_init() {
return nil
}
}
- file_woodpecker_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
+ file_woodpecker_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*WaitRequest); i {
case 0:
return &v.state
@@ -1363,7 +1504,7 @@ func file_woodpecker_proto_init() {
return nil
}
}
- file_woodpecker_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
+ file_woodpecker_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*DoneRequest); i {
case 0:
return &v.state
@@ -1375,7 +1516,7 @@ func file_woodpecker_proto_init() {
return nil
}
}
- file_woodpecker_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
+ file_woodpecker_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ExtendRequest); i {
case 0:
return &v.state
@@ -1387,7 +1528,7 @@ func file_woodpecker_proto_init() {
return nil
}
}
- file_woodpecker_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
+ file_woodpecker_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*UploadRequest); i {
case 0:
return &v.state
@@ -1399,7 +1540,7 @@ func file_woodpecker_proto_init() {
return nil
}
}
- file_woodpecker_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
+ file_woodpecker_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*UpdateRequest); i {
case 0:
return &v.state
@@ -1411,7 +1552,7 @@ func file_woodpecker_proto_init() {
return nil
}
}
- file_woodpecker_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
+ file_woodpecker_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*LogRequest); i {
case 0:
return &v.state
@@ -1423,7 +1564,7 @@ func file_woodpecker_proto_init() {
return nil
}
}
- file_woodpecker_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
+ file_woodpecker_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*Empty); i {
case 0:
return &v.state
@@ -1435,20 +1576,79 @@ func file_woodpecker_proto_init() {
return nil
}
}
+ file_woodpecker_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*ReportHealthRequest); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_woodpecker_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*RegisterAgentRequest); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_woodpecker_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*RegisterAgentResponse); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_woodpecker_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*AuthRequest); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_woodpecker_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*AuthReply); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
}
type x struct{}
out := protoimpl.TypeBuilder{
File: protoimpl.DescBuilder{
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_woodpecker_proto_rawDesc,
- NumEnums: 1,
- NumMessages: 19,
+ NumEnums: 0,
+ NumMessages: 22,
NumExtensions: 0,
NumServices: 2,
},
GoTypes: file_woodpecker_proto_goTypes,
DependencyIndexes: file_woodpecker_proto_depIdxs,
- EnumInfos: file_woodpecker_proto_enumTypes,
MessageInfos: file_woodpecker_proto_msgTypes,
}.Build()
File_woodpecker_proto = out.File
diff --git a/pipeline/rpc/proto/woodpecker.proto b/pipeline/rpc/proto/woodpecker.proto
index 39852c5f4..ab50694f9 100644
--- a/pipeline/rpc/proto/woodpecker.proto
+++ b/pipeline/rpc/proto/woodpecker.proto
@@ -3,6 +3,19 @@ syntax = "proto3";
option go_package = "github.com/woodpecker-ci/woodpecker/pipeline/rpc/proto";
package proto;
+service Woodpecker {
+ rpc Next (NextRequest) returns (NextReply) {}
+ rpc Init (InitRequest) returns (Empty) {}
+ rpc Wait (WaitRequest) returns (Empty) {}
+ rpc Done (DoneRequest) returns (Empty) {}
+ rpc Extend (ExtendRequest) returns (Empty) {}
+ rpc Update (UpdateRequest) returns (Empty) {}
+ rpc Upload (UploadRequest) returns (Empty) {}
+ rpc Log (LogRequest) returns (Empty) {}
+ rpc RegisterAgent (RegisterAgentRequest) returns (RegisterAgentResponse) {}
+ rpc ReportHealth (ReportHealthRequest) returns (Empty) {}
+}
+
message File {
string name = 1;
string step = 2;
@@ -39,38 +52,6 @@ message Pipeline {
bytes payload = 3;
}
-message HealthCheckRequest {
- string service = 1;
-}
-
-message HealthCheckResponse {
- enum ServingStatus {
- UNKNOWN = 0;
- SERVING = 1;
- NOT_SERVING = 2;
- }
- ServingStatus status = 1;
-}
-
-service Woodpecker {
- rpc Next (NextRequest) returns (NextReply) {}
- rpc Init (InitRequest) returns (Empty) {}
- rpc Wait (WaitRequest) returns (Empty) {}
- rpc Done (DoneRequest) returns (Empty) {}
- rpc Extend (ExtendRequest) returns (Empty) {}
- rpc Update (UpdateRequest) returns (Empty) {}
- rpc Upload (UploadRequest) returns (Empty) {}
- rpc Log (LogRequest) returns (Empty) {}
-}
-
-service Health {
- rpc Check(HealthCheckRequest) returns (HealthCheckResponse);
-}
-
-//
-// next
-//
-
message NextRequest {
Filter filter = 1;
}
@@ -113,5 +94,36 @@ message LogRequest {
}
message Empty {
-
+}
+
+message ReportHealthRequest {
+ string status = 1;
+}
+
+message RegisterAgentRequest {
+ string platform = 1;
+ int32 capacity = 2;
+ string backend = 3;
+ string version = 4;
+}
+
+message RegisterAgentResponse {
+ int64 agent_id = 1;
+}
+
+// Woodpecker auth service is a simple service to authenticate agents and aquire a token
+
+service WoodpeckerAuth {
+ rpc Auth (AuthRequest) returns (AuthReply) {}
+}
+
+message AuthRequest {
+ string agent_token = 1;
+ int64 agent_id = 2;
+}
+
+message AuthReply {
+ string status = 1;
+ int64 agent_id = 2;
+ string access_token = 3;
}
diff --git a/pipeline/rpc/proto/woodpecker_grpc.pb.go b/pipeline/rpc/proto/woodpecker_grpc.pb.go
index c888d003b..692284b1c 100644
--- a/pipeline/rpc/proto/woodpecker_grpc.pb.go
+++ b/pipeline/rpc/proto/woodpecker_grpc.pb.go
@@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.2.0
-// - protoc v3.21.7
+// - protoc v3.12.4
// source: woodpecker.proto
package proto
@@ -30,6 +30,8 @@ type WoodpeckerClient interface {
Update(ctx context.Context, in *UpdateRequest, opts ...grpc.CallOption) (*Empty, error)
Upload(ctx context.Context, in *UploadRequest, opts ...grpc.CallOption) (*Empty, error)
Log(ctx context.Context, in *LogRequest, opts ...grpc.CallOption) (*Empty, error)
+ RegisterAgent(ctx context.Context, in *RegisterAgentRequest, opts ...grpc.CallOption) (*RegisterAgentResponse, error)
+ ReportHealth(ctx context.Context, in *ReportHealthRequest, opts ...grpc.CallOption) (*Empty, error)
}
type woodpeckerClient struct {
@@ -112,6 +114,24 @@ func (c *woodpeckerClient) Log(ctx context.Context, in *LogRequest, opts ...grpc
return out, nil
}
+func (c *woodpeckerClient) RegisterAgent(ctx context.Context, in *RegisterAgentRequest, opts ...grpc.CallOption) (*RegisterAgentResponse, error) {
+ out := new(RegisterAgentResponse)
+ err := c.cc.Invoke(ctx, "/proto.Woodpecker/RegisterAgent", in, out, opts...)
+ if err != nil {
+ return nil, err
+ }
+ return out, nil
+}
+
+func (c *woodpeckerClient) ReportHealth(ctx context.Context, in *ReportHealthRequest, opts ...grpc.CallOption) (*Empty, error) {
+ out := new(Empty)
+ err := c.cc.Invoke(ctx, "/proto.Woodpecker/ReportHealth", in, out, opts...)
+ if err != nil {
+ return nil, err
+ }
+ return out, nil
+}
+
// WoodpeckerServer is the server API for Woodpecker service.
// All implementations must embed UnimplementedWoodpeckerServer
// for forward compatibility
@@ -124,6 +144,8 @@ type WoodpeckerServer interface {
Update(context.Context, *UpdateRequest) (*Empty, error)
Upload(context.Context, *UploadRequest) (*Empty, error)
Log(context.Context, *LogRequest) (*Empty, error)
+ RegisterAgent(context.Context, *RegisterAgentRequest) (*RegisterAgentResponse, error)
+ ReportHealth(context.Context, *ReportHealthRequest) (*Empty, error)
mustEmbedUnimplementedWoodpeckerServer()
}
@@ -155,6 +177,12 @@ func (UnimplementedWoodpeckerServer) Upload(context.Context, *UploadRequest) (*E
func (UnimplementedWoodpeckerServer) Log(context.Context, *LogRequest) (*Empty, error) {
return nil, status.Errorf(codes.Unimplemented, "method Log not implemented")
}
+func (UnimplementedWoodpeckerServer) RegisterAgent(context.Context, *RegisterAgentRequest) (*RegisterAgentResponse, error) {
+ return nil, status.Errorf(codes.Unimplemented, "method RegisterAgent not implemented")
+}
+func (UnimplementedWoodpeckerServer) ReportHealth(context.Context, *ReportHealthRequest) (*Empty, error) {
+ return nil, status.Errorf(codes.Unimplemented, "method ReportHealth not implemented")
+}
func (UnimplementedWoodpeckerServer) mustEmbedUnimplementedWoodpeckerServer() {}
// UnsafeWoodpeckerServer may be embedded to opt out of forward compatibility for this service.
@@ -312,6 +340,42 @@ func _Woodpecker_Log_Handler(srv interface{}, ctx context.Context, dec func(inte
return interceptor(ctx, in, info, handler)
}
+func _Woodpecker_RegisterAgent_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(RegisterAgentRequest)
+ if err := dec(in); err != nil {
+ return nil, err
+ }
+ if interceptor == nil {
+ return srv.(WoodpeckerServer).RegisterAgent(ctx, in)
+ }
+ info := &grpc.UnaryServerInfo{
+ Server: srv,
+ FullMethod: "/proto.Woodpecker/RegisterAgent",
+ }
+ handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+ return srv.(WoodpeckerServer).RegisterAgent(ctx, req.(*RegisterAgentRequest))
+ }
+ return interceptor(ctx, in, info, handler)
+}
+
+func _Woodpecker_ReportHealth_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(ReportHealthRequest)
+ if err := dec(in); err != nil {
+ return nil, err
+ }
+ if interceptor == nil {
+ return srv.(WoodpeckerServer).ReportHealth(ctx, in)
+ }
+ info := &grpc.UnaryServerInfo{
+ Server: srv,
+ FullMethod: "/proto.Woodpecker/ReportHealth",
+ }
+ handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+ return srv.(WoodpeckerServer).ReportHealth(ctx, req.(*ReportHealthRequest))
+ }
+ return interceptor(ctx, in, info, handler)
+}
+
// Woodpecker_ServiceDesc is the grpc.ServiceDesc for Woodpecker service.
// It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy)
@@ -351,91 +415,99 @@ var Woodpecker_ServiceDesc = grpc.ServiceDesc{
MethodName: "Log",
Handler: _Woodpecker_Log_Handler,
},
+ {
+ MethodName: "RegisterAgent",
+ Handler: _Woodpecker_RegisterAgent_Handler,
+ },
+ {
+ MethodName: "ReportHealth",
+ Handler: _Woodpecker_ReportHealth_Handler,
+ },
},
Streams: []grpc.StreamDesc{},
Metadata: "woodpecker.proto",
}
-// HealthClient is the client API for Health service.
+// WoodpeckerAuthClient is the client API for WoodpeckerAuth 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 HealthClient interface {
- Check(ctx context.Context, in *HealthCheckRequest, opts ...grpc.CallOption) (*HealthCheckResponse, error)
+type WoodpeckerAuthClient interface {
+ Auth(ctx context.Context, in *AuthRequest, opts ...grpc.CallOption) (*AuthReply, error)
}
-type healthClient struct {
+type woodpeckerAuthClient struct {
cc grpc.ClientConnInterface
}
-func NewHealthClient(cc grpc.ClientConnInterface) HealthClient {
- return &healthClient{cc}
+func NewWoodpeckerAuthClient(cc grpc.ClientConnInterface) WoodpeckerAuthClient {
+ return &woodpeckerAuthClient{cc}
}
-func (c *healthClient) Check(ctx context.Context, in *HealthCheckRequest, opts ...grpc.CallOption) (*HealthCheckResponse, error) {
- out := new(HealthCheckResponse)
- err := c.cc.Invoke(ctx, "/proto.Health/Check", in, out, opts...)
+func (c *woodpeckerAuthClient) Auth(ctx context.Context, in *AuthRequest, opts ...grpc.CallOption) (*AuthReply, error) {
+ out := new(AuthReply)
+ err := c.cc.Invoke(ctx, "/proto.WoodpeckerAuth/Auth", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
-// HealthServer is the server API for Health service.
-// All implementations must embed UnimplementedHealthServer
+// WoodpeckerAuthServer is the server API for WoodpeckerAuth service.
+// All implementations must embed UnimplementedWoodpeckerAuthServer
// for forward compatibility
-type HealthServer interface {
- Check(context.Context, *HealthCheckRequest) (*HealthCheckResponse, error)
- mustEmbedUnimplementedHealthServer()
+type WoodpeckerAuthServer interface {
+ Auth(context.Context, *AuthRequest) (*AuthReply, error)
+ mustEmbedUnimplementedWoodpeckerAuthServer()
}
-// UnimplementedHealthServer must be embedded to have forward compatible implementations.
-type UnimplementedHealthServer struct {
+// UnimplementedWoodpeckerAuthServer must be embedded to have forward compatible implementations.
+type UnimplementedWoodpeckerAuthServer struct {
}
-func (UnimplementedHealthServer) Check(context.Context, *HealthCheckRequest) (*HealthCheckResponse, error) {
- return nil, status.Errorf(codes.Unimplemented, "method Check not implemented")
+func (UnimplementedWoodpeckerAuthServer) Auth(context.Context, *AuthRequest) (*AuthReply, error) {
+ return nil, status.Errorf(codes.Unimplemented, "method Auth not implemented")
}
-func (UnimplementedHealthServer) mustEmbedUnimplementedHealthServer() {}
+func (UnimplementedWoodpeckerAuthServer) mustEmbedUnimplementedWoodpeckerAuthServer() {}
-// UnsafeHealthServer may be embedded to opt out of forward compatibility for this service.
-// Use of this interface is not recommended, as added methods to HealthServer will
+// UnsafeWoodpeckerAuthServer may be embedded to opt out of forward compatibility for this service.
+// Use of this interface is not recommended, as added methods to WoodpeckerAuthServer will
// result in compilation errors.
-type UnsafeHealthServer interface {
- mustEmbedUnimplementedHealthServer()
+type UnsafeWoodpeckerAuthServer interface {
+ mustEmbedUnimplementedWoodpeckerAuthServer()
}
-func RegisterHealthServer(s grpc.ServiceRegistrar, srv HealthServer) {
- s.RegisterService(&Health_ServiceDesc, srv)
+func RegisterWoodpeckerAuthServer(s grpc.ServiceRegistrar, srv WoodpeckerAuthServer) {
+ s.RegisterService(&WoodpeckerAuth_ServiceDesc, srv)
}
-func _Health_Check_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
- in := new(HealthCheckRequest)
+func _WoodpeckerAuth_Auth_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(AuthRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
- return srv.(HealthServer).Check(ctx, in)
+ return srv.(WoodpeckerAuthServer).Auth(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
- FullMethod: "/proto.Health/Check",
+ FullMethod: "/proto.WoodpeckerAuth/Auth",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
- return srv.(HealthServer).Check(ctx, req.(*HealthCheckRequest))
+ return srv.(WoodpeckerAuthServer).Auth(ctx, req.(*AuthRequest))
}
return interceptor(ctx, in, info, handler)
}
-// Health_ServiceDesc is the grpc.ServiceDesc for Health service.
+// WoodpeckerAuth_ServiceDesc is the grpc.ServiceDesc for WoodpeckerAuth service.
// It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy)
-var Health_ServiceDesc = grpc.ServiceDesc{
- ServiceName: "proto.Health",
- HandlerType: (*HealthServer)(nil),
+var WoodpeckerAuth_ServiceDesc = grpc.ServiceDesc{
+ ServiceName: "proto.WoodpeckerAuth",
+ HandlerType: (*WoodpeckerAuthServer)(nil),
Methods: []grpc.MethodDesc{
{
- MethodName: "Check",
- Handler: _Health_Check_Handler,
+ MethodName: "Auth",
+ Handler: _WoodpeckerAuth_Auth_Handler,
},
},
Streams: []grpc.StreamDesc{},
diff --git a/server/api/agent.go b/server/api/agent.go
new file mode 100644
index 000000000..c798b0f48
--- /dev/null
+++ b/server/api/agent.go
@@ -0,0 +1,130 @@
+// Copyright 2022 Woodpecker Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package api
+
+import (
+ "encoding/base32"
+ "net/http"
+ "strconv"
+
+ "github.com/gin-gonic/gin"
+ "github.com/gorilla/securecookie"
+
+ "github.com/woodpecker-ci/woodpecker/server/model"
+ "github.com/woodpecker-ci/woodpecker/server/router/middleware/session"
+ "github.com/woodpecker-ci/woodpecker/server/store"
+)
+
+func GetAgents(c *gin.Context) {
+ agents, err := store.FromContext(c).AgentList()
+ if err != nil {
+ c.String(500, "Error getting agent list. %s", err)
+ return
+ }
+ c.JSON(http.StatusOK, agents)
+}
+
+func GetAgent(c *gin.Context) {
+ agentID, err := strconv.ParseInt(c.Param("agent"), 10, 64)
+ if err != nil {
+ _ = c.AbortWithError(http.StatusBadRequest, err)
+ return
+ }
+
+ agent, err := store.FromContext(c).AgentFind(agentID)
+ if err != nil {
+ c.String(http.StatusNotFound, "Cannot find agent. %s", err)
+ return
+ }
+ c.JSON(http.StatusOK, agent)
+}
+
+func PatchAgent(c *gin.Context) {
+ _store := store.FromContext(c)
+
+ in := &model.Agent{}
+ err := c.Bind(in)
+ if err != nil {
+ c.AbortWithStatus(http.StatusBadRequest)
+ return
+ }
+
+ agentID, err := strconv.ParseInt(c.Param("agent"), 10, 64)
+ if err != nil {
+ _ = c.AbortWithError(http.StatusBadRequest, err)
+ return
+ }
+
+ agent, err := _store.AgentFind(agentID)
+ if err != nil {
+ c.AbortWithStatus(http.StatusNotFound)
+ return
+ }
+ agent.Name = in.Name
+
+ err = _store.AgentUpdate(agent)
+ if err != nil {
+ c.AbortWithStatus(http.StatusConflict)
+ return
+ }
+
+ c.JSON(http.StatusOK, agent)
+}
+
+// PostAgent create a new agent with a random token so a new agent can connect to the server
+func PostAgent(c *gin.Context) {
+ in := &model.Agent{}
+ err := c.Bind(in)
+ if err != nil {
+ c.String(http.StatusBadRequest, err.Error())
+ return
+ }
+
+ user := session.User(c)
+
+ agent := &model.Agent{
+ Name: in.Name,
+ OwnerID: user.ID,
+ Token: base32.StdEncoding.EncodeToString(
+ securecookie.GenerateRandomKey(32),
+ ),
+ }
+ if err = store.FromContext(c).AgentCreate(agent); err != nil {
+ c.String(http.StatusInternalServerError, err.Error())
+ return
+ }
+ c.JSON(http.StatusOK, agent)
+}
+
+func DeleteAgent(c *gin.Context) {
+ _store := store.FromContext(c)
+
+ agentID, err := strconv.ParseInt(c.Param("agent"), 10, 64)
+ if err != nil {
+ _ = c.AbortWithError(http.StatusBadRequest, err)
+ return
+ }
+
+ agent, err := _store.AgentFind(agentID)
+ if err != nil {
+ c.String(http.StatusNotFound, "Cannot find user. %s", err)
+ return
+ }
+ if err = _store.AgentDelete(agent); err != nil {
+ c.String(http.StatusInternalServerError, "Error deleting user. %s", err)
+ return
+ }
+ c.String(http.StatusOK, "")
+}
diff --git a/server/config.go b/server/config.go
index 4e4152fce..774f4ea0b 100644
--- a/server/config.go
+++ b/server/config.go
@@ -60,7 +60,7 @@ var Config = struct {
OAuthHost string
Host string
Port string
- Pass string
+ AgentToken string
Docs string
StatusContext string
StatusContextFormat string
diff --git a/server/grpc/auth_server.go b/server/grpc/auth_server.go
new file mode 100644
index 000000000..d2ec52fd9
--- /dev/null
+++ b/server/grpc/auth_server.go
@@ -0,0 +1,65 @@
+package grpc
+
+import (
+ "context"
+
+ "github.com/rs/zerolog/log"
+
+ "github.com/woodpecker-ci/woodpecker/pipeline/rpc/proto"
+ "github.com/woodpecker-ci/woodpecker/server"
+ "github.com/woodpecker-ci/woodpecker/server/model"
+ "github.com/woodpecker-ci/woodpecker/server/store"
+)
+
+type WoodpeckerAuthServer struct {
+ proto.UnimplementedWoodpeckerAuthServer
+ jwtManager *JWTManager
+ agentMasterToken string
+ store store.Store
+}
+
+func NewWoodpeckerAuthServer(jwtManager *JWTManager, agentMasterToken string, store store.Store) *WoodpeckerAuthServer {
+ return &WoodpeckerAuthServer{jwtManager: jwtManager, agentMasterToken: agentMasterToken, store: store}
+}
+
+func (s *WoodpeckerAuthServer) Auth(c context.Context, req *proto.AuthRequest) (*proto.AuthReply, error) {
+ agent, err := s.getAgent(c, req.AgentId, req.AgentToken)
+ if err != nil {
+ return nil, err
+ }
+
+ accessToken, err := s.jwtManager.Generate(agent.ID)
+ if err != nil {
+ return nil, err
+ }
+
+ return &proto.AuthReply{
+ Status: "ok",
+ AgentId: agent.ID,
+ AccessToken: accessToken,
+ }, nil
+}
+
+func (s *WoodpeckerAuthServer) getAgent(c context.Context, agentID int64, agentToken string) (*model.Agent, error) {
+ if agentToken == s.agentMasterToken && agentID == -1 {
+ agent := new(model.Agent)
+ agent.Name = ""
+ agent.OwnerID = -1 // system agent
+ agent.Token = server.Config.Server.AgentToken
+ agent.Backend = ""
+ agent.Platform = ""
+ agent.Capacity = -1
+ err := s.store.AgentCreate(agent)
+ if err != nil {
+ log.Err(err).Msgf("Error creating system agent: %s", err)
+ return nil, err
+ }
+ return agent, nil
+ }
+
+ if agentToken == s.agentMasterToken {
+ return s.store.AgentFind(agentID)
+ }
+
+ return s.store.AgentFindByToken(agentToken)
+}
diff --git a/server/grpc/authorizer.go b/server/grpc/authorizer.go
new file mode 100644
index 000000000..fc3c0a59b
--- /dev/null
+++ b/server/grpc/authorizer.go
@@ -0,0 +1,93 @@
+package grpc
+
+import (
+ "context"
+ "fmt"
+
+ "google.golang.org/grpc"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/metadata"
+ "google.golang.org/grpc/status"
+)
+
+type StreamContextWrapper interface {
+ grpc.ServerStream
+ SetContext(context.Context)
+}
+
+type wrapper struct {
+ grpc.ServerStream
+ ctx context.Context
+}
+
+func (w *wrapper) Context() context.Context {
+ return w.ctx
+}
+
+func (w *wrapper) SetContext(ctx context.Context) {
+ w.ctx = ctx
+}
+
+func newStreamContextWrapper(inner grpc.ServerStream) StreamContextWrapper {
+ ctx := inner.Context()
+ return &wrapper{
+ inner,
+ ctx,
+ }
+}
+
+type Authorizer struct {
+ jwtManager *JWTManager
+}
+
+func NewAuthorizer(jwtManager *JWTManager) *Authorizer {
+ return &Authorizer{jwtManager: jwtManager}
+}
+
+func (a *Authorizer) StreamInterceptor(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+ _stream := newStreamContextWrapper(stream)
+
+ newCtx, err := a.authorize(stream.Context(), info.FullMethod)
+ if err != nil {
+ return err
+ }
+
+ _stream.SetContext(newCtx)
+
+ return handler(srv, _stream)
+}
+
+func (a *Authorizer) UnaryInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) {
+ newCtx, err := a.authorize(ctx, info.FullMethod)
+ if err != nil {
+ return nil, err
+ }
+ return handler(newCtx, req)
+}
+
+func (a *Authorizer) authorize(ctx context.Context, fullMethod string) (context.Context, error) {
+ // bypass auth for token endpoint
+ if fullMethod == "/proto.WoodpeckerAuth/Auth" {
+ return ctx, nil
+ }
+
+ md, ok := metadata.FromIncomingContext(ctx)
+ if !ok {
+ return ctx, status.Errorf(codes.Unauthenticated, "metadata is not provided")
+ }
+
+ values := md["token"]
+ if len(values) == 0 {
+ return ctx, status.Errorf(codes.Unauthenticated, "token is not provided")
+ }
+
+ accessToken := values[0]
+ claims, err := a.jwtManager.Verify(accessToken)
+ if err != nil {
+ return ctx, status.Errorf(codes.Unauthenticated, "access token is invalid: %v", err)
+ }
+
+ md.Append("agent_id", fmt.Sprintf("%d", claims.AgentID))
+
+ return metadata.NewIncomingContext(ctx, md), nil
+}
diff --git a/server/grpc/jwt_manager.go b/server/grpc/jwt_manager.go
new file mode 100644
index 000000000..5ec2e5721
--- /dev/null
+++ b/server/grpc/jwt_manager.go
@@ -0,0 +1,73 @@
+package grpc
+
+import (
+ "errors"
+ "fmt"
+ "time"
+
+ "github.com/golang-jwt/jwt/v4"
+)
+
+// JWTManager is a JSON web token manager
+type JWTManager struct {
+ secretKey string
+ tokenDuration time.Duration
+}
+
+// UserClaims is a custom JWT claims that contains some user's information
+type AgentTokenClaims struct {
+ jwt.RegisteredClaims
+ AgentID int64 `json:"agent_id"`
+}
+
+const jwtTokenDuration = 1 * time.Hour
+
+// NewJWTManager returns a new JWT manager
+func NewJWTManager(secretKey string) *JWTManager {
+ return &JWTManager{secretKey, jwtTokenDuration}
+}
+
+// Generate generates and signs a new token for a user
+func (manager *JWTManager) Generate(agentID int64) (string, error) {
+ claims := AgentTokenClaims{
+ RegisteredClaims: jwt.RegisteredClaims{
+ Issuer: "woodpecker",
+ Subject: fmt.Sprintf("%d", agentID),
+ Audience: jwt.ClaimStrings{},
+ NotBefore: jwt.NewNumericDate(time.Now()),
+ IssuedAt: jwt.NewNumericDate(time.Now()),
+ ID: fmt.Sprintf("%d", agentID),
+ ExpiresAt: jwt.NewNumericDate(time.Now().Add(manager.tokenDuration)),
+ },
+ AgentID: agentID,
+ }
+
+ token := jwt.NewWithClaims(jwt.SigningMethodHS256, claims)
+ return token.SignedString([]byte(manager.secretKey))
+}
+
+// Verify verifies the access token string and return a user claim if the token is valid
+func (manager *JWTManager) Verify(accessToken string) (*AgentTokenClaims, error) {
+ token, err := jwt.ParseWithClaims(
+ accessToken,
+ &AgentTokenClaims{},
+ func(token *jwt.Token) (interface{}, error) {
+ _, ok := token.Method.(*jwt.SigningMethodHMAC)
+ if !ok {
+ return nil, errors.New("unexpected token signing method")
+ }
+
+ return []byte(manager.secretKey), nil
+ },
+ )
+ if err != nil {
+ return nil, fmt.Errorf("invalid token: %w", err)
+ }
+
+ claims, ok := token.Claims.(*AgentTokenClaims)
+ if !ok {
+ return nil, errors.New("invalid token claims")
+ }
+
+ return claims, nil
+}
diff --git a/server/grpc/rpc.go b/server/grpc/rpc.go
index 22befa5ae..2405a0c3b 100644
--- a/server/grpc/rpc.go
+++ b/server/grpc/rpc.go
@@ -22,12 +22,15 @@ import (
"bytes"
"context"
"encoding/json"
+ "errors"
"fmt"
"strconv"
+ "time"
"github.com/rs/zerolog/log"
"github.com/prometheus/client_golang/prometheus"
+ "google.golang.org/grpc/metadata"
grpcMetadata "google.golang.org/grpc/metadata"
"github.com/woodpecker-ci/woodpecker/pipeline/rpc"
@@ -382,6 +385,40 @@ func (s *RPC) Log(c context.Context, id string, line *rpc.Line) error {
return nil
}
+func (s *RPC) RegisterAgent(ctx context.Context, platform, backend, version string, capacity int32) (int64, error) {
+ agent, err := s.getAgentFromContext(ctx)
+ if err != nil {
+ return -1, err
+ }
+
+ agent.Backend = backend
+ agent.Platform = platform
+ agent.Capacity = capacity
+ agent.Version = version
+
+ err = s.store.AgentUpdate(agent)
+ if err != nil {
+ return -1, err
+ }
+
+ return agent.ID, nil
+}
+
+func (s *RPC) ReportHealth(ctx context.Context, status string) error {
+ agent, err := s.getAgentFromContext(ctx)
+ if err != nil {
+ return err
+ }
+
+ if status != "I am alive!" {
+ return errors.New("Are you alive?")
+ }
+
+ agent.LastContact = time.Now().Unix()
+
+ return s.store.AgentUpdate(agent)
+}
+
func (s *RPC) completeChildrenIfParentCompleted(steps []*model.Step, completedWorkflow *model.Step) {
for _, p := range steps {
if p.Running() && p.PPID == completedWorkflow.PID {
@@ -438,3 +475,23 @@ func (s *RPC) notify(c context.Context, repo *model.Repo, pipeline *model.Pipeli
}
return nil
}
+
+func (s *RPC) getAgentFromContext(ctx context.Context) (*model.Agent, error) {
+ md, ok := metadata.FromIncomingContext(ctx)
+ if !ok {
+ return nil, errors.New("metadata is not provided")
+ }
+
+ values := md["agent_id"]
+ if len(values) == 0 {
+ return nil, errors.New("agent_id is not provided")
+ }
+
+ _agentID := values[0]
+ agentID, err := strconv.ParseInt(_agentID, 10, 64)
+ if err != nil {
+ return nil, errors.New("agent_id is not a valid integer")
+ }
+
+ return s.store.AgentFind(agentID)
+}
diff --git a/server/grpc/server.go b/server/grpc/server.go
index 021d922c8..e2f18ffff 100644
--- a/server/grpc/server.go
+++ b/server/grpc/server.go
@@ -163,3 +163,16 @@ func (s *WoodpeckerServer) Log(c context.Context, req *proto.LogRequest) (*proto
err := s.peer.Log(c, req.GetId(), line)
return res, err
}
+
+func (s *WoodpeckerServer) RegisterAgent(c context.Context, req *proto.RegisterAgentRequest) (*proto.RegisterAgentResponse, error) {
+ res := new(proto.RegisterAgentResponse)
+ agentID, err := s.peer.RegisterAgent(c, req.GetPlatform(), req.GetBackend(), req.GetVersion(), req.GetCapacity())
+ res.AgentId = agentID
+ return res, err
+}
+
+func (s *WoodpeckerServer) ReportHealth(c context.Context, req *proto.ReportHealthRequest) (*proto.Empty, error) {
+ res := new(proto.Empty)
+ err := s.peer.ReportHealth(c, req.GetStatus())
+ return res, err
+}
diff --git a/server/model/agent.go b/server/model/agent.go
index 02a3fe700..594b309b3 100644
--- a/server/model/agent.go
+++ b/server/model/agent.go
@@ -14,18 +14,25 @@
package model
-// TODO: check if it is actually used or just some relict from the past
-
type Agent struct {
- ID int64 `xorm:"pk autoincr 'agent_id'"`
- Addr string `xorm:"UNIQUE VARCHAR(250) 'agent_addr'"`
- Platform string `xorm:"VARCHAR(500) 'agent_platform'"`
- Capacity int64 `xorm:"agent_capacity"`
- Created int64 `xorm:"created 'agent_created'"`
- Updated int64 `xorm:"updated 'agent_updated'"`
+ ID int64 `json:"id" xorm:"pk autoincr 'id'"`
+ Created int64 `json:"created" xorm:"created"`
+ Updated int64 `json:"updated" xorm:"updated"`
+ Name string `json:"name"`
+ OwnerID int64 `json:"owner_id" xorm:"'owner_id'"`
+ Token string `json:"token"`
+ LastContact int64 `json:"last_contact"`
+ Platform string `json:"platform" xorm:"VARCHAR(100)"`
+ Backend string `json:"backend" xorm:"VARCHAR(100)"`
+ Capacity int32 `json:"capacity"`
+ Version string `json:"version"`
}
// TableName return database table name for xorm
func (Agent) TableName() string {
return "agents"
}
+
+func (a *Agent) IsSystemAgent() bool {
+ return a.OwnerID == -1
+}
diff --git a/server/router/api.go b/server/router/api.go
index a35a0c871..aa23bdb54 100644
--- a/server/router/api.go
+++ b/server/router/api.go
@@ -166,6 +166,16 @@ func apiRoutes(e *gin.Engine) {
logLevel.POST("", api.SetLogLevel)
}
+ agentBase := apiBase.Group("/agents")
+ {
+ agentBase.Use(session.MustAdmin())
+ agentBase.GET("", api.GetAgents)
+ agentBase.POST("", api.PostAgent)
+ agentBase.GET("/:agent", api.GetAgent)
+ agentBase.PATCH("/:agent", api.PatchAgent)
+ agentBase.DELETE("/:agent", api.DeleteAgent)
+ }
+
apiBase.GET("/signature/public-key", session.MustUser(), api.GetSignaturePublicKey)
apiBase.POST("/hook", api.PostHook)
diff --git a/server/store/datastore/agent.go b/server/store/datastore/agent.go
new file mode 100644
index 000000000..b299eb883
--- /dev/null
+++ b/server/store/datastore/agent.go
@@ -0,0 +1,52 @@
+// Copyright 2021 Woodpecker Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package datastore
+
+import (
+ "github.com/woodpecker-ci/woodpecker/server/model"
+)
+
+func (s storage) AgentList() ([]*model.Agent, error) {
+ agents := make([]*model.Agent, 0, 10)
+ return agents, s.engine.Find(&agents)
+}
+
+func (s storage) AgentFind(id int64) (*model.Agent, error) {
+ agent := new(model.Agent)
+ return agent, wrapGet(s.engine.ID(id).Get(agent))
+}
+
+func (s storage) AgentFindByToken(token string) (*model.Agent, error) {
+ agent := &model.Agent{
+ Token: token,
+ }
+ return agent, wrapGet(s.engine.Get(agent))
+}
+
+func (s storage) AgentCreate(agent *model.Agent) error {
+ // only Insert set auto created ID back to object
+ _, err := s.engine.Insert(agent)
+ return err
+}
+
+func (s storage) AgentUpdate(agent *model.Agent) error {
+ _, err := s.engine.ID(agent.ID).AllCols().Update(agent)
+ return err
+}
+
+func (s storage) AgentDelete(agent *model.Agent) error {
+ _, err := s.engine.ID(agent.ID).Delete(new(model.Agent))
+ return err
+}
diff --git a/server/store/datastore/migration/008_recreate_agents_table.go b/server/store/datastore/migration/008_recreate_agents_table.go
new file mode 100644
index 000000000..5cbbc6463
--- /dev/null
+++ b/server/store/datastore/migration/008_recreate_agents_table.go
@@ -0,0 +1,31 @@
+// Copyright 2022 Woodpecker Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package migration
+
+import (
+ "xorm.io/xorm"
+
+ "github.com/woodpecker-ci/woodpecker/server/model"
+)
+
+var recreateAgentsTable = task{
+ name: "recreate-agents-table",
+ fn: func(sess *xorm.Session) error {
+ if err := dropTable(sess, "agents"); err != nil {
+ return err
+ }
+ return sess.Sync2(new(model.Agent))
+ },
+}
diff --git a/server/store/datastore/migration/common.go b/server/store/datastore/migration/common.go
index 1e5f28a49..646c88478 100644
--- a/server/store/datastore/migration/common.go
+++ b/server/store/datastore/migration/common.go
@@ -23,6 +23,17 @@ import (
"xorm.io/xorm/schemas"
)
+func dropTable(sess *xorm.Session, table string) error {
+ dialect := sess.Engine().Dialect().URI().DBType
+ switch dialect {
+ case schemas.MYSQL, schemas.POSTGRES, schemas.SQLITE:
+ _, err := sess.Exec(fmt.Sprintf("DROP TABLE `%s`;", table))
+ return err
+ default:
+ return fmt.Errorf("dialect '%s' not supported", dialect)
+ }
+}
+
func renameTable(sess *xorm.Session, old, new string) error {
dialect := sess.Engine().Dialect().URI().DBType
switch dialect {
diff --git a/server/store/datastore/migration/migration.go b/server/store/datastore/migration/migration.go
index 0f2a73908..6d03d7904 100644
--- a/server/store/datastore/migration/migration.go
+++ b/server/store/datastore/migration/migration.go
@@ -35,6 +35,7 @@ var migrationTasks = []*task{
&dropSenders,
&alterTableLogUpdateColumnLogDataType,
&alterTableSecretsAddUserCol,
+ &recreateAgentsTable,
&lowercaseSecretNames,
&renameBuildsToPipeline,
&renameColumnsBuildsToPipeline,
diff --git a/server/store/mocks/store.go b/server/store/mocks/store.go
index 4f9794fe4..c0220df20 100644
--- a/server/store/mocks/store.go
+++ b/server/store/mocks/store.go
@@ -15,6 +15,117 @@ type Store struct {
mock.Mock
}
+// AgentCreate provides a mock function with given fields: _a0
+func (_m *Store) AgentCreate(_a0 *model.Agent) error {
+ ret := _m.Called(_a0)
+
+ var r0 error
+ if rf, ok := ret.Get(0).(func(*model.Agent) error); ok {
+ r0 = rf(_a0)
+ } else {
+ r0 = ret.Error(0)
+ }
+
+ return r0
+}
+
+// AgentDelete provides a mock function with given fields: _a0
+func (_m *Store) AgentDelete(_a0 *model.Agent) error {
+ ret := _m.Called(_a0)
+
+ var r0 error
+ if rf, ok := ret.Get(0).(func(*model.Agent) error); ok {
+ r0 = rf(_a0)
+ } else {
+ r0 = ret.Error(0)
+ }
+
+ return r0
+}
+
+// AgentFind provides a mock function with given fields: _a0
+func (_m *Store) AgentFind(_a0 int64) (*model.Agent, error) {
+ ret := _m.Called(_a0)
+
+ var r0 *model.Agent
+ if rf, ok := ret.Get(0).(func(int64) *model.Agent); ok {
+ r0 = rf(_a0)
+ } else {
+ if ret.Get(0) != nil {
+ r0 = ret.Get(0).(*model.Agent)
+ }
+ }
+
+ var r1 error
+ if rf, ok := ret.Get(1).(func(int64) error); ok {
+ r1 = rf(_a0)
+ } else {
+ r1 = ret.Error(1)
+ }
+
+ return r0, r1
+}
+
+// AgentFindByToken provides a mock function with given fields: _a0
+func (_m *Store) AgentFindByToken(_a0 string) (*model.Agent, error) {
+ ret := _m.Called(_a0)
+
+ var r0 *model.Agent
+ if rf, ok := ret.Get(0).(func(string) *model.Agent); ok {
+ r0 = rf(_a0)
+ } else {
+ if ret.Get(0) != nil {
+ r0 = ret.Get(0).(*model.Agent)
+ }
+ }
+
+ var r1 error
+ if rf, ok := ret.Get(1).(func(string) error); ok {
+ r1 = rf(_a0)
+ } else {
+ r1 = ret.Error(1)
+ }
+
+ return r0, r1
+}
+
+// AgentList provides a mock function with given fields:
+func (_m *Store) AgentList() ([]*model.Agent, error) {
+ ret := _m.Called()
+
+ var r0 []*model.Agent
+ if rf, ok := ret.Get(0).(func() []*model.Agent); ok {
+ r0 = rf()
+ } else {
+ if ret.Get(0) != nil {
+ r0 = ret.Get(0).([]*model.Agent)
+ }
+ }
+
+ var r1 error
+ if rf, ok := ret.Get(1).(func() error); ok {
+ r1 = rf()
+ } else {
+ r1 = ret.Error(1)
+ }
+
+ return r0, r1
+}
+
+// AgentUpdate provides a mock function with given fields: _a0
+func (_m *Store) AgentUpdate(_a0 *model.Agent) error {
+ ret := _m.Called(_a0)
+
+ var r0 error
+ if rf, ok := ret.Get(0).(func(*model.Agent) error); ok {
+ r0 = rf(_a0)
+ } else {
+ r0 = ret.Error(0)
+ }
+
+ return r0
+}
+
// Close provides a mock function with given fields:
func (_m *Store) Close() error {
ret := _m.Called()
diff --git a/server/store/store.go b/server/store/store.go
index 306698f82..7deb66f2d 100644
--- a/server/store/store.go
+++ b/server/store/store.go
@@ -179,6 +179,14 @@ type Store interface {
CronListNextExecute(int64, int64) ([]*model.Cron, error)
CronGetLock(*model.Cron, int64) (bool, error)
+ // Agent
+ AgentCreate(*model.Agent) error
+ AgentFind(int64) (*model.Agent, error)
+ AgentFindByToken(string) (*model.Agent, error)
+ AgentList() ([]*model.Agent, error)
+ AgentUpdate(*model.Agent) error
+ AgentDelete(*model.Agent) error
+
// Store operations
Ping() error
Close() error
diff --git a/web/components.d.ts b/web/components.d.ts
index 8a2d022af..8a8bd2f2f 100644
--- a/web/components.d.ts
+++ b/web/components.d.ts
@@ -9,6 +9,7 @@ declare module '@vue/runtime-core' {
export interface GlobalComponents {
ActionsTab: typeof import('./src/components/repo/settings/ActionsTab.vue')['default']
ActivePipelines: typeof import('./src/components/layout/header/ActivePipelines.vue')['default']
+ AdminAgentsTab: typeof import('./src/components/admin/settings/AdminAgentsTab.vue')['default']
AdminSecretsTab: typeof import('./src/components/admin/settings/AdminSecretsTab.vue')['default']
BadgeTab: typeof import('./src/components/repo/settings/BadgeTab.vue')['default']
Button: typeof import('./src/components/atomic/Button.vue')['default']
diff --git a/web/src/assets/locales/en.json b/web/src/assets/locales/en.json
index 4ee285ede..529a43df2 100644
--- a/web/src/assets/locales/en.json
+++ b/web/src/assets/locales/en.json
@@ -321,6 +321,29 @@
"events": "Available at following events",
"pr_warning": "Please be careful with this option as a bad actor can submit a malicious pull request that exposes your secrets."
}
+ },
+ "agents": {
+ "agents": "Agents",
+ "desc": "Agents registered for this server",
+ "none": "There are no agents yet.",
+ "add": "Add agent",
+ "save": "Save agent",
+ "show": "Show agents",
+ "created": "Agent created",
+ "saved": "Agent saved",
+ "deleted": "Agent deleted",
+ "name": {
+ "name": "Name",
+ "placeholder": "Name of the agent"
+ },
+ "token": "Token",
+ "platform": "Platform",
+ "backend": "Backend",
+ "capacity": "Capacity",
+ "version": "Version",
+ "last_contact": "Last contact",
+ "never": "Never",
+ "delete_confirm": "Do you really want to delete this agent? It wont be able to connected to the server anymore."
}
}
},
diff --git a/web/src/components/admin/settings/AdminAgentsTab.vue b/web/src/components/admin/settings/AdminAgentsTab.vue
new file mode 100644
index 000000000..ab9348769
--- /dev/null
+++ b/web/src/components/admin/settings/AdminAgentsTab.vue
@@ -0,0 +1,161 @@
+
+ {{ $t('admin.settings.agents.desc') }}{{ $t('admin.settings.agents.agents') }}
+