feat: Add initial implementation of Vulnerability Resolver Jobs
Some checks failed
Docs CI / lint-and-preview (push) Has been cancelled

- Created project for StellaOps.Scanner.Analyzers.Native.Tests with necessary dependencies.
- Documented roles and guidelines in AGENTS.md for Scheduler module.
- Implemented IResolverJobService interface and InMemoryResolverJobService for handling resolver jobs.
- Added ResolverBacklogNotifier and ResolverBacklogService for monitoring job metrics.
- Developed API endpoints for managing resolver jobs and retrieving metrics.
- Defined models for resolver job requests and responses.
- Integrated dependency injection for resolver job services.
- Implemented ImpactIndexSnapshot for persisting impact index data.
- Introduced SignalsScoringOptions for configurable scoring weights in reachability scoring.
- Added unit tests for ReachabilityScoringService and RuntimeFactsIngestionService.
- Created dotnet-filter.sh script to handle command-line arguments for dotnet.
- Established nuget-prime project for managing package downloads.
This commit is contained in:
master
2025-11-18 07:52:15 +02:00
parent e69b57d467
commit 8355e2ff75
299 changed files with 13293 additions and 2444 deletions

View File

@@ -0,0 +1,9 @@
# Worker SDK (Go) — Task Tracker
| Task ID | Status | Notes | Updated (UTC) |
| --- | --- | --- | --- |
| WORKER-GO-32-001 | DONE | Initial Go SDK scaffold with config binding, auth headers, claim/ack client, smoke sample, and unit tests. | 2025-11-17 |
| WORKER-GO-32-002 | DONE | Heartbeat/progress helpers, logging hooks, metrics, and jittered retries. | 2025-11-17 |
| WORKER-GO-33-001 | DONE | Artifact publish helpers, checksum hashing, metadata payload, idempotency guard. | 2025-11-17 |
| WORKER-GO-33-002 | DONE | Error classification/backoff helpers and structured failure reporting. | 2025-11-17 |
| WORKER-GO-34-001 | DONE | Backfill range execution helpers, watermark handshake, artifact dedupe verification. | 2025-11-17 |

View File

@@ -0,0 +1,45 @@
package main
import (
"context"
"log"
"time"
"git.stella-ops.org/stellaops/orchestrator/worker-sdk-go/pkg/workersdk"
)
func main() {
client, err := workersdk.New(workersdk.Config{
BaseURL: "http://localhost:8080",
APIKey: "dev-token",
TenantID: "local-tenant",
ProjectID: "demo-project",
})
if err != nil {
log.Fatalf("configure client: %v", err)
}
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
claim, err := client.Claim(ctx, workersdk.ClaimJobRequest{WorkerID: "demo-worker", Capabilities: []string{"pack-run"}})
if err != nil {
log.Fatalf("claim job: %v", err)
}
if claim == nil {
log.Println("no work available")
return
}
// ... perform work using claim.Payload ...
// heartbeat and progress
_ = client.Heartbeat(ctx, claim.JobID, claim.LeaseID)
_ = client.Progress(ctx, claim.JobID, claim.LeaseID, 50, "halfway")
if err := client.Ack(ctx, workersdk.AckJobRequest{JobID: claim.JobID, LeaseID: claim.LeaseID, Status: "succeeded"}); err != nil {
log.Fatalf("ack job: %v", err)
}
log.Printf("acknowledged job %s", claim.JobID)
}

View File

@@ -0,0 +1,3 @@
module git.stella-ops.org/stellaops/orchestrator/worker-sdk-go
go 1.21

View File

@@ -0,0 +1,31 @@
package transport
import (
"context"
"net/http"
)
// RoundTripper abstracts HTTP transport so we can stub in tests without
// depending on the default client.
type RoundTripper interface {
RoundTrip(*http.Request) (*http.Response, error)
}
// Client wraps an http.Client-like implementation.
type Client interface {
Do(req *http.Request) (*http.Response, error)
}
// DefaultClient returns a minimal http.Client with sane defaults.
func DefaultClient(rt RoundTripper) *http.Client {
if rt == nil {
return &http.Client{}
}
return &http.Client{Transport: rt}
}
// Do wraps an HTTP call using the provided Client.
func Do(ctx context.Context, c Client, req *http.Request) (*http.Response, error) {
req = req.WithContext(ctx)
return c.Do(req)
}

View File

@@ -0,0 +1,66 @@
package workersdk
import (
"context"
"crypto/sha256"
"encoding/hex"
"fmt"
"io"
)
// StorageClient is a minimal interface for artifact storage backends.
type StorageClient interface {
PutObject(ctx context.Context, key string, body io.Reader, metadata map[string]string) error
}
// ArtifactPublishRequest describes an artifact to upload.
type ArtifactPublishRequest struct {
JobID string
LeaseID string
ObjectKey string
Content io.Reader
ContentLength int64
ContentType string
ArtifactType string
IdempotencyKey string
Storage StorageClient
}
// ArtifactPublishResponse returns checksum metadata.
type ArtifactPublishResponse struct {
SHA256 string
Size int64
}
// PublishArtifact uploads artifact content with checksum metadata and idempotency guard.
func (c *Client) PublishArtifact(ctx context.Context, req ArtifactPublishRequest) (*ArtifactPublishResponse, error) {
if req.JobID == "" || req.LeaseID == "" {
return nil, fmt.Errorf("JobID and LeaseID are required")
}
if req.ObjectKey == "" {
return nil, fmt.Errorf("ObjectKey is required")
}
if req.Storage == nil {
return nil, fmt.Errorf("Storage client is required")
}
// Compute SHA256 while streaming.
hasher := sha256.New()
tee := io.TeeReader(req.Content, hasher)
// Wrap to enforce known length? length optional; storage client may use metadata.
metadata := map[string]string{
"x-stellaops-job-id": req.JobID,
"x-stellaops-lease": req.LeaseID,
"x-stellaops-type": req.ArtifactType,
"x-stellaops-ct": req.ContentType,
}
if req.IdempotencyKey != "" {
metadata["x-idempotency-key"] = req.IdempotencyKey
}
if err := req.Storage.PutObject(ctx, req.ObjectKey, tee, metadata); err != nil {
return nil, err
}
sum := hex.EncodeToString(hasher.Sum(nil))
return &ArtifactPublishResponse{SHA256: sum, Size: req.ContentLength}, nil
}

View File

@@ -0,0 +1,61 @@
package workersdk
import (
"bytes"
"context"
"io"
"testing"
)
type memStorage struct {
key string
data []byte
metadata map[string]string
}
func (m *memStorage) PutObject(ctx context.Context, key string, body io.Reader, metadata map[string]string) error {
b, err := io.ReadAll(body)
if err != nil {
return err
}
m.key = key
m.data = b
m.metadata = metadata
return nil
}
func TestPublishArtifact(t *testing.T) {
store := &memStorage{}
client, err := New(Config{BaseURL: "https://example"})
if err != nil {
t.Fatalf("new client: %v", err)
}
content := []byte("hello")
resp, err := client.PublishArtifact(context.Background(), ArtifactPublishRequest{
JobID: "job1",
LeaseID: "lease1",
ObjectKey: "artifacts/job1/output.txt",
Content: bytes.NewReader(content),
ContentLength: int64(len(content)),
ContentType: "text/plain",
ArtifactType: "log",
IdempotencyKey: "idem-1",
Storage: store,
})
if err != nil {
t.Fatalf("publish: %v", err)
}
if resp.SHA256 == "" || resp.Size != 5 {
t.Fatalf("unexpected resp: %+v", resp)
}
if store.key != "artifacts/job1/output.txt" {
t.Fatalf("key mismatch: %s", store.key)
}
if store.metadata["x-idempotency-key"] != "idem-1" {
t.Fatalf("idempotency missing")
}
if store.metadata["x-stellaops-job-id"] != "job1" {
t.Fatalf("job metadata missing")
}
}

View File

@@ -0,0 +1,86 @@
package workersdk
import (
"context"
"fmt"
"time"
)
// Range represents an inclusive backfill window.
type Range struct {
Start time.Time
End time.Time
}
// Validate ensures start <= end.
func (r Range) Validate() error {
if r.End.Before(r.Start) {
return fmt.Errorf("range end before start")
}
return nil
}
// WatermarkHandshake ensures the worker's view of the watermark matches orchestrator-provided value.
type WatermarkHandshake struct {
Expected string
Current string
}
func (w WatermarkHandshake) Validate() error {
if w.Expected == "" {
return fmt.Errorf("expected watermark required")
}
if w.Expected != w.Current {
return fmt.Errorf("watermark mismatch")
}
return nil
}
// Deduper tracks processed artifact digests to prevent duplicate publication.
type Deduper struct {
seen map[string]struct{}
}
// NewDeduper creates a deduper.
func NewDeduper() *Deduper {
return &Deduper{seen: make(map[string]struct{})}
}
// Seen returns true if digest already processed; marks new digests.
func (d *Deduper) Seen(digest string) bool {
if digest == "" {
return false
}
if _, ok := d.seen[digest]; ok {
return true
}
d.seen[digest] = struct{}{}
return false
}
// ExecuteRange iterates [start,end] by step days, invoking fn for each day.
func ExecuteRange(ctx context.Context, r Range, step time.Duration, fn func(context.Context, time.Time) error) error {
if err := r.Validate(); err != nil {
return err
}
if step <= 0 {
return fmt.Errorf("step must be positive")
}
for ts := r.Start; !ts.After(r.End); ts = ts.Add(step) {
if err := fn(ctx, ts); err != nil {
return err
}
}
return nil
}
// VerifyAndPublishArtifact wraps PublishArtifact with dedupe and watermark guard.
func (c *Client) VerifyAndPublishArtifact(ctx context.Context, wm WatermarkHandshake, dedupe *Deduper, req ArtifactPublishRequest) (*ArtifactPublishResponse, error) {
if err := wm.Validate(); err != nil {
return nil, err
}
if dedupe != nil && dedupe.Seen(req.IdempotencyKey) {
return nil, fmt.Errorf("duplicate artifact idempotency key")
}
return c.PublishArtifact(ctx, req)
}

View File

@@ -0,0 +1,85 @@
package workersdk
import (
"bytes"
"context"
"io"
"testing"
"time"
)
type stubStorage struct{}
func (stubStorage) PutObject(ctx context.Context, key string, body io.Reader, metadata map[string]string) error {
return nil
}
func TestRangeValidation(t *testing.T) {
r := Range{Start: time.Now(), End: time.Now().Add(-time.Hour)}
if err := r.Validate(); err == nil {
t.Fatalf("expected error for invalid range")
}
}
func TestExecuteRange(t *testing.T) {
start := time.Date(2025, 11, 15, 0, 0, 0, 0, time.UTC)
end := start.Add(48 * time.Hour)
r := Range{Start: start, End: end}
calls := 0
err := ExecuteRange(context.Background(), r, 24*time.Hour, func(ctx context.Context, ts time.Time) error {
calls++
return nil
})
if err != nil {
t.Fatalf("execute range: %v", err)
}
if calls != 3 {
t.Fatalf("expected 3 calls, got %d", calls)
}
}
func TestWatermarkMismatch(t *testing.T) {
wm := WatermarkHandshake{Expected: "abc", Current: "def"}
if err := wm.Validate(); err == nil {
t.Fatal("expected mismatch error")
}
}
func TestDeduper(t *testing.T) {
d := NewDeduper()
if d.Seen("sha") {
t.Fatal("should be new")
}
if !d.Seen("sha") {
t.Fatal("should detect duplicate")
}
}
func TestVerifyAndPublishArtifactDuplicate(t *testing.T) {
d := NewDeduper()
c, _ := New(Config{BaseURL: "https://x"})
d.Seen("idem1")
_, err := c.VerifyAndPublishArtifact(
context.Background(),
WatermarkHandshake{Expected: "w", Current: "w"},
d,
ArtifactPublishRequest{IdempotencyKey: "idem1", Storage: stubStorage{}, JobID: "j", LeaseID: "l", ObjectKey: "k", Content: bytes.NewReader([]byte{}), ArtifactType: "log"},
)
if err == nil {
t.Fatal("expected duplicate error")
}
}
func TestVerifyAndPublishArtifactWatermark(t *testing.T) {
d := NewDeduper()
c, _ := New(Config{BaseURL: "https://x"})
_, err := c.VerifyAndPublishArtifact(
context.Background(),
WatermarkHandshake{Expected: "w1", Current: "w2"},
d,
ArtifactPublishRequest{IdempotencyKey: "idem2", Storage: stubStorage{}, JobID: "j", LeaseID: "l", ObjectKey: "k", Content: bytes.NewReader([]byte{}), ArtifactType: "log"},
)
if err == nil {
t.Fatal("expected watermark error")
}
}

View File

@@ -0,0 +1,243 @@
package workersdk
import (
"bytes"
"context"
"encoding/json"
"fmt"
"io"
"net/http"
"net/url"
"path"
"time"
"git.stella-ops.org/stellaops/orchestrator/worker-sdk-go/internal/transport"
)
// Client provides job claim/acknowledge operations.
type Client struct {
baseURL *url.URL
apiKey string
tenantID string
projectID string
userAgent string
http transport.Client
logger Logger
metrics MetricsSink
}
// New creates a configured Client.
func New(cfg Config) (*Client, error) {
if err := cfg.validate(); err != nil {
return nil, err
}
parsed, err := url.Parse(cfg.BaseURL)
if err != nil {
return nil, fmt.Errorf("invalid BaseURL: %w", err)
}
ua := cfg.UserAgent
if ua == "" {
ua = "stellaops-worker-sdk-go/0.1"
}
return &Client{
baseURL: parsed,
apiKey: cfg.APIKey,
tenantID: cfg.TenantID,
projectID: cfg.ProjectID,
userAgent: ua,
http: cfg.httpClient(),
logger: cfg.logger(),
metrics: cfg.metrics(),
}, nil
}
// ClaimJobRequest represents a worker's desire to lease a job.
type ClaimJobRequest struct {
WorkerID string `json:"worker_id"`
Capabilities []string `json:"capabilities,omitempty"`
}
// ClaimJobResponse returns the leased job payload.
type ClaimJobResponse struct {
JobID string `json:"job_id"`
LeaseID string `json:"lease_id"`
ExpiresAt time.Time `json:"expires_at"`
JobType string `json:"job_type"`
Payload json.RawMessage `json:"payload"`
RetryAfter int `json:"retry_after_seconds,omitempty"`
NotBefore *time.Time `json:"not_before,omitempty"`
TraceID string `json:"trace_id,omitempty"`
}
// AckJobRequest represents completion of a job.
type AckJobRequest struct {
JobID string `json:"job_id"`
LeaseID string `json:"lease_id"`
Status string `json:"status"`
Message string `json:"message,omitempty"`
Rotating string `json:"rotating_token,omitempty"`
}
// Claim requests the next available job for the worker.
func (c *Client) Claim(ctx context.Context, req ClaimJobRequest) (*ClaimJobResponse, error) {
if req.WorkerID == "" {
return nil, fmt.Errorf("WorkerID is required")
}
endpoint := c.resolve("/api/jobs/lease")
resp, err := c.doClaim(ctx, endpoint, req)
if err == nil {
c.metrics.IncClaimed()
}
c.logger.Info(ctx, "claim", map[string]any{"worker_id": req.WorkerID, "err": err})
return resp, err
}
// Ack acknowledges job completion or failure.
func (c *Client) Ack(ctx context.Context, req AckJobRequest) error {
if req.JobID == "" || req.LeaseID == "" || req.Status == "" {
return fmt.Errorf("JobID, LeaseID, and Status are required")
}
endpoint := c.resolve(path.Join("/api/jobs", req.JobID, "ack"))
payload, err := json.Marshal(req)
if err != nil {
return fmt.Errorf("marshal ack request: %w", err)
}
httpReq, err := http.NewRequest(http.MethodPost, endpoint, bytes.NewReader(payload))
if err != nil {
return err
}
c.applyHeaders(httpReq)
httpReq.Header.Set("Content-Type", "application/json")
resp, err := transport.Do(ctx, c.http, httpReq)
if err != nil {
return err
}
defer resp.Body.Close()
if resp.StatusCode >= 300 {
b, _ := io.ReadAll(io.LimitReader(resp.Body, 8<<10))
return fmt.Errorf("ack failed: %s (%s)", resp.Status, string(b))
}
c.metrics.IncAck(req.Status)
return nil
}
// Heartbeat reports liveness for a job lease.
func (c *Client) Heartbeat(ctx context.Context, jobID, leaseID string) error {
if jobID == "" || leaseID == "" {
return fmt.Errorf("JobID and LeaseID are required")
}
endpoint := c.resolve(path.Join("/api/jobs", jobID, "heartbeat"))
payload, _ := json.Marshal(map[string]string{"lease_id": leaseID})
httpReq, err := http.NewRequest(http.MethodPost, endpoint, bytes.NewReader(payload))
if err != nil {
return err
}
c.applyHeaders(httpReq)
httpReq.Header.Set("Content-Type", "application/json")
start := time.Now()
resp, err := transport.Do(ctx, c.http, httpReq)
if err != nil {
c.metrics.IncHeartbeatFailures()
return err
}
defer resp.Body.Close()
if resp.StatusCode >= 300 {
b, _ := io.ReadAll(io.LimitReader(resp.Body, 8<<10))
c.metrics.IncHeartbeatFailures()
return fmt.Errorf("heartbeat failed: %s (%s)", resp.Status, string(b))
}
c.metrics.ObserveHeartbeatLatency(time.Since(start).Seconds())
return nil
}
// Progress reports worker progress (0-100) with optional message.
func (c *Client) Progress(ctx context.Context, jobID, leaseID string, pct int, message string) error {
if pct < 0 || pct > 100 {
return fmt.Errorf("pct must be 0-100")
}
payload, _ := json.Marshal(map[string]any{
"lease_id": leaseID,
"progress": pct,
"message": message,
})
endpoint := c.resolve(path.Join("/api/jobs", jobID, "progress"))
httpReq, err := http.NewRequest(http.MethodPost, endpoint, bytes.NewReader(payload))
if err != nil {
return err
}
c.applyHeaders(httpReq)
httpReq.Header.Set("Content-Type", "application/json")
resp, err := transport.Do(ctx, c.http, httpReq)
if err != nil {
return err
}
defer resp.Body.Close()
if resp.StatusCode >= 300 {
b, _ := io.ReadAll(io.LimitReader(resp.Body, 8<<10))
return fmt.Errorf("progress failed: %s (%s)", resp.Status, string(b))
}
return nil
}
func (c *Client) doClaim(ctx context.Context, endpoint string, req ClaimJobRequest) (*ClaimJobResponse, error) {
payload, err := json.Marshal(req)
if err != nil {
return nil, fmt.Errorf("marshal claim request: %w", err)
}
httpReq, err := http.NewRequest(http.MethodPost, endpoint, bytes.NewReader(payload))
if err != nil {
return nil, err
}
c.applyHeaders(httpReq)
httpReq.Header.Set("Content-Type", "application/json")
resp, err := transport.Do(ctx, c.http, httpReq)
if err != nil {
return nil, err
}
defer resp.Body.Close()
if resp.StatusCode == http.StatusNoContent {
return nil, nil // no work available
}
if resp.StatusCode >= 300 {
b, _ := io.ReadAll(io.LimitReader(resp.Body, 8<<10))
return nil, fmt.Errorf("claim failed: %s (%s)", resp.Status, string(b))
}
var out ClaimJobResponse
decoder := json.NewDecoder(resp.Body)
decoder.DisallowUnknownFields()
if err := decoder.Decode(&out); err != nil {
return nil, fmt.Errorf("decode claim response: %w", err)
}
return &out, nil
}
func (c *Client) applyHeaders(r *http.Request) {
if c.apiKey != "" {
r.Header.Set("Authorization", "Bearer "+c.apiKey)
}
if c.tenantID != "" {
r.Header.Set("X-StellaOps-Tenant", c.tenantID)
}
if c.projectID != "" {
r.Header.Set("X-StellaOps-Project", c.projectID)
}
r.Header.Set("Accept", "application/json")
if c.userAgent != "" {
r.Header.Set("User-Agent", c.userAgent)
}
}
func (c *Client) resolve(p string) string {
clone := *c.baseURL
clone.Path = path.Join(clone.Path, p)
return clone.String()
}

View File

@@ -0,0 +1,136 @@
package workersdk
import (
"context"
"encoding/json"
"net/http"
"net/http/httptest"
"testing"
"time"
)
type claimRecorded struct {
Method string
Path string
Auth string
Tenant string
Project string
Body ClaimJobRequest
}
type ackRecorded struct {
Method string
Path string
Body AckJobRequest
}
func TestClaimAndAck(t *testing.T) {
var claimRec claimRecorded
var ackRec ackRecorded
srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
switch r.URL.Path {
case "/api/jobs/lease":
claimRec.Method = r.Method
claimRec.Path = r.URL.Path
claimRec.Auth = r.Header.Get("Authorization")
claimRec.Tenant = r.Header.Get("X-StellaOps-Tenant")
claimRec.Project = r.Header.Get("X-StellaOps-Project")
if err := json.NewDecoder(r.Body).Decode(&claimRec.Body); err != nil {
t.Fatalf("decode claim: %v", err)
}
resp := ClaimJobResponse{
JobID: "123",
LeaseID: "lease-1",
ExpiresAt: time.Date(2025, 11, 17, 0, 0, 0, 0, time.UTC),
JobType: "demo",
Payload: json.RawMessage(`{"key":"value"}`),
}
w.Header().Set("Content-Type", "application/json")
json.NewEncoder(w).Encode(resp)
case "/api/jobs/123/heartbeat":
w.WriteHeader(http.StatusAccepted)
case "/api/jobs/123/progress":
w.WriteHeader(http.StatusAccepted)
default:
ackRec.Method = r.Method
ackRec.Path = r.URL.Path
if err := json.NewDecoder(r.Body).Decode(&ackRec.Body); err != nil {
t.Fatalf("decode ack: %v", err)
}
w.WriteHeader(http.StatusAccepted)
}
}))
defer srv.Close()
client, err := New(Config{
BaseURL: srv.URL,
APIKey: "token-1",
TenantID: "tenant-a",
ProjectID: "project-1",
})
if err != nil {
t.Fatalf("new client: %v", err)
}
ctx := context.Background()
claimResp, err := client.Claim(ctx, ClaimJobRequest{WorkerID: "worker-7", Capabilities: []string{"scan"}})
if err != nil {
t.Fatalf("claim: %v", err)
}
if claimRec.Method != http.MethodPost || claimRec.Path != "/api/jobs/lease" {
t.Fatalf("unexpected claim method/path: %s %s", claimRec.Method, claimRec.Path)
}
if claimRec.Auth != "Bearer token-1" {
t.Fatalf("auth header mismatch: %s", claimRec.Auth)
}
if claimRec.Tenant != "tenant-a" || claimRec.Project != "project-1" {
t.Fatalf("tenant/project headers missing: %s %s", claimRec.Tenant, claimRec.Project)
}
if claimRec.Body.WorkerID != "worker-7" {
t.Fatalf("worker id missing")
}
if claimResp == nil || claimResp.JobID != "123" || claimResp.LeaseID != "lease-1" {
t.Fatalf("claim response mismatch: %+v", claimResp)
}
err = client.Ack(ctx, AckJobRequest{JobID: claimResp.JobID, LeaseID: claimResp.LeaseID, Status: "succeeded"})
if err != nil {
t.Fatalf("ack error: %v", err)
}
if err := client.Heartbeat(ctx, claimResp.JobID, claimResp.LeaseID); err != nil {
t.Fatalf("heartbeat error: %v", err)
}
if err := client.Progress(ctx, claimResp.JobID, claimResp.LeaseID, 50, "halfway"); err != nil {
t.Fatalf("progress error: %v", err)
}
if ackRec.Method != http.MethodPost {
t.Fatalf("ack method mismatch: %s", ackRec.Method)
}
if ackRec.Path != "/api/jobs/123/ack" {
t.Fatalf("ack path mismatch: %s", ackRec.Path)
}
if ackRec.Body.Status != "succeeded" || ackRec.Body.JobID != "123" {
t.Fatalf("ack body mismatch: %+v", ackRec.Body)
}
}
func TestClaimMissingWorker(t *testing.T) {
client, err := New(Config{BaseURL: "https://example.invalid"})
if err != nil {
t.Fatalf("new client: %v", err)
}
if _, err := client.Claim(context.Background(), ClaimJobRequest{}); err == nil {
t.Fatal("expected error for missing worker id")
}
}
func TestConfigValidation(t *testing.T) {
if _, err := New(Config{}); err == nil {
t.Fatal("expected error for missing base url")
}
}

View File

@@ -0,0 +1,49 @@
package workersdk
import (
"errors"
"net/http"
"strings"
"git.stella-ops.org/stellaops/orchestrator/worker-sdk-go/internal/transport"
)
// Config holds SDK configuration.
type Config struct {
BaseURL string
APIKey string
TenantID string
ProjectID string
UserAgent string
Client transport.Client
Logger Logger
Metrics MetricsSink
}
func (c *Config) validate() error {
if strings.TrimSpace(c.BaseURL) == "" {
return errors.New("BaseURL is required")
}
return nil
}
func (c *Config) httpClient() transport.Client {
if c.Client != nil {
return c.Client
}
return transport.DefaultClient(http.DefaultTransport)
}
func (c *Config) logger() Logger {
if c.Logger != nil {
return c.Logger
}
return NoopLogger{}
}
func (c *Config) metrics() MetricsSink {
if c.Metrics != nil {
return c.Metrics
}
return NoopMetrics{}
}

View File

@@ -0,0 +1,29 @@
package workersdk
// ErrorCode represents orchestrator error categories.
type ErrorCode string
const (
ErrorCodeTemporary ErrorCode = "temporary"
ErrorCodePermanent ErrorCode = "permanent"
ErrorCodeFatal ErrorCode = "fatal"
ErrorCodeUnauth ErrorCode = "unauthorized"
ErrorCodeQuota ErrorCode = "quota_exceeded"
ErrorCodeValidation ErrorCode = "validation"
)
// ErrorClassification maps HTTP status to codes and retryability.
func ErrorClassification(status int) (ErrorCode, bool) {
switch {
case status == 401 || status == 403:
return ErrorCodeUnauth, false
case status >= 500 && status < 600:
return ErrorCodeTemporary, true
case status == 429:
return ErrorCodeQuota, true
case status >= 400 && status < 500:
return ErrorCodePermanent, false
default:
return "", false
}
}

View File

@@ -0,0 +1,24 @@
package workersdk
import "testing"
func TestErrorClassification(t *testing.T) {
cases := []struct {
status int
code ErrorCode
retry bool
}{
{500, ErrorCodeTemporary, true},
{503, ErrorCodeTemporary, true},
{429, ErrorCodeQuota, true},
{401, ErrorCodeUnauth, false},
{400, ErrorCodePermanent, false},
{404, ErrorCodePermanent, false},
}
for _, c := range cases {
code, retry := ErrorClassification(c.status)
if code != c.code || retry != c.retry {
t.Fatalf("status %d -> got %s retry %v", c.status, code, retry)
}
}
}

View File

@@ -0,0 +1,15 @@
package workersdk
import "context"
// Logger is a minimal structured logger interface.
type Logger interface {
Info(ctx context.Context, msg string, fields map[string]any)
Error(ctx context.Context, msg string, fields map[string]any)
}
// NoopLogger is used when no logger provided.
type NoopLogger struct{}
func (NoopLogger) Info(_ context.Context, _ string, _ map[string]any) {}
func (NoopLogger) Error(_ context.Context, _ string, _ map[string]any) {}

View File

@@ -0,0 +1,17 @@
package workersdk
// MetricsSink allows callers to wire Prometheus or other metrics systems.
type MetricsSink interface {
IncClaimed()
IncAck(status string)
ObserveHeartbeatLatency(seconds float64)
IncHeartbeatFailures()
}
// NoopMetrics is the default sink when none is provided.
type NoopMetrics struct{}
func (NoopMetrics) IncClaimed() {}
func (NoopMetrics) IncAck(_ string) {}
func (NoopMetrics) ObserveHeartbeatLatency(_ float64) {}
func (NoopMetrics) IncHeartbeatFailures() {}

View File

@@ -0,0 +1,53 @@
package workersdk
import (
"context"
"math/rand"
"time"
)
// RetryPolicy defines retry behavior.
type RetryPolicy struct {
MaxAttempts int
BaseDelay time.Duration
MaxDelay time.Duration
Jitter float64 // between 0 and 1, represents percentage of jitter.
}
// DefaultRetryPolicy returns exponential backoff with jitter suitable for worker I/O.
func DefaultRetryPolicy() RetryPolicy {
return RetryPolicy{MaxAttempts: 5, BaseDelay: 200 * time.Millisecond, MaxDelay: 5 * time.Second, Jitter: 0.2}
}
// Retry executes fn with retries according to policy.
func Retry(ctx context.Context, policy RetryPolicy, fn func() error) error {
if policy.MaxAttempts <= 0 {
policy = DefaultRetryPolicy()
}
delay := policy.BaseDelay
for attempt := 1; attempt <= policy.MaxAttempts; attempt++ {
err := fn()
if err == nil {
return nil
}
if attempt == policy.MaxAttempts {
return err
}
// apply jitter
jitter := 1 + (policy.Jitter * (rand.Float64()*2 - 1))
sleepFor := time.Duration(float64(delay) * jitter)
if sleepFor > policy.MaxDelay {
sleepFor = policy.MaxDelay
}
select {
case <-ctx.Done():
return ctx.Err()
case <-time.After(sleepFor):
}
delay *= 2
if delay > policy.MaxDelay {
delay = policy.MaxDelay
}
}
return nil
}