feat: python client (#109)

* (wip) python SDK

* feat: python client, initial version finished

* fix: add curl to migration dockerfile

* add insecure option for grpc

* create docs and publishing workflow

* delete old hatchet folder
This commit is contained in:
abelanger5
2024-01-20 06:18:25 -08:00
committed by GitHub
parent 52fde1e704
commit 0c94f0d933
52 changed files with 2800 additions and 65 deletions

View File

@@ -0,0 +1,27 @@
name: Publish Python SDK
on:
workflow_dispatch:
jobs:
publish:
runs-on: ubuntu-latest
steps:
- name: Checkout Repository
uses: actions/checkout@v2
- name: Set up Python
uses: actions/setup-python@v2
with:
python-version: "3.x"
- name: Install Poetry
run: |
pipx install poetry==1.7.1
- name: Run publish.sh script
run: |
cd python-client
sh publish.sh
env:
POETRY_PYPI_TOKEN_PYPI: ${{ secrets.POETRY_PYPI_TOKEN_PYPI }}

View File

@@ -55,6 +55,10 @@ tasks:
generate-proto:
cmds:
- sh ./hack/proto/proto.sh
generate-proto-python:
dir: ./python-client
cmds:
- sh ./generate.sh
generate-sqlc:
cmds:
- npx --yes prisma migrate diff --from-empty --to-schema-datasource prisma/schema.prisma --script > internal/repository/prisma/dbsqlc/schema.sql

View File

@@ -3,6 +3,9 @@
FROM golang:1.21-alpine as base
WORKDIR /hatchet
# curl is needed for things like signaling cloudsql proxy container to stop after a migration
RUN apk update && apk add --no-cache curl
COPY go.mod go.sum ./
RUN go mod download

View File

@@ -6,6 +6,8 @@ import (
"os"
"sync"
"github.com/spf13/cobra"
"github.com/hatchet-dev/hatchet/internal/config/loader"
"github.com/hatchet-dev/hatchet/internal/services/admin"
"github.com/hatchet-dev/hatchet/internal/services/dispatcher"
@@ -17,7 +19,6 @@ import (
"github.com/hatchet-dev/hatchet/internal/services/ticker"
"github.com/hatchet-dev/hatchet/internal/telemetry"
"github.com/hatchet-dev/hatchet/pkg/cmdutils"
"github.com/spf13/cobra"
"net/http"
_ "net/http/pprof"
@@ -141,8 +142,7 @@ func startEngineOrDie(cf *loader.ConfigLoader, interruptCh <-chan interface{}) {
return
}
// create the grpc server
s, err := grpc.NewServer(
grpcOpts := []grpc.ServerOpt{
grpc.WithIngestor(ei),
grpc.WithDispatcher(d),
grpc.WithAdmin(adminSvc),
@@ -150,6 +150,15 @@ func startEngineOrDie(cf *loader.ConfigLoader, interruptCh <-chan interface{}) {
grpc.WithTLSConfig(sc.TLSConfig),
grpc.WithPort(sc.Runtime.GRPCPort),
grpc.WithBindAddress(sc.Runtime.GRPCBindAddress),
}
if sc.Runtime.GRPCInsecure {
grpcOpts = append(grpcOpts, grpc.WithInsecure())
}
// create the grpc server
s, err := grpc.NewServer(
grpcOpts...,
)
if err != nil {

View File

@@ -1,5 +1,6 @@
{
"index": "Introduction",
"quickstart": "Quickstart",
"go-sdk": "Go SDK"
"go-sdk": "Go SDK",
"python-sdk": "Python SDK"
}

View File

@@ -1,4 +1,5 @@
{
"setup": "Setup",
"creating-a-worker": "Creating a Worker",
"creating-a-workflow": "Creating a Workflow",
"pushing-events": "Pushing Events",

View File

@@ -0,0 +1,33 @@
# Setup
## Installation
To install the Hatchet Go SDK:
```sh
go get github.com/hatchet-dev/hatchet
```
## Usage
You should have generated a set of client certificates (or received them through Hatchet cloud). These certificates are `ca.cert`, `client-worker.pem`, and `client-worker.key`. You should also make note of the Hatchet server address and the tenant ID you'd like to connect to.
```sh
HATCHET_CLIENT_TENANT_ID=<tenant-id>
HATCHET_CLIENT_TLS_ROOT_CA_FILE=./certs/ca.cert
HATCHET_CLIENT_TLS_CERT_FILE=./certs/client-worker.pem
HATCHET_CLIENT_TLS_KEY_FILE=./certs/client-worker.key
HATCHET_CLIENT_TLS_SERVER_NAME=<hatchet-domain>
```
Assuming these variables are set in the environment, you can then initialize a client with the following code:
```go
c, err := client.New(
client.WithHostPort("<hatchet-domain>", 443),
)
if err != nil {
return fmt.Errorf("error creating client: %w", err)
}
```

View File

@@ -0,0 +1,6 @@
{
"setup": "Setup",
"creating-a-worker": "Creating a Worker",
"creating-a-workflow": "Creating a Workflow",
"pushing-events": "Pushing Events"
}

View File

@@ -0,0 +1,20 @@
# Creating a Worker
Workers can be created via the `hatchet.worker()` method, after instantiating a `hatchet` instance. It will automatically read in any `HATCHET_CLIENT` environment variables, which can be set in the process by using something like `dotenv`. For example:
```py
from hatchet-sdk import Hatchet
from dotenv import load_dotenv
load_dotenv()
hatchet = Hatchet()
# workflow code...
worker = hatchet.worker('test-worker')
worker.register_workflow(workflow)
# worker.start() is blocking
worker.start()
```

View File

@@ -0,0 +1,84 @@
# Creating a Workflow
To create a workflow, simply create a new class and use the `hatchet.workflow` and `hatchet.step` decorators to define the structure of your workflow. For example, a simple 2-step workflow would look like:
```py
from hatchet-sdk import Hatchet
hatchet = Hatchet()
@hatchet.workflow(on_events=["user:create"])
class MyWorkflow:
@hatchet.step()
def step1(self, context):
print("executed step1")
pass
@hatchet.step(parents=["step1"])
def step2(self, context):
print("executed step2")
pass
```
You'll notice that the workflow defines a workflow trigger (in this case, `on_events`), and the workflow definition. The workflow definition is a series of steps, which can be defined using the `hatchet.step` decorator. Each step must be a method on the class, and must accept a `context` argument. The `context` argument is a `Context` object, which contains information about the workflow, such as the input data and the output data of previous steps.
To create multi-step workflows, you can use `parents` to define the steps which the current step depends on.
## Getting Access to the Input Data
You can get access to the workflow's input data, such as the event data or other specified input data, by using the `context.workflow_input()` method on the `context`. For example, given the following event:
```json
{
"name": "test"
}
```
You can get access to the event data by doing the following:
```py
@hatchet.workflow(on_events=["user:create"])
class MyWorkflow:
@hatchet.step()
def step1(self, context : Context):
print("executed step1", context.workflow_input())
pass
```
## Step Outputs
Step outputs should be a `dict` and are optional. For example:
```py
@hatchet.workflow(on_events=["user:create"])
class MyWorkflow:
@hatchet.step()
def step1(self, context : Context):
return {
"step-1-output": "test"
}
```
Future steps can access this output by calling `context.step_output("<step>")`. In this example, a future step could access this data via `context.step_output("step1")`.
## Cron Schedules
You can declare a cron schedule by passing `on_crons` to the `hatchet.workflow` decorator. For example, to trigger a workflow every 5 minutes, you can do the following:
```go
from hatchet-sdk import Hatchet
hatchet = Hatchet()
@hatchet.workflow(on_crons=["*/5 * * * *"])
class MyWorkflow:
@hatchet.step()
def step1(self, context):
print("executed step1")
pass
@hatchet.step(parents=["step1"])
def step2(self, context):
print("executed step2")
pass
```

View File

@@ -0,0 +1,18 @@
# Pushing Events
Events can be pushed via the client's `client.event.push` method:
```py
from hatchet import new_client
client = new_client()
client.event.push(
"user:create",
{
"test": "test"
}
)
```
Events should be JSON serializable, so should generally use a `dict` or a type that serializes into a JSON object.

View File

@@ -0,0 +1,28 @@
# Setup
## Installation
If using `pip`, you can run:
```sh
pip install hatchet-sdk
```
If using `poetry`:
```sh
poetry add hatchet-sdk
```
## Usage
You should have generated a set of client certificates (or received them through Hatchet cloud). These certificates are `ca.cert`, `client-worker.pem`, and `client-worker.key`. You should also make note of the Hatchet server address and the tenant ID you'd like to connect to.
```sh
HATCHET_CLIENT_TENANT_ID=<tenant-id>
HATCHET_CLIENT_TLS_ROOT_CA_FILE=./certs/ca.cert
HATCHET_CLIENT_TLS_CERT_FILE=./certs/client-worker.pem
HATCHET_CLIENT_TLS_KEY_FILE=./certs/client-worker.key
HATCHET_CLIENT_TLS_SERVER_NAME=<hatchet-domain>
HATCHET_CLIENT_HOST_PORT=<hatchet-domain>:443
```

View File

@@ -3,14 +3,15 @@ package server
import (
"crypto/tls"
"github.com/rs/zerolog"
"github.com/spf13/viper"
"github.com/hatchet-dev/hatchet/internal/auth/cookie"
"github.com/hatchet-dev/hatchet/internal/config/database"
"github.com/hatchet-dev/hatchet/internal/config/shared"
"github.com/hatchet-dev/hatchet/internal/services/ingestor"
"github.com/hatchet-dev/hatchet/internal/taskqueue"
"github.com/hatchet-dev/hatchet/internal/validator"
"github.com/rs/zerolog"
"github.com/spf13/viper"
)
type ServerConfigFile struct {
@@ -42,6 +43,9 @@ type ConfigFileRuntime struct {
// GRPCBindAddress is the address that the grpc server binds to. Should set to 0.0.0.0 if binding in docker container.
GRPCBindAddress string `mapstructure:"grpcBindAddress" json:"grpcBindAddress,omitempty" default:"127.0.0.1"`
// GRPCInsecure controls whether the grpc server is insecure or uses certs
GRPCInsecure bool `mapstructure:"grpcInsecure" json:"grpcInsecure,omitempty" default:"false"`
}
type ConfigFileAuth struct {
@@ -118,6 +122,7 @@ func BindAllEnv(v *viper.Viper) {
v.BindEnv("runtime.url", "SERVER_URL")
v.BindEnv("runtime.grpcPort", "SERVER_GRPC_PORT")
v.BindEnv("runtime.grpcBindAddress", "SERVER_GRPC_BIND_ADDRESS")
v.BindEnv("runtime.grpcInsecure", "SERVER_GRPC_INSECURE")
v.BindEnv("services", "SERVER_SERVICES")
// auth options

View File

@@ -85,3 +85,30 @@ INSERT INTO "JobRunLookupData" (
SET
"data" = jsonb_set("JobRunLookupData"."data", @fieldPath::text[], @jsonData::jsonb, true),
"updatedAt" = CURRENT_TIMESTAMP;
-- name: UpdateJobRunLookupDataWithStepRun :exec
WITH readable_id AS (
SELECT "readableId"
FROM "Step"
WHERE "id" = (
SELECT "stepId"
FROM "StepRun"
WHERE "id" = @stepRunId::uuid
)
)
UPDATE "JobRunLookupData"
SET
"data" = jsonb_set(
"JobRunLookupData"."data",
ARRAY['steps', (SELECT "readableId" FROM readable_id)],
@jsonData::jsonb,
true
),
"updatedAt" = CURRENT_TIMESTAMP
WHERE
"jobRunId" = (
SELECT "jobRunId"
FROM "StepRun"
WHERE "id" = @stepRunId::uuid
)
AND "tenantId" = @tenantId::uuid;

View File

@@ -138,6 +138,45 @@ func (q *Queries) UpdateJobRun(ctx context.Context, db DBTX, arg UpdateJobRunPar
return &i, err
}
const updateJobRunLookupDataWithStepRun = `-- name: UpdateJobRunLookupDataWithStepRun :exec
WITH readable_id AS (
SELECT "readableId"
FROM "Step"
WHERE "id" = (
SELECT "stepId"
FROM "StepRun"
WHERE "id" = $2::uuid
)
)
UPDATE "JobRunLookupData"
SET
"data" = jsonb_set(
"JobRunLookupData"."data",
ARRAY['steps', (SELECT "readableId" FROM readable_id)],
$1::jsonb,
true
),
"updatedAt" = CURRENT_TIMESTAMP
WHERE
"jobRunId" = (
SELECT "jobRunId"
FROM "StepRun"
WHERE "id" = $2::uuid
)
AND "tenantId" = $3::uuid
`
type UpdateJobRunLookupDataWithStepRunParams struct {
Jsondata []byte `json:"jsondata"`
Steprunid pgtype.UUID `json:"steprunid"`
Tenantid pgtype.UUID `json:"tenantid"`
}
func (q *Queries) UpdateJobRunLookupDataWithStepRun(ctx context.Context, db DBTX, arg UpdateJobRunLookupDataWithStepRunParams) error {
_, err := db.Exec(ctx, updateJobRunLookupDataWithStepRun, arg.Jsondata, arg.Steprunid, arg.Tenantid)
return err
}
const upsertJobRunLookupData = `-- name: UpsertJobRunLookupData :exec
INSERT INTO "JobRunLookupData" (
"id",

View File

@@ -2,7 +2,8 @@ package prisma
import (
"context"
"encoding/json"
"fmt"
"strings"
"time"
"github.com/jackc/pgx/v5"
@@ -123,32 +124,70 @@ func (s *stepRunRepository) ListStepRuns(tenantId string, opts *repository.ListS
).Exec(context.Background())
}
var retrier = func(l *zerolog.Logger, f func() error) error {
retries := 0
for {
err := f()
if err != nil {
// deadlock detected, retry
if strings.Contains(err.Error(), "deadlock detected") {
retries++
if retries > 3 {
return fmt.Errorf("could not update job run lookup data: %w", err)
} else {
l.Err(err).Msgf("deadlock detected, retry %d", retries)
time.Sleep(100 * time.Millisecond)
}
} else {
return fmt.Errorf("could not update job run lookup data: %w", err)
}
}
if err == nil {
if retries > 0 {
l.Info().Msgf("deadlock resolved after %d retries", retries)
}
break
}
}
return nil
}
func (s *stepRunRepository) UpdateStepRun(tenantId, stepRunId string, opts *repository.UpdateStepRunOpts) (*db.StepRunModel, error) {
if err := s.v.Validate(opts); err != nil {
return nil, err
}
updateParams, resolveJobRunParams, resolveLaterStepRunsParams, err := getUpdateParams(tenantId, stepRunId, opts)
updateParams, updateJobRunLookupDataParams, resolveJobRunParams, resolveLaterStepRunsParams, err := getUpdateParams(tenantId, stepRunId, opts)
if err != nil {
return nil, err
}
tx, err := s.pool.Begin(context.Background())
err = retrier(s.l, func() error {
tx, err := s.pool.Begin(context.Background())
if err != nil {
return nil, err
}
if err != nil {
return err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(context.Background(), s.l, tx.Rollback)
err = s.updateStepRun(tx, tenantId, updateParams, resolveJobRunParams, resolveLaterStepRunsParams)
err = s.updateStepRun(tx, tenantId, updateParams, updateJobRunLookupDataParams, resolveJobRunParams, resolveLaterStepRunsParams)
if err != nil {
return nil, err
}
if err != nil {
return err
}
err = tx.Commit(context.Background())
err = tx.Commit(context.Background())
return err
})
if err != nil {
return nil, err
@@ -176,7 +215,7 @@ func (s *stepRunRepository) QueueStepRun(tenantId, stepRunId string, opts *repos
return nil, err
}
updateParams, resolveJobRunParams, resolveLaterStepRunsParams, err := getUpdateParams(tenantId, stepRunId, opts)
updateParams, updateJobRunLookupDataParams, resolveJobRunParams, resolveLaterStepRunsParams, err := getUpdateParams(tenantId, stepRunId, opts)
if err != nil {
return nil, err
@@ -204,7 +243,7 @@ func (s *stepRunRepository) QueueStepRun(tenantId, stepRunId string, opts *repos
return nil, repository.StepRunIsNotPendingErr
}
err = s.updateStepRun(tx, tenantId, updateParams, resolveJobRunParams, resolveLaterStepRunsParams)
err = s.updateStepRun(tx, tenantId, updateParams, updateJobRunLookupDataParams, resolveJobRunParams, resolveLaterStepRunsParams)
if err != nil {
return nil, err
@@ -239,6 +278,7 @@ func getUpdateParams(
opts *repository.UpdateStepRunOpts,
) (
updateParams dbsqlc.UpdateStepRunParams,
updateJobRunLookupDataParams *dbsqlc.UpdateJobRunLookupDataWithStepRunParams,
resolveJobRunParams dbsqlc.ResolveJobRunStatusParams,
resolveLaterStepRunsParams dbsqlc.ResolveLaterStepRunsParams,
err error,
@@ -251,6 +291,14 @@ func getUpdateParams(
Tenantid: pgTenantId,
}
if opts.Output != nil {
updateJobRunLookupDataParams = &dbsqlc.UpdateJobRunLookupDataWithStepRunParams{
Steprunid: pgStepRunId,
Tenantid: pgTenantId,
Jsondata: opts.Output,
}
}
resolveJobRunParams = dbsqlc.ResolveJobRunStatusParams{
Steprunid: pgStepRunId,
Tenantid: pgTenantId,
@@ -281,18 +329,18 @@ func getUpdateParams(
runStatus := dbsqlc.NullStepRunStatus{}
if err := runStatus.Scan(string(*opts.Status)); err != nil {
return updateParams, resolveJobRunParams, resolveLaterStepRunsParams, err
return updateParams, nil, resolveJobRunParams, resolveLaterStepRunsParams, err
}
updateParams.Status = runStatus
}
if opts.Input != nil {
updateParams.Input = []byte(json.RawMessage(*opts.Input))
updateParams.Input = opts.Input
}
if opts.Output != nil {
updateParams.Output = []byte(json.RawMessage(*opts.Output))
updateParams.Output = opts.Output
}
if opts.Error != nil {
@@ -307,30 +355,33 @@ func getUpdateParams(
updateParams.CancelledReason = sqlchelpers.TextFromStr(*opts.CancelledReason)
}
return updateParams, resolveJobRunParams, resolveLaterStepRunsParams, nil
return updateParams, updateJobRunLookupDataParams, resolveJobRunParams, resolveLaterStepRunsParams, nil
}
func (s *stepRunRepository) updateStepRun(
tx pgx.Tx,
tenantId string,
updateParams dbsqlc.UpdateStepRunParams, resolveJobRunParams dbsqlc.ResolveJobRunStatusParams, resolveLaterStepRunsParams dbsqlc.ResolveLaterStepRunsParams,
updateParams dbsqlc.UpdateStepRunParams,
updateJobRunLookupDataParams *dbsqlc.UpdateJobRunLookupDataWithStepRunParams,
resolveJobRunParams dbsqlc.ResolveJobRunStatusParams,
resolveLaterStepRunsParams dbsqlc.ResolveLaterStepRunsParams,
) error {
_, err := s.queries.UpdateStepRun(context.Background(), tx, updateParams)
if err != nil {
return err
return fmt.Errorf("could not update step run: %w", err)
}
_, err = s.queries.ResolveLaterStepRuns(context.Background(), tx, resolveLaterStepRunsParams)
if err != nil {
return err
return fmt.Errorf("could not resolve later step runs: %w", err)
}
jobRun, err := s.queries.ResolveJobRunStatus(context.Background(), tx, resolveJobRunParams)
if err != nil {
return err
return fmt.Errorf("could not resolve job run status: %w", err)
}
resolveWorkflowRunParams := dbsqlc.ResolveWorkflowRunStatusParams{
@@ -341,7 +392,16 @@ func (s *stepRunRepository) updateStepRun(
_, err = s.queries.ResolveWorkflowRunStatus(context.Background(), tx, resolveWorkflowRunParams)
if err != nil {
return err
return fmt.Errorf("could not resolve workflow run status: %w", err)
}
// update the job run lookup data if not nil
if updateJobRunLookupDataParams != nil {
err = s.queries.UpdateJobRunLookupDataWithStepRun(context.Background(), tx, *updateJobRunLookupDataParams)
if err != nil {
return fmt.Errorf("could not update job run lookup data: %w", err)
}
}
return nil

View File

@@ -186,6 +186,11 @@ func (t *tickerRepository) UpdateStaleTickers(onStale func(tickerId string, getV
return err
}
// if there are no active tickers, we can't reassign the stale tickers
if len(activeTickers) == 0 {
return nil
}
tickersToDelete := make([]pgtype.UUID, 0)
for i, ticker := range staleTickers {

View File

@@ -6,7 +6,6 @@ import (
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
"github.com/steebchen/prisma-client-go/runtime/types"
)
type ListAllStepRunsOpts struct {
@@ -44,9 +43,9 @@ type UpdateStepRunOpts struct {
Error *string
Input *types.JSON
Input []byte
Output *types.JSON
Output []byte
}
func StepRunStatusPtr(status db.StepRunStatus) *db.StepRunStatus {

View File

@@ -7,6 +7,11 @@ import (
"fmt"
"time"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"google.golang.org/protobuf/types/known/timestamppb"
"google.golang.org/protobuf/types/known/wrapperspb"
"github.com/hatchet-dev/hatchet/internal/datautils"
"github.com/hatchet-dev/hatchet/internal/repository"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
@@ -14,10 +19,6 @@ import (
"github.com/hatchet-dev/hatchet/internal/services/shared/tasktypes"
"github.com/hatchet-dev/hatchet/internal/taskqueue"
"github.com/hatchet-dev/hatchet/pkg/client/types"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"google.golang.org/protobuf/types/known/timestamppb"
"google.golang.org/protobuf/types/known/wrapperspb"
)
func (a *AdminServiceImpl) GetWorkflowByName(ctx context.Context, req *contracts.GetWorkflowByNameRequest) (*contracts.Workflow, error) {

View File

@@ -6,6 +6,11 @@ import (
"fmt"
"net"
"github.com/rs/zerolog"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
"github.com/hatchet-dev/hatchet/internal/logger"
"github.com/hatchet-dev/hatchet/internal/services/admin"
admincontracts "github.com/hatchet-dev/hatchet/internal/services/admin/contracts"
@@ -13,9 +18,6 @@ import (
dispatchercontracts "github.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts"
"github.com/hatchet-dev/hatchet/internal/services/ingestor"
eventcontracts "github.com/hatchet-dev/hatchet/internal/services/ingestor/contracts"
"github.com/rs/zerolog"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
)
type Server struct {
@@ -31,6 +33,7 @@ type Server struct {
dispatcher dispatcher.Dispatcher
admin admin.AdminService
tls *tls.Config
insecure bool
}
type ServerOpt func(*ServerOpts)
@@ -43,6 +46,7 @@ type ServerOpts struct {
dispatcher dispatcher.Dispatcher
admin admin.AdminService
tls *tls.Config
insecure bool
}
func defaultServerOpts() *ServerOpts {
@@ -52,6 +56,7 @@ func defaultServerOpts() *ServerOpts {
l: &logger,
port: 7070,
bindAddress: "127.0.0.1",
insecure: false,
}
}
@@ -85,6 +90,12 @@ func WithTLSConfig(tls *tls.Config) ServerOpt {
}
}
func WithInsecure() ServerOpt {
return func(opts *ServerOpts) {
opts.insecure = true
}
}
func WithDispatcher(d dispatcher.Dispatcher) ServerOpt {
return func(opts *ServerOpts) {
opts.dispatcher = d
@@ -119,6 +130,7 @@ func NewServer(fs ...ServerOpt) (*Server, error) {
dispatcher: opts.dispatcher,
admin: opts.admin,
tls: opts.tls,
insecure: opts.insecure,
}, nil
}
@@ -135,7 +147,15 @@ func (s *Server) startGRPC(ctx context.Context) error {
return fmt.Errorf("failed to listen: %w", err)
}
grpcServer := grpc.NewServer(grpc.Creds(credentials.NewTLS(s.tls)))
serverOpts := []grpc.ServerOption{}
if s.insecure {
serverOpts = append(serverOpts, grpc.Creds(insecure.NewCredentials()))
} else {
serverOpts = append(serverOpts, grpc.Creds(credentials.NewTLS(s.tls)))
}
grpcServer := grpc.NewServer(serverOpts...)
if s.ingestor != nil {
eventcontracts.RegisterEventsServiceServer(grpcServer, s.ingestor)

View File

@@ -9,7 +9,6 @@ import (
"time"
"github.com/rs/zerolog"
"github.com/steebchen/prisma-client-go/runtime/types"
"golang.org/x/sync/errgroup"
"github.com/hatchet-dev/hatchet/internal/datautils"
@@ -509,13 +508,13 @@ func (ec *JobsControllerImpl) queueStepRun(ctx context.Context, tenantId, stepId
}
}
newInput, err := datautils.ToJSONType(inputData)
inputDataBytes, err := json.Marshal(inputData)
if err != nil {
return fmt.Errorf("could not convert input data to json: %w", err)
}
updateStepOpts.Input = newInput
updateStepOpts.Input = inputDataBytes
}
// begin transaction and make sure step run is in a pending status
@@ -698,20 +697,22 @@ func (ec *JobsControllerImpl) handleStepRunFinished(ctx context.Context, task *t
return fmt.Errorf("could not parse started at: %w", err)
}
var stepOutput string
var stepOutput []byte
stepOutput, err = strconv.Unquote(payload.StepOutputData)
if payload.StepOutputData != "" {
stepOutputStr, err := strconv.Unquote(payload.StepOutputData)
if err != nil {
stepOutput = payload.StepOutputData
if err != nil {
stepOutputStr = payload.StepOutputData
}
stepOutput = []byte(stepOutputStr)
}
outputJSON := types.JSON(json.RawMessage(stepOutput))
stepRun, err := ec.repo.StepRun().UpdateStepRun(metadata.TenantId, payload.StepRunId, &repository.UpdateStepRunOpts{
FinishedAt: &finishedAt,
Status: repository.StepRunStatusPtr(db.StepRunStatusSucceeded),
Output: &outputJSON,
Output: stepOutput,
})
if err != nil {
@@ -728,26 +729,29 @@ func (ec *JobsControllerImpl) handleStepRunFinished(ctx context.Context, task *t
servertel.WithJobRunModel(span, jobRun)
stepReadableId, ok := stepRun.Step().ReadableID()
// stepReadableId, ok := stepRun.Step().ReadableID()
// only update the job lookup data if the step has a readable id to key
if ok && stepReadableId != "" {
unquoted, err := strconv.Unquote(payload.StepOutputData)
// if ok && stepReadableId != "" {
// if payload.StepOutputData != "" {
// fmt.Println("UPDATING WITH PAYLOAD", payload.StepOutputData)
// unquoted, err := strconv.Unquote(payload.StepOutputData)
if err != nil {
unquoted = payload.StepOutputData
}
// if err != nil {
// unquoted = payload.StepOutputData
// }
// update the job run lookup data
err = ec.repo.JobRun().UpdateJobRunLookupData(metadata.TenantId, stepRun.JobRunID, &repository.UpdateJobRunLookupDataOpts{
FieldPath: []string{"steps", stepReadableId},
Data: []byte(unquoted),
})
// // update the job run lookup data
// err = ec.repo.JobRun().UpdateJobRunLookupData(metadata.TenantId, stepRun.JobRunID, &repository.UpdateJobRunLookupDataOpts{
// FieldPath: []string{"steps", stepReadableId},
// Data: []byte(unquoted),
// })
if err != nil {
return fmt.Errorf("could not update job run lookup data: %w", err)
}
}
// if err != nil {
// return fmt.Errorf("could not update job run lookup data: %w", err)
// }
// }
// }
// queue the next step runs
nextStepRuns, err := ec.repo.StepRun().ListStartableStepRuns(metadata.TenantId, jobRun.ID, stepRun.ID)

160
python-client/.gitignore vendored Normal file
View File

@@ -0,0 +1,160 @@
# Byte-compiled / optimized / DLL files
__pycache__/
*.py[cod]
*$py.class
# C extensions
*.so
# Distribution / packaging
.Python
build/
develop-eggs/
dist/
downloads/
eggs/
.eggs/
lib/
lib64/
parts/
sdist/
var/
wheels/
share/python-wheels/
*.egg-info/
.installed.cfg
*.egg
MANIFEST
# PyInstaller
# Usually these files are written by a python script from a template
# before PyInstaller builds the exe, so as to inject date/other infos into it.
*.manifest
*.spec
# Installer logs
pip-log.txt
pip-delete-this-directory.txt
# Unit test / coverage reports
htmlcov/
.tox/
.nox/
.coverage
.coverage.*
.cache
nosetests.xml
coverage.xml
*.cover
*.py,cover
.hypothesis/
.pytest_cache/
cover/
# Translations
*.mo
*.pot
# Django stuff:
*.log
local_settings.py
db.sqlite3
db.sqlite3-journal
# Flask stuff:
instance/
.webassets-cache
# Scrapy stuff:
.scrapy
# Sphinx documentation
docs/_build/
# PyBuilder
.pybuilder/
target/
# Jupyter Notebook
.ipynb_checkpoints
# IPython
profile_default/
ipython_config.py
# pyenv
# For a library or package, you might want to ignore these files since the code is
# intended to run in multiple environments; otherwise, check them in:
# .python-version
# pipenv
# According to pypa/pipenv#598, it is recommended to include Pipfile.lock in version control.
# However, in case of collaboration, if having platform-specific dependencies or dependencies
# having no cross-platform support, pipenv may install dependencies that don't work, or not
# install all needed dependencies.
#Pipfile.lock
# poetry
# Similar to Pipfile.lock, it is generally recommended to include poetry.lock in version control.
# This is especially recommended for binary packages to ensure reproducibility, and is more
# commonly ignored for libraries.
# https://python-poetry.org/docs/basic-usage/#commit-your-poetrylock-file-to-version-control
#poetry.lock
# pdm
# Similar to Pipfile.lock, it is generally recommended to include pdm.lock in version control.
#pdm.lock
# pdm stores project-wide configurations in .pdm.toml, but it is recommended to not include it
# in version control.
# https://pdm.fming.dev/#use-with-ide
.pdm.toml
# PEP 582; used by e.g. github.com/David-OConnor/pyflow and github.com/pdm-project/pdm
__pypackages__/
# Celery stuff
celerybeat-schedule
celerybeat.pid
# SageMath parsed files
*.sage.py
# Environments
.env
.venv
env/
venv/
ENV/
env.bak/
venv.bak/
# Spyder project settings
.spyderproject
.spyproject
# Rope project settings
.ropeproject
# mkdocs documentation
/site
# mypy
.mypy_cache/
.dmypy.json
dmypy.json
# Pyre type checker
.pyre/
# pytype static type analyzer
.pytype/
# Cython debug symbols
cython_debug/
# PyCharm
# JetBrains specific template is maintained in a separate JetBrains.gitignore that can
# be found at https://github.com/github/gitignore/blob/main/Global/JetBrains.gitignore
# and can be added to the global gitignore or merged into this file. For a more nuclear
# option (not recommended) you can uncomment the following to ignore the entire idea folder.
#.idea/

3
python-client/README.md Normal file
View File

@@ -0,0 +1,3 @@
# Hatchet Python SDK
This is the [Hatchet](https://hatchet.run) Python SDK. For usage, see the [docs](https://docs.hatchet.run/python-sdk).

View File

@@ -0,0 +1,10 @@
from hatchet import new_client
client = new_client()
client.event.push(
"user:create",
{
"test": "test"
}
)

View File

@@ -0,0 +1,42 @@
from hatchet import Hatchet, Context
hatchet = Hatchet()
@hatchet.workflow(on_events=["user:create"])
class MyWorkflow:
def __init__(self):
self.my_value = "test"
@hatchet.step()
def step1(self, context : Context):
print("executed step1", context.workflow_input())
return {
"step1": "step1",
}
@hatchet.step()
def step2(self, context : Context):
print("executed step2", context.workflow_input())
return {
"step2": "step2",
}
@hatchet.step(parents=["step1", "step2"])
def step3(self, context : Context):
print("executed step3", context.workflow_input(), context.step_output("step1"), context.step_output("step2"))
return {
"step3": "step3",
}
@hatchet.step(parents=["step1", "step3"])
def step4(self, context : Context):
print("executed step4", context.workflow_input(), context.step_output("step1"), context.step_output("step3"))
return {
"step4": "step4",
}
workflow = MyWorkflow()
worker = hatchet.worker('test-worker')
worker.register_workflow(workflow)
worker.start()

View File

@@ -0,0 +1,10 @@
from hatchet import new_client
client = new_client()
client.event.push(
"user:create",
{
"test": "test"
}
)

View File

@@ -0,0 +1,24 @@
from hatchet import Hatchet
hatchet = Hatchet()
@hatchet.workflow(on_events=["user:create"])
class MyWorkflow:
def __init__(self):
self.my_value = "test"
@hatchet.step()
def step1(self, context):
print("executed step1")
pass
@hatchet.step(parents=["step1"])
def step2(self, context):
print("executed step2")
pass
workflow = MyWorkflow()
worker = hatchet.worker('test-worker')
worker.register_workflow(workflow)
worker.start()

View File

@@ -0,0 +1,7 @@
#!/bin/bash
#
# Builds python auto-generated protobuf files
poetry run python -m grpc_tools.protoc --proto_path=../api-contracts/dispatcher --python_out=./hatchet --pyi_out=./hatchet --grpc_python_out=./hatchet dispatcher.proto
poetry run python -m grpc_tools.protoc --proto_path=../api-contracts/events --python_out=./hatchet --pyi_out=./hatchet --grpc_python_out=./hatchet events.proto
poetry run python -m grpc_tools.protoc --proto_path=../api-contracts/workflows --python_out=./hatchet --pyi_out=./hatchet --grpc_python_out=./hatchet workflows.proto

View File

@@ -0,0 +1,4 @@
from .hatchet import Hatchet
from .worker import Worker
from .client import new_client
from .context import Context

View File

@@ -0,0 +1,72 @@
# relative imports
from .clients.admin import AdminClientImpl, new_admin
from .clients.events import EventClientImpl, new_event
from .clients.dispatcher import DispatcherClientImpl, new_dispatcher
from .loader import ConfigLoader, ClientConfig
import grpc
class Client:
def admin(self):
raise NotImplementedError
def dispatcher(self):
raise NotImplementedError
def event(self):
raise NotImplementedError
class ClientImpl(Client):
def __init__(self, event_client, admin_client : AdminClientImpl, dispatcher_client):
# self.conn = conn
# self.tenant_id = tenant_id
# self.logger = logger
# self.validator = validator
self.admin = admin_client
self.dispatcher = dispatcher_client
self.event = event_client
def admin(self) -> AdminClientImpl:
return self.admin
def dispatcher(self) -> DispatcherClientImpl:
return self.dispatcher
def event(self) -> EventClientImpl:
return self.event
def with_host_port(host: str, port: int):
def with_host_port_impl(config: ClientConfig):
config.host = host
config.port = port
return with_host_port_impl
def new_client(*opts_functions):
config : ClientConfig = ConfigLoader(".").load_client_config()
for opt_function in opts_functions:
opt_function(config)
if config.tls_config is None:
raise ValueError("TLS config is required")
if config.host_port is None:
raise ValueError("Host and port are required")
root = open(config.tls_config.ca_file, "rb").read()
private_key = open(config.tls_config.key_file, "rb").read()
certificate_chain = open(config.tls_config.cert_file, "rb").read()
conn = grpc.secure_channel(
target=config.host_port,
credentials=grpc.ssl_channel_credentials(root_certificates=root, private_key=private_key, certificate_chain=certificate_chain),
options=[('grpc.ssl_target_name_override', config.tls_config.server_name)],
)
# Instantiate client implementations
event_client = new_event(conn, config)
admin_client = new_admin(conn, config)
dispatcher_client = new_dispatcher(conn, config)
return ClientImpl(event_client, admin_client, dispatcher_client)

View File

@@ -0,0 +1,81 @@
from typing import List
import grpc
from google.protobuf import timestamp_pb2
from ..workflows_pb2_grpc import WorkflowServiceStub
from ..workflows_pb2 import CreateWorkflowVersionOpts, ScheduleWorkflowRequest, PutWorkflowRequest, GetWorkflowByNameRequest, Workflow
from ..loader import ClientConfig
from ..semver import bump_minor_version
def new_admin(conn, config: ClientConfig):
return AdminClientImpl(
client=WorkflowServiceStub(conn),
tenant_id=config.tenant_id,
# logger=shared_opts['logger'],
# validator=shared_opts['validator'],
)
class AdminClientImpl:
def __init__(self, client : WorkflowServiceStub, tenant_id):
self.client = client
self.tenant_id = tenant_id
# self.logger = logger
# self.validator = validator
def put_workflow(self, workflow: CreateWorkflowVersionOpts, auto_version: bool = False):
if workflow.version == "" and not auto_version:
raise ValueError("PutWorkflow error: workflow version is required, or use with_auto_version")
existing_workflow : Workflow = None
# Get existing workflow by name
try:
existing_workflow : Workflow = self.client.GetWorkflowByName(
GetWorkflowByNameRequest(
tenant_id=self.tenant_id,
name=workflow.name,
)
)
except grpc.RpcError as e:
if e.code() == grpc.StatusCode.NOT_FOUND:
should_put = True
else:
raise ValueError(f"Could not get workflow: {e}")
# Determine if we should put the workflow
should_put = False
if auto_version:
if workflow.version == "":
workflow.version = "v0.1.0"
should_put = True
elif existing_workflow and existing_workflow.versions:
if auto_version:
workflow.version = bump_minor_version(existing_workflow.versions[0].version)
should_put = True
elif existing_workflow.versions[0].version != workflow.version:
should_put = True
else:
should_put = False
else:
should_put = True
# Put the workflow if conditions are met
if should_put:
try:
self.client.PutWorkflow(
PutWorkflowRequest(
tenant_id=self.tenant_id,
opts=workflow,
)
)
except grpc.RpcError as e:
raise ValueError(f"Could not create/update workflow: {e}")
def schedule_workflow(self, workflow_id : str, schedules : List[timestamp_pb2.Timestamp]):
try:
self.client.ScheduleWorkflow(ScheduleWorkflowRequest(
tenant_id=self.tenant_id,
workflow_id=workflow_id,
schedules=schedules,
))
except grpc.RpcError as e:
raise ValueError(f"gRPC error: {e}")

View File

@@ -0,0 +1,183 @@
# relative imports
from ..dispatcher_pb2 import ActionEvent, ActionEventResponse, ActionType, AssignedAction, WorkerListenRequest, WorkerRegisterRequest, WorkerUnsubscribeRequest, WorkerRegisterResponse
from ..dispatcher_pb2_grpc import DispatcherStub
import time
from ..loader import ClientConfig
import json
import grpc
from typing import Callable, List, Union
def new_dispatcher(conn, config: ClientConfig):
return DispatcherClientImpl(
client=DispatcherStub(conn),
tenant_id=config.tenant_id,
# logger=shared_opts['logger'],
# validator=shared_opts['validator'],
)
class DispatcherClient:
def get_action_listener(self, ctx, req):
raise NotImplementedError
def send_action_event(self, ctx, in_):
raise NotImplementedError
DEFAULT_ACTION_LISTENER_RETRY_INTERVAL = 5 # seconds
DEFAULT_ACTION_LISTENER_RETRY_COUNT = 5
class GetActionListenerRequest:
def __init__(self, worker_name: str, services: List[str], actions: List[str]):
self.worker_name = worker_name
self.services = services
self.actions = actions
class Action:
def __init__(self, worker_id: str, tenant_id: str, job_id: str, job_name: str, job_run_id: str, step_id: str, step_run_id: str, action_id: str, action_payload: str, action_type: ActionType):
self.worker_id = worker_id
self.tenant_id = tenant_id
self.job_id = job_id
self.job_name = job_name
self.job_run_id = job_run_id
self.step_id = step_id
self.step_run_id = step_run_id
self.action_id = action_id
self.action_payload = action_payload
self.action_type = action_type
class WorkerActionListener:
def actions(self, ctx, err_ch):
raise NotImplementedError
def unregister(self):
raise NotImplementedError
# enum for START_STEP_RUN and CANCEL_STEP_RUN
START_STEP_RUN = 0
CANCEL_STEP_RUN = 1
class ActionListenerImpl(WorkerActionListener):
def __init__(self, client : DispatcherStub, tenant_id, listen_client, worker_id):
self.client = client
self.tenant_id = tenant_id
self.listen_client = listen_client
self.worker_id = worker_id
# self.logger = logger
# self.validator = validator
def actions(self):
while True:
try:
for assigned_action in self.listen_client:
assigned_action : AssignedAction
# Process the received action
action_type = self.map_action_type(assigned_action.actionType)
if assigned_action.actionPayload is None or assigned_action.actionPayload == "":
action_payload = None
else:
action_payload = self.parse_action_payload(assigned_action.actionPayload)
action = Action(
tenant_id=assigned_action.tenantId,
worker_id=self.worker_id,
job_id=assigned_action.jobId,
job_name=assigned_action.jobName,
job_run_id=assigned_action.jobRunId,
step_id=assigned_action.stepId,
step_run_id=assigned_action.stepRunId,
action_id=assigned_action.actionId,
action_payload=action_payload,
action_type=action_type,
)
yield action
except grpc.RpcError as e:
# Handle different types of errors
if e.code() == grpc.StatusCode.CANCELLED:
# Context cancelled, unsubscribe and close
# self.logger.debug("Context cancelled, closing listener")
break
elif e.code() == grpc.StatusCode.UNAVAILABLE:
# Retry logic
self.retry_subscribe()
else:
# Unknown error, report and break
# self.logger.error(f"Failed to receive message: {e}")
# err_ch(e)
break
def parse_action_payload(self, payload : str):
try:
payload_data = json.loads(payload)
except json.JSONDecodeError as e:
raise ValueError(f"Error decoding payload: {e}")
return payload_data
def map_action_type(self, action_type):
if action_type == ActionType.START_STEP_RUN:
return START_STEP_RUN
elif action_type == ActionType.CANCEL_STEP_RUN:
return CANCEL_STEP_RUN
else:
# self.logger.error(f"Unknown action type: {action_type}")
return None
def retry_subscribe(self):
retries = 0
while retries < DEFAULT_ACTION_LISTENER_RETRY_COUNT:
try:
time.sleep(DEFAULT_ACTION_LISTENER_RETRY_INTERVAL)
self.listen_client = self.client.Listen(WorkerListenRequest(
tenantId=self.tenant_id,
workerId=self.worker_id
))
return
except grpc.RpcError as e:
retries += 1
# self.logger.error(f"Failed to retry subscription: {e}")
raise Exception(f"Could not subscribe to the worker after {DEFAULT_ACTION_LISTENER_RETRY_COUNT} retries")
def unregister(self):
try:
self.client.Unsubscribe(
WorkerUnsubscribeRequest(
tenant_id=self.tenant_id,
worker_id=self.worker_id
)
)
except grpc.RpcError as e:
raise Exception(f"Failed to unsubscribe: {e}")
class DispatcherClientImpl(DispatcherClient):
def __init__(self, client : DispatcherStub, tenant_id):
self.client = client
self.tenant_id = tenant_id
# self.logger = logger
# self.validator = validator
def get_action_listener(self, req: GetActionListenerRequest) -> ActionListenerImpl:
# Register the worker
response : WorkerRegisterResponse = self.client.Register(WorkerRegisterRequest(
tenantId=self.tenant_id,
workerName=req.worker_name,
actions=req.actions,
services=req.services
))
# Subscribe to the worker
listener = self.client.Listen(WorkerListenRequest(
tenantId=self.tenant_id,
workerId=response.workerId,
))
return ActionListenerImpl(self.client, self.tenant_id, listener, response.workerId)
def send_action_event(self, in_: ActionEvent):
response : ActionEventResponse = self.client.SendActionEvent(in_)
return response

View File

@@ -0,0 +1,41 @@
from ..events_pb2_grpc import EventsServiceStub
from ..events_pb2 import PushEventRequest
import datetime
from ..loader import ClientConfig
import json
import grpc
from google.protobuf import timestamp_pb2
def new_event(conn, config: ClientConfig):
return EventClientImpl(
client=EventsServiceStub(conn),
tenant_id=config.tenant_id,
# logger=shared_opts['logger'],
# validator=shared_opts['validator'],
)
class EventClientImpl:
def __init__(self, client, tenant_id):
self.client = client
self.tenant_id = tenant_id
# self.logger = logger
# self.validator = validator
def push(self, event_key, payload):
try:
payload_bytes = json.dumps(payload).encode('utf-8')
except json.UnicodeEncodeError as e:
raise ValueError(f"Error encoding payload: {e}")
request = PushEventRequest(
tenantId=self.tenant_id,
key=event_key,
payload=payload_bytes,
eventTimestamp=timestamp_pb2.Timestamp().FromDatetime(datetime.datetime.now()),
)
try:
self.client.Push(request)
except grpc.RpcError as e:
raise ValueError(f"gRPC error: {e}")

View File

@@ -0,0 +1,17 @@
import json
class Context:
def __init__(self, payload: str):
self.data = json.loads(payload)
def step_output(self, step: str):
try:
return self.data['parents'][step]
except KeyError:
raise ValueError(f"Step output for '{step}' not found")
def triggered_by_event(self) -> bool:
return self.data.get('triggered_by', '') == 'event'
def workflow_input(self):
return self.data.get('input', {})

View File

@@ -0,0 +1,48 @@
# -*- coding: utf-8 -*-
# Generated by the protocol buffer compiler. DO NOT EDIT!
# source: dispatcher.proto
# Protobuf Python Version: 4.25.0
"""Generated protocol buffer code."""
from google.protobuf import descriptor as _descriptor
from google.protobuf import descriptor_pool as _descriptor_pool
from google.protobuf import symbol_database as _symbol_database
from google.protobuf.internal import builder as _builder
# @@protoc_insertion_point(imports)
_sym_db = _symbol_database.Default()
from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x10\x64ispatcher.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"`\n\x15WorkerRegisterRequest\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x12\n\nworkerName\x18\x02 \x01(\t\x12\x0f\n\x07\x61\x63tions\x18\x03 \x03(\t\x12\x10\n\x08services\x18\x04 \x03(\t\"P\n\x16WorkerRegisterResponse\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t\x12\x12\n\nworkerName\x18\x03 \x01(\t\"\xc1\x01\n\x0e\x41ssignedAction\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\r\n\x05jobId\x18\x02 \x01(\t\x12\x0f\n\x07jobName\x18\x03 \x01(\t\x12\x10\n\x08jobRunId\x18\x04 \x01(\t\x12\x0e\n\x06stepId\x18\x05 \x01(\t\x12\x11\n\tstepRunId\x18\x06 \x01(\t\x12\x10\n\x08\x61\x63tionId\x18\x07 \x01(\t\x12\x1f\n\nactionType\x18\x08 \x01(\x0e\x32\x0b.ActionType\x12\x15\n\ractionPayload\x18\t \x01(\t\"9\n\x13WorkerListenRequest\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t\">\n\x18WorkerUnsubscribeRequest\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t\"?\n\x19WorkerUnsubscribeResponse\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t\"\xf6\x01\n\x0b\x41\x63tionEvent\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t\x12\r\n\x05jobId\x18\x03 \x01(\t\x12\x10\n\x08jobRunId\x18\x04 \x01(\t\x12\x0e\n\x06stepId\x18\x05 \x01(\t\x12\x11\n\tstepRunId\x18\x06 \x01(\t\x12\x10\n\x08\x61\x63tionId\x18\x07 \x01(\t\x12\x32\n\x0e\x65ventTimestamp\x18\x08 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12#\n\teventType\x18\t \x01(\x0e\x32\x10.ActionEventType\x12\x14\n\x0c\x65ventPayload\x18\n \x01(\t\"9\n\x13\x41\x63tionEventResponse\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t*5\n\nActionType\x12\x12\n\x0eSTART_STEP_RUN\x10\x00\x12\x13\n\x0f\x43\x41NCEL_STEP_RUN\x10\x01*\x86\x01\n\x0f\x41\x63tionEventType\x12\x1b\n\x17STEP_EVENT_TYPE_UNKNOWN\x10\x00\x12\x1b\n\x17STEP_EVENT_TYPE_STARTED\x10\x01\x12\x1d\n\x19STEP_EVENT_TYPE_COMPLETED\x10\x02\x12\x1a\n\x16STEP_EVENT_TYPE_FAILED\x10\x03\x32\x81\x02\n\nDispatcher\x12=\n\x08Register\x12\x16.WorkerRegisterRequest\x1a\x17.WorkerRegisterResponse\"\x00\x12\x33\n\x06Listen\x12\x14.WorkerListenRequest\x1a\x0f.AssignedAction\"\x00\x30\x01\x12\x37\n\x0fSendActionEvent\x12\x0c.ActionEvent\x1a\x14.ActionEventResponse\"\x00\x12\x46\n\x0bUnsubscribe\x12\x19.WorkerUnsubscribeRequest\x1a\x1a.WorkerUnsubscribeResponse\"\x00\x42GZEgithub.com/hatchet-dev/hatchet/internal/services/dispatcher/contractsb\x06proto3')
_globals = globals()
_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals)
_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'dispatcher_pb2', _globals)
if _descriptor._USE_C_DESCRIPTORS == False:
_globals['DESCRIPTOR']._options = None
_globals['DESCRIPTOR']._serialized_options = b'ZEgithub.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts'
_globals['_ACTIONTYPE']._serialized_start=925
_globals['_ACTIONTYPE']._serialized_end=978
_globals['_ACTIONEVENTTYPE']._serialized_start=981
_globals['_ACTIONEVENTTYPE']._serialized_end=1115
_globals['_WORKERREGISTERREQUEST']._serialized_start=53
_globals['_WORKERREGISTERREQUEST']._serialized_end=149
_globals['_WORKERREGISTERRESPONSE']._serialized_start=151
_globals['_WORKERREGISTERRESPONSE']._serialized_end=231
_globals['_ASSIGNEDACTION']._serialized_start=234
_globals['_ASSIGNEDACTION']._serialized_end=427
_globals['_WORKERLISTENREQUEST']._serialized_start=429
_globals['_WORKERLISTENREQUEST']._serialized_end=486
_globals['_WORKERUNSUBSCRIBEREQUEST']._serialized_start=488
_globals['_WORKERUNSUBSCRIBEREQUEST']._serialized_end=550
_globals['_WORKERUNSUBSCRIBERESPONSE']._serialized_start=552
_globals['_WORKERUNSUBSCRIBERESPONSE']._serialized_end=615
_globals['_ACTIONEVENT']._serialized_start=618
_globals['_ACTIONEVENT']._serialized_end=864
_globals['_ACTIONEVENTRESPONSE']._serialized_start=866
_globals['_ACTIONEVENTRESPONSE']._serialized_end=923
_globals['_DISPATCHER']._serialized_start=1118
_globals['_DISPATCHER']._serialized_end=1375
# @@protoc_insertion_point(module_scope)

View File

@@ -0,0 +1,126 @@
from google.protobuf import timestamp_pb2 as _timestamp_pb2
from google.protobuf.internal import containers as _containers
from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper
from google.protobuf import descriptor as _descriptor
from google.protobuf import message as _message
from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union
DESCRIPTOR: _descriptor.FileDescriptor
class ActionType(int, metaclass=_enum_type_wrapper.EnumTypeWrapper):
__slots__ = ()
START_STEP_RUN: _ClassVar[ActionType]
CANCEL_STEP_RUN: _ClassVar[ActionType]
class ActionEventType(int, metaclass=_enum_type_wrapper.EnumTypeWrapper):
__slots__ = ()
STEP_EVENT_TYPE_UNKNOWN: _ClassVar[ActionEventType]
STEP_EVENT_TYPE_STARTED: _ClassVar[ActionEventType]
STEP_EVENT_TYPE_COMPLETED: _ClassVar[ActionEventType]
STEP_EVENT_TYPE_FAILED: _ClassVar[ActionEventType]
START_STEP_RUN: ActionType
CANCEL_STEP_RUN: ActionType
STEP_EVENT_TYPE_UNKNOWN: ActionEventType
STEP_EVENT_TYPE_STARTED: ActionEventType
STEP_EVENT_TYPE_COMPLETED: ActionEventType
STEP_EVENT_TYPE_FAILED: ActionEventType
class WorkerRegisterRequest(_message.Message):
__slots__ = ("tenantId", "workerName", "actions", "services")
TENANTID_FIELD_NUMBER: _ClassVar[int]
WORKERNAME_FIELD_NUMBER: _ClassVar[int]
ACTIONS_FIELD_NUMBER: _ClassVar[int]
SERVICES_FIELD_NUMBER: _ClassVar[int]
tenantId: str
workerName: str
actions: _containers.RepeatedScalarFieldContainer[str]
services: _containers.RepeatedScalarFieldContainer[str]
def __init__(self, tenantId: _Optional[str] = ..., workerName: _Optional[str] = ..., actions: _Optional[_Iterable[str]] = ..., services: _Optional[_Iterable[str]] = ...) -> None: ...
class WorkerRegisterResponse(_message.Message):
__slots__ = ("tenantId", "workerId", "workerName")
TENANTID_FIELD_NUMBER: _ClassVar[int]
WORKERID_FIELD_NUMBER: _ClassVar[int]
WORKERNAME_FIELD_NUMBER: _ClassVar[int]
tenantId: str
workerId: str
workerName: str
def __init__(self, tenantId: _Optional[str] = ..., workerId: _Optional[str] = ..., workerName: _Optional[str] = ...) -> None: ...
class AssignedAction(_message.Message):
__slots__ = ("tenantId", "jobId", "jobName", "jobRunId", "stepId", "stepRunId", "actionId", "actionType", "actionPayload")
TENANTID_FIELD_NUMBER: _ClassVar[int]
JOBID_FIELD_NUMBER: _ClassVar[int]
JOBNAME_FIELD_NUMBER: _ClassVar[int]
JOBRUNID_FIELD_NUMBER: _ClassVar[int]
STEPID_FIELD_NUMBER: _ClassVar[int]
STEPRUNID_FIELD_NUMBER: _ClassVar[int]
ACTIONID_FIELD_NUMBER: _ClassVar[int]
ACTIONTYPE_FIELD_NUMBER: _ClassVar[int]
ACTIONPAYLOAD_FIELD_NUMBER: _ClassVar[int]
tenantId: str
jobId: str
jobName: str
jobRunId: str
stepId: str
stepRunId: str
actionId: str
actionType: ActionType
actionPayload: str
def __init__(self, tenantId: _Optional[str] = ..., jobId: _Optional[str] = ..., jobName: _Optional[str] = ..., jobRunId: _Optional[str] = ..., stepId: _Optional[str] = ..., stepRunId: _Optional[str] = ..., actionId: _Optional[str] = ..., actionType: _Optional[_Union[ActionType, str]] = ..., actionPayload: _Optional[str] = ...) -> None: ...
class WorkerListenRequest(_message.Message):
__slots__ = ("tenantId", "workerId")
TENANTID_FIELD_NUMBER: _ClassVar[int]
WORKERID_FIELD_NUMBER: _ClassVar[int]
tenantId: str
workerId: str
def __init__(self, tenantId: _Optional[str] = ..., workerId: _Optional[str] = ...) -> None: ...
class WorkerUnsubscribeRequest(_message.Message):
__slots__ = ("tenantId", "workerId")
TENANTID_FIELD_NUMBER: _ClassVar[int]
WORKERID_FIELD_NUMBER: _ClassVar[int]
tenantId: str
workerId: str
def __init__(self, tenantId: _Optional[str] = ..., workerId: _Optional[str] = ...) -> None: ...
class WorkerUnsubscribeResponse(_message.Message):
__slots__ = ("tenantId", "workerId")
TENANTID_FIELD_NUMBER: _ClassVar[int]
WORKERID_FIELD_NUMBER: _ClassVar[int]
tenantId: str
workerId: str
def __init__(self, tenantId: _Optional[str] = ..., workerId: _Optional[str] = ...) -> None: ...
class ActionEvent(_message.Message):
__slots__ = ("tenantId", "workerId", "jobId", "jobRunId", "stepId", "stepRunId", "actionId", "eventTimestamp", "eventType", "eventPayload")
TENANTID_FIELD_NUMBER: _ClassVar[int]
WORKERID_FIELD_NUMBER: _ClassVar[int]
JOBID_FIELD_NUMBER: _ClassVar[int]
JOBRUNID_FIELD_NUMBER: _ClassVar[int]
STEPID_FIELD_NUMBER: _ClassVar[int]
STEPRUNID_FIELD_NUMBER: _ClassVar[int]
ACTIONID_FIELD_NUMBER: _ClassVar[int]
EVENTTIMESTAMP_FIELD_NUMBER: _ClassVar[int]
EVENTTYPE_FIELD_NUMBER: _ClassVar[int]
EVENTPAYLOAD_FIELD_NUMBER: _ClassVar[int]
tenantId: str
workerId: str
jobId: str
jobRunId: str
stepId: str
stepRunId: str
actionId: str
eventTimestamp: _timestamp_pb2.Timestamp
eventType: ActionEventType
eventPayload: str
def __init__(self, tenantId: _Optional[str] = ..., workerId: _Optional[str] = ..., jobId: _Optional[str] = ..., jobRunId: _Optional[str] = ..., stepId: _Optional[str] = ..., stepRunId: _Optional[str] = ..., actionId: _Optional[str] = ..., eventTimestamp: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., eventType: _Optional[_Union[ActionEventType, str]] = ..., eventPayload: _Optional[str] = ...) -> None: ...
class ActionEventResponse(_message.Message):
__slots__ = ("tenantId", "workerId")
TENANTID_FIELD_NUMBER: _ClassVar[int]
WORKERID_FIELD_NUMBER: _ClassVar[int]
tenantId: str
workerId: str
def __init__(self, tenantId: _Optional[str] = ..., workerId: _Optional[str] = ...) -> None: ...

View File

@@ -0,0 +1,165 @@
# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT!
"""Client and server classes corresponding to protobuf-defined services."""
import grpc
# import dispatcher_pb2 as dispatcher__pb2
from . import dispatcher_pb2 as dispatcher__pb2
class DispatcherStub(object):
"""Missing associated documentation comment in .proto file."""
def __init__(self, channel):
"""Constructor.
Args:
channel: A grpc.Channel.
"""
self.Register = channel.unary_unary(
'/Dispatcher/Register',
request_serializer=dispatcher__pb2.WorkerRegisterRequest.SerializeToString,
response_deserializer=dispatcher__pb2.WorkerRegisterResponse.FromString,
)
self.Listen = channel.unary_stream(
'/Dispatcher/Listen',
request_serializer=dispatcher__pb2.WorkerListenRequest.SerializeToString,
response_deserializer=dispatcher__pb2.AssignedAction.FromString,
)
self.SendActionEvent = channel.unary_unary(
'/Dispatcher/SendActionEvent',
request_serializer=dispatcher__pb2.ActionEvent.SerializeToString,
response_deserializer=dispatcher__pb2.ActionEventResponse.FromString,
)
self.Unsubscribe = channel.unary_unary(
'/Dispatcher/Unsubscribe',
request_serializer=dispatcher__pb2.WorkerUnsubscribeRequest.SerializeToString,
response_deserializer=dispatcher__pb2.WorkerUnsubscribeResponse.FromString,
)
class DispatcherServicer(object):
"""Missing associated documentation comment in .proto file."""
def Register(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def Listen(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def SendActionEvent(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def Unsubscribe(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def add_DispatcherServicer_to_server(servicer, server):
rpc_method_handlers = {
'Register': grpc.unary_unary_rpc_method_handler(
servicer.Register,
request_deserializer=dispatcher__pb2.WorkerRegisterRequest.FromString,
response_serializer=dispatcher__pb2.WorkerRegisterResponse.SerializeToString,
),
'Listen': grpc.unary_stream_rpc_method_handler(
servicer.Listen,
request_deserializer=dispatcher__pb2.WorkerListenRequest.FromString,
response_serializer=dispatcher__pb2.AssignedAction.SerializeToString,
),
'SendActionEvent': grpc.unary_unary_rpc_method_handler(
servicer.SendActionEvent,
request_deserializer=dispatcher__pb2.ActionEvent.FromString,
response_serializer=dispatcher__pb2.ActionEventResponse.SerializeToString,
),
'Unsubscribe': grpc.unary_unary_rpc_method_handler(
servicer.Unsubscribe,
request_deserializer=dispatcher__pb2.WorkerUnsubscribeRequest.FromString,
response_serializer=dispatcher__pb2.WorkerUnsubscribeResponse.SerializeToString,
),
}
generic_handler = grpc.method_handlers_generic_handler(
'Dispatcher', rpc_method_handlers)
server.add_generic_rpc_handlers((generic_handler,))
# This class is part of an EXPERIMENTAL API.
class Dispatcher(object):
"""Missing associated documentation comment in .proto file."""
@staticmethod
def Register(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/Dispatcher/Register',
dispatcher__pb2.WorkerRegisterRequest.SerializeToString,
dispatcher__pb2.WorkerRegisterResponse.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
@staticmethod
def Listen(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_stream(request, target, '/Dispatcher/Listen',
dispatcher__pb2.WorkerListenRequest.SerializeToString,
dispatcher__pb2.AssignedAction.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
@staticmethod
def SendActionEvent(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/Dispatcher/SendActionEvent',
dispatcher__pb2.ActionEvent.SerializeToString,
dispatcher__pb2.ActionEventResponse.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
@staticmethod
def Unsubscribe(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/Dispatcher/Unsubscribe',
dispatcher__pb2.WorkerUnsubscribeRequest.SerializeToString,
dispatcher__pb2.WorkerUnsubscribeResponse.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)

View File

@@ -0,0 +1,38 @@
# -*- coding: utf-8 -*-
# Generated by the protocol buffer compiler. DO NOT EDIT!
# source: events.proto
# Protobuf Python Version: 4.25.0
"""Generated protocol buffer code."""
from google.protobuf import descriptor as _descriptor
from google.protobuf import descriptor_pool as _descriptor_pool
from google.protobuf import symbol_database as _symbol_database
from google.protobuf.internal import builder as _builder
# @@protoc_insertion_point(imports)
_sym_db = _symbol_database.Default()
from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0c\x65vents.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"|\n\x05\x45vent\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x0f\n\x07\x65ventId\x18\x02 \x01(\t\x12\x0b\n\x03key\x18\x03 \x01(\t\x12\x0f\n\x07payload\x18\x04 \x01(\t\x12\x32\n\x0e\x65ventTimestamp\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\"v\n\x10PushEventRequest\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x0b\n\x03key\x18\x02 \x01(\t\x12\x0f\n\x07payload\x18\x03 \x01(\t\x12\x32\n\x0e\x65ventTimestamp\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\"A\n\x10ListEventRequest\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x0e\n\x06offset\x18\x02 \x01(\x05\x12\x0b\n\x03key\x18\x03 \x01(\t\"+\n\x11ListEventResponse\x12\x16\n\x06\x65vents\x18\x01 \x03(\x0b\x32\x06.Event\"7\n\x12ReplayEventRequest\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x0f\n\x07\x65ventId\x18\x02 \x01(\t2\x99\x01\n\rEventsService\x12#\n\x04Push\x12\x11.PushEventRequest\x1a\x06.Event\"\x00\x12/\n\x04List\x12\x11.ListEventRequest\x1a\x12.ListEventResponse\"\x00\x12\x32\n\x11ReplaySingleEvent\x12\x13.ReplayEventRequest\x1a\x06.Event\"\x00\x42GZEgithub.com/hatchet-dev/hatchet/internal/services/dispatcher/contractsb\x06proto3')
_globals = globals()
_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals)
_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'events_pb2', _globals)
if _descriptor._USE_C_DESCRIPTORS == False:
_globals['DESCRIPTOR']._options = None
_globals['DESCRIPTOR']._serialized_options = b'ZEgithub.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts'
_globals['_EVENT']._serialized_start=49
_globals['_EVENT']._serialized_end=173
_globals['_PUSHEVENTREQUEST']._serialized_start=175
_globals['_PUSHEVENTREQUEST']._serialized_end=293
_globals['_LISTEVENTREQUEST']._serialized_start=295
_globals['_LISTEVENTREQUEST']._serialized_end=360
_globals['_LISTEVENTRESPONSE']._serialized_start=362
_globals['_LISTEVENTRESPONSE']._serialized_end=405
_globals['_REPLAYEVENTREQUEST']._serialized_start=407
_globals['_REPLAYEVENTREQUEST']._serialized_end=462
_globals['_EVENTSSERVICE']._serialized_start=465
_globals['_EVENTSSERVICE']._serialized_end=618
# @@protoc_insertion_point(module_scope)

View File

@@ -0,0 +1,57 @@
from google.protobuf import timestamp_pb2 as _timestamp_pb2
from google.protobuf.internal import containers as _containers
from google.protobuf import descriptor as _descriptor
from google.protobuf import message as _message
from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union
DESCRIPTOR: _descriptor.FileDescriptor
class Event(_message.Message):
__slots__ = ("tenantId", "eventId", "key", "payload", "eventTimestamp")
TENANTID_FIELD_NUMBER: _ClassVar[int]
EVENTID_FIELD_NUMBER: _ClassVar[int]
KEY_FIELD_NUMBER: _ClassVar[int]
PAYLOAD_FIELD_NUMBER: _ClassVar[int]
EVENTTIMESTAMP_FIELD_NUMBER: _ClassVar[int]
tenantId: str
eventId: str
key: str
payload: str
eventTimestamp: _timestamp_pb2.Timestamp
def __init__(self, tenantId: _Optional[str] = ..., eventId: _Optional[str] = ..., key: _Optional[str] = ..., payload: _Optional[str] = ..., eventTimestamp: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ...) -> None: ...
class PushEventRequest(_message.Message):
__slots__ = ("tenantId", "key", "payload", "eventTimestamp")
TENANTID_FIELD_NUMBER: _ClassVar[int]
KEY_FIELD_NUMBER: _ClassVar[int]
PAYLOAD_FIELD_NUMBER: _ClassVar[int]
EVENTTIMESTAMP_FIELD_NUMBER: _ClassVar[int]
tenantId: str
key: str
payload: str
eventTimestamp: _timestamp_pb2.Timestamp
def __init__(self, tenantId: _Optional[str] = ..., key: _Optional[str] = ..., payload: _Optional[str] = ..., eventTimestamp: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ...) -> None: ...
class ListEventRequest(_message.Message):
__slots__ = ("tenantId", "offset", "key")
TENANTID_FIELD_NUMBER: _ClassVar[int]
OFFSET_FIELD_NUMBER: _ClassVar[int]
KEY_FIELD_NUMBER: _ClassVar[int]
tenantId: str
offset: int
key: str
def __init__(self, tenantId: _Optional[str] = ..., offset: _Optional[int] = ..., key: _Optional[str] = ...) -> None: ...
class ListEventResponse(_message.Message):
__slots__ = ("events",)
EVENTS_FIELD_NUMBER: _ClassVar[int]
events: _containers.RepeatedCompositeFieldContainer[Event]
def __init__(self, events: _Optional[_Iterable[_Union[Event, _Mapping]]] = ...) -> None: ...
class ReplayEventRequest(_message.Message):
__slots__ = ("tenantId", "eventId")
TENANTID_FIELD_NUMBER: _ClassVar[int]
EVENTID_FIELD_NUMBER: _ClassVar[int]
tenantId: str
eventId: str
def __init__(self, tenantId: _Optional[str] = ..., eventId: _Optional[str] = ...) -> None: ...

View File

@@ -0,0 +1,132 @@
# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT!
"""Client and server classes corresponding to protobuf-defined services."""
import grpc
# import events_pb2 as events__pb2
from . import events_pb2 as events__pb2
class EventsServiceStub(object):
"""Missing associated documentation comment in .proto file."""
def __init__(self, channel):
"""Constructor.
Args:
channel: A grpc.Channel.
"""
self.Push = channel.unary_unary(
'/EventsService/Push',
request_serializer=events__pb2.PushEventRequest.SerializeToString,
response_deserializer=events__pb2.Event.FromString,
)
self.List = channel.unary_unary(
'/EventsService/List',
request_serializer=events__pb2.ListEventRequest.SerializeToString,
response_deserializer=events__pb2.ListEventResponse.FromString,
)
self.ReplaySingleEvent = channel.unary_unary(
'/EventsService/ReplaySingleEvent',
request_serializer=events__pb2.ReplayEventRequest.SerializeToString,
response_deserializer=events__pb2.Event.FromString,
)
class EventsServiceServicer(object):
"""Missing associated documentation comment in .proto file."""
def Push(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def List(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def ReplaySingleEvent(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def add_EventsServiceServicer_to_server(servicer, server):
rpc_method_handlers = {
'Push': grpc.unary_unary_rpc_method_handler(
servicer.Push,
request_deserializer=events__pb2.PushEventRequest.FromString,
response_serializer=events__pb2.Event.SerializeToString,
),
'List': grpc.unary_unary_rpc_method_handler(
servicer.List,
request_deserializer=events__pb2.ListEventRequest.FromString,
response_serializer=events__pb2.ListEventResponse.SerializeToString,
),
'ReplaySingleEvent': grpc.unary_unary_rpc_method_handler(
servicer.ReplaySingleEvent,
request_deserializer=events__pb2.ReplayEventRequest.FromString,
response_serializer=events__pb2.Event.SerializeToString,
),
}
generic_handler = grpc.method_handlers_generic_handler(
'EventsService', rpc_method_handlers)
server.add_generic_rpc_handlers((generic_handler,))
# This class is part of an EXPERIMENTAL API.
class EventsService(object):
"""Missing associated documentation comment in .proto file."""
@staticmethod
def Push(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/EventsService/Push',
events__pb2.PushEventRequest.SerializeToString,
events__pb2.Event.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
@staticmethod
def List(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/EventsService/List',
events__pb2.ListEventRequest.SerializeToString,
events__pb2.ListEventResponse.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
@staticmethod
def ReplaySingleEvent(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/EventsService/ReplaySingleEvent',
events__pb2.ReplayEventRequest.SerializeToString,
events__pb2.Event.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)

View File

@@ -0,0 +1,33 @@
from .client import new_client
from typing import List
from .workflow import WorkflowMeta
from .worker import Worker
class Hatchet:
def __init__(self):
# initialize a client
self.client = new_client()
def workflow(self, name : str='', on_events : list=[], on_crons : list=[]):
def inner(cls):
cls.on_events = on_events
cls.on_crons = on_crons
cls.name = name or str(cls.__name__)
cls.client = self.client
# Define a new class with the same name and bases as the original, but with WorkflowMeta as its metaclass
return WorkflowMeta(cls.__name__, cls.__bases__, dict(cls.__dict__))
return inner
def step(self, name : str='', parents : List[str] = []):
def inner(func):
func._step_name = name or func.__name__
func._step_parents = parents
return func
return inner
def worker(self, name: str, max_threads: int = 200):
return Worker(name, max_threads)

View File

@@ -0,0 +1,50 @@
import os
import yaml
from typing import Any, Optional, Dict
class ClientTLSConfig:
def __init__(self, cert_file: str, key_file: str, ca_file: str, server_name: str):
self.cert_file = cert_file
self.key_file = key_file
self.ca_file = ca_file
self.server_name = server_name
class ClientConfig:
def __init__(self, tenant_id: str, tls_config: ClientTLSConfig, host_port: str="localhost:7070"):
self.tenant_id = tenant_id
self.tls_config = tls_config
self.host_port = host_port
class ConfigLoader:
def __init__(self, directory: str):
self.directory = directory
def load_client_config(self) -> ClientConfig:
config_file_path = os.path.join(self.directory, "client.yaml")
config_data : Any = {
"tls": {},
}
# determine if client.yaml exists
if os.path.exists(config_file_path):
with open(config_file_path, 'r') as file:
config_data = yaml.safe_load(file)
tls_config = self._load_tls_config(config_data['tls'])
tenant_id = config_data['tenantId'] if 'tenantId' in config_data else self._get_env_var('HATCHET_CLIENT_TENANT_ID')
host_port = config_data['hostPort'] if 'hostPort' in config_data else self._get_env_var('HATCHET_CLIENT_HOST_PORT')
return ClientConfig(tenant_id, tls_config, host_port)
def _load_tls_config(self, tls_data: Dict) -> ClientTLSConfig:
cert_file = tls_data['tlsCertFile'] if 'tlsCertFile' in tls_data else self._get_env_var('HATCHET_CLIENT_TLS_CERT_FILE')
key_file = tls_data['tlsKeyFile'] if 'tlsKeyFile' in tls_data else self._get_env_var('HATCHET_CLIENT_TLS_KEY_FILE')
ca_file = tls_data['tlsRootCAFile'] if 'tlsRootCAFile' in tls_data else self._get_env_var('HATCHET_CLIENT_TLS_ROOT_CA_FILE')
server_name = tls_data['tlsServerName'] if 'tlsServerName' in tls_data else self._get_env_var('HATCHET_CLIENT_TLS_SERVER_NAME')
return ClientTLSConfig(cert_file, key_file, ca_file, server_name)
@staticmethod
def _get_env_var(env_var: str, default: Optional[str] = None) -> str:
return os.environ.get(env_var, default)

View File

@@ -0,0 +1,30 @@
def bump_minor_version(version: str) -> str:
"""
Bumps the minor version of a semantic version string. NOTE this doesn't follow full semver,
missing the build metadata and pre-release version.
:param version: A semantic version string in the format major.minor.patch
:return: A string with the minor version bumped and patch version reset to 0
:raises ValueError: If the input is not a valid semantic version string
"""
# if it starts with a v, remove it
had_v = False
if version.startswith('v'):
version = version[1:]
had_v = True
parts = version.split('.')
if len(parts) != 3:
raise ValueError(f"Invalid semantic version: {version}")
try:
major, minor, _ = map(int, parts)
except ValueError:
raise ValueError(f"Invalid semantic version: {version}")
new_minor = minor + 1
new_version = f"{major}.{new_minor}.0"
if had_v:
new_version = "v" + new_version
return new_version

View File

@@ -0,0 +1,137 @@
import json
from typing import Any, Callable, Dict
from .workflow import WorkflowMeta
from .clients.dispatcher import GetActionListenerRequest, ActionListenerImpl, Action
from .dispatcher_pb2 import ActionType, ActionEvent, ActionEventType, STEP_EVENT_TYPE_COMPLETED, STEP_EVENT_TYPE_STARTED
from .client import new_client
from concurrent.futures import ThreadPoolExecutor, Future
from google.protobuf.timestamp_pb2 import Timestamp
from .context import Context
# Worker class
class Worker:
def __init__(self, name: str, max_threads: int = 200):
self.name = name
self.thread_pool = ThreadPoolExecutor(max_workers=max_threads)
self.futures: Dict[str, Future] = {} # Store step run ids and futures
self.action_registry : dict[str, Callable[..., Any]] = {}
def handle_start_step_run(self, action : Action):
action_name = action.action_id # Assuming action object has 'name' attribute
context = Context(action.action_payload) # Assuming action object has 'context' attribute
# Find the corresponding action function from the registry
action_func = self.action_registry.get(action_name)
if action_func:
def callback(future : Future):
# Get the output from the future
try:
output = future.result()
except Exception as e:
# TODO: handle errors
print("error:", e)
raise e
# TODO: case on cancelled errors and such
# Create an action event
try:
event = self.get_action_finished_event(action, output)
except Exception as e:
print("error on action finished event:", e)
raise e
# Send the action event to the dispatcher
self.client.dispatcher.send_action_event(event)
# Remove the future from the dictionary
del self.futures[action.step_run_id]
del self.futures[action.step_run_id + "_callback"]
# Submit the action to the thread pool
future = self.thread_pool.submit(action_func, context)
callback = self.thread_pool.submit(callback, future)
self.futures[action.step_run_id] = future
self.futures[action.step_run_id + "_callback"] = callback
# send an event that the step run has started
try:
event = self.get_action_event(action, STEP_EVENT_TYPE_STARTED)
except Exception as e:
print("error on action event:", e)
# Send the action event to the dispatcher
self.client.dispatcher.send_action_event(event)
def handle_cancel_step_run(self, action : Action):
step_run_id = action.step_run_id
future = self.futures.get(step_run_id)
if future:
future.cancel()
del self.futures[step_run_id]
def get_action_event(self, action : Action, event_type : ActionEventType) -> ActionEvent:
# timestamp
# eventTimestamp = datetime.datetime.now(datetime.timezone.utc)
# eventTimestamp = eventTimestamp.isoformat()
eventTimestamp = Timestamp()
eventTimestamp.GetCurrentTime()
return ActionEvent(
tenantId=action.tenant_id,
workerId=action.worker_id,
jobId=action.job_id,
jobRunId=action.job_run_id,
stepId=action.step_id,
stepRunId=action.step_run_id,
actionId=action.action_id,
eventTimestamp=eventTimestamp,
eventType=event_type,
)
def get_action_finished_event(self, action : Action, output : Any) -> ActionEvent:
try:
event = self.get_action_event(action, STEP_EVENT_TYPE_COMPLETED)
except Exception as e:
print("error on get action event:", e)
raise e
output_bytes = ''
if output is not None:
output_bytes = json.dumps(output)
event.eventPayload = output_bytes
return event
def register_workflow(self, workflow : WorkflowMeta):
def create_action_function(action_func):
def action_function(context):
return action_func(workflow, context)
return action_function
for action_name, action_func in workflow.get_actions():
self.action_registry[action_name] = create_action_function(action_func)
def start(self):
self.client = new_client()
listener : ActionListenerImpl = self.client.dispatcher.get_action_listener(GetActionListenerRequest(
worker_name="test-worker",
services=["default"],
actions=self.action_registry.keys(),
))
generator = listener.actions()
for action in generator:
if action.action_type == ActionType.START_STEP_RUN:
self.handle_start_step_run(action)
elif action.action_type == ActionType.CANCEL_STEP_RUN:
self.handle_cancel_step_run(action)
pass # Replace this with your actual processing code

View File

@@ -0,0 +1,64 @@
from .client import new_client
from .workflows_pb2 import CreateWorkflowVersionOpts, CreateWorkflowJobOpts, CreateWorkflowStepOpts
from typing import Callable, List, Tuple, Any
stepsType = List[Tuple[str, Callable[..., Any]]]
class WorkflowMeta(type):
def __new__(cls, name, bases, attrs):
serviceName = "default"
steps: stepsType = [(func_name, attrs.pop(func_name)) for func_name, func in list(attrs.items()) if hasattr(func, '_step_name')]
# Define __init__ and get_step_order methods
original_init = attrs.get('__init__') # Get the original __init__ if it exists
def __init__(self, *args, **kwargs):
if original_init:
original_init(self, *args, **kwargs) # Call original __init__
def get_actions(self) -> stepsType:
return [(serviceName + ":" + func_name, func) for func_name, func in steps]
# Add these methods and steps to class attributes
attrs['__init__'] = __init__
attrs['get_actions'] = get_actions
for step_name, step_func in steps:
attrs[step_name] = step_func
# create a new hatchet client
client = attrs['client'] if 'client' in attrs else new_client()
attrs['client'] = client
name = attrs['name']
event_triggers = attrs['on_events']
cron_triggers = attrs['on_crons']
createStepOpts: List[CreateWorkflowStepOpts] = [
CreateWorkflowStepOpts(
readable_id=func_name,
action="default:" + func_name,
timeout="60s",
inputs='{}',
parents=[x for x in func._step_parents] # Assuming this is how you get the parents
)
for func_name, func in attrs.items() if hasattr(func, '_step_name')
]
client.admin.put_workflow(CreateWorkflowVersionOpts(
name=name,
version="v0.62.0",
event_triggers=event_triggers,
cron_triggers=cron_triggers,
jobs=[
CreateWorkflowJobOpts(
name="my-job",
timeout="60s",
steps=createStepOpts,
)
]
), auto_version=True)
return super(WorkflowMeta, cls).__new__(cls, name, bases, attrs)

View File

@@ -0,0 +1,63 @@
# -*- coding: utf-8 -*-
# Generated by the protocol buffer compiler. DO NOT EDIT!
# source: workflows.proto
# Protobuf Python Version: 4.25.0
"""Generated protocol buffer code."""
from google.protobuf import descriptor as _descriptor
from google.protobuf import descriptor_pool as _descriptor_pool
from google.protobuf import symbol_database as _symbol_database
from google.protobuf.internal import builder as _builder
# @@protoc_insertion_point(imports)
_sym_db = _symbol_database.Default()
from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2
from google.protobuf import wrappers_pb2 as google_dot_protobuf_dot_wrappers__pb2
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0fworkflows.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/wrappers.proto\"Q\n\x12PutWorkflowRequest\x12\x11\n\ttenant_id\x18\x01 \x01(\t\x12(\n\x04opts\x18\x02 \x01(\x0b\x32\x1a.CreateWorkflowVersionOpts\"\xdc\x01\n\x19\x43reateWorkflowVersionOpts\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12\x0f\n\x07version\x18\x03 \x01(\t\x12\x16\n\x0e\x65vent_triggers\x18\x04 \x03(\t\x12\x15\n\rcron_triggers\x18\x05 \x03(\t\x12\x36\n\x12scheduled_triggers\x18\x06 \x03(\x0b\x32\x1a.google.protobuf.Timestamp\x12$\n\x04jobs\x18\x07 \x03(\x0b\x32\x16.CreateWorkflowJobOpts\"s\n\x15\x43reateWorkflowJobOpts\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12\x0f\n\x07timeout\x18\x03 \x01(\t\x12&\n\x05steps\x18\x04 \x03(\x0b\x32\x17.CreateWorkflowStepOpts\"o\n\x16\x43reateWorkflowStepOpts\x12\x13\n\x0breadable_id\x18\x01 \x01(\t\x12\x0e\n\x06\x61\x63tion\x18\x02 \x01(\t\x12\x0f\n\x07timeout\x18\x03 \x01(\t\x12\x0e\n\x06inputs\x18\x04 \x01(\t\x12\x0f\n\x07parents\x18\x05 \x03(\t\")\n\x14ListWorkflowsRequest\x12\x11\n\ttenant_id\x18\x01 \x01(\t\"\x7f\n\x17ScheduleWorkflowRequest\x12\x11\n\ttenant_id\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12-\n\tschedules\x18\x03 \x03(\x0b\x32\x1a.google.protobuf.Timestamp\x12\r\n\x05input\x18\x04 \x01(\t\"5\n\x15ListWorkflowsResponse\x12\x1c\n\tworkflows\x18\x01 \x03(\x0b\x32\t.Workflow\"D\n\x1cListWorkflowsForEventRequest\x12\x11\n\ttenant_id\x18\x01 \x01(\t\x12\x11\n\tevent_key\x18\x02 \x01(\t\"\xee\x01\n\x08Workflow\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x11\n\ttenant_id\x18\x05 \x01(\t\x12\x0c\n\x04name\x18\x06 \x01(\t\x12\x31\n\x0b\x64\x65scription\x18\x07 \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\"\n\x08versions\x18\x08 \x03(\x0b\x32\x10.WorkflowVersion\"\xeb\x01\n\x0fWorkflowVersion\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0f\n\x07version\x18\x05 \x01(\t\x12\r\n\x05order\x18\x06 \x01(\x05\x12\x13\n\x0bworkflow_id\x18\x07 \x01(\t\x12#\n\x08triggers\x18\x08 \x01(\x0b\x32\x11.WorkflowTriggers\x12\x12\n\x04jobs\x18\t \x03(\x0b\x32\x04.Job\"\x80\x02\n\x10WorkflowTriggers\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x1b\n\x13workflow_version_id\x18\x05 \x01(\t\x12\x11\n\ttenant_id\x18\x06 \x01(\t\x12(\n\x06\x65vents\x18\x07 \x03(\x0b\x32\x18.WorkflowTriggerEventRef\x12&\n\x05\x63rons\x18\x08 \x03(\x0b\x32\x17.WorkflowTriggerCronRef\"?\n\x17WorkflowTriggerEventRef\x12\x11\n\tparent_id\x18\x01 \x01(\t\x12\x11\n\tevent_key\x18\x02 \x01(\t\"9\n\x16WorkflowTriggerCronRef\x12\x11\n\tparent_id\x18\x01 \x01(\t\x12\x0c\n\x04\x63ron\x18\x02 \x01(\t\"\xa7\x02\n\x03Job\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x11\n\ttenant_id\x18\x05 \x01(\t\x12\x1b\n\x13workflow_version_id\x18\x06 \x01(\t\x12\x0c\n\x04name\x18\x07 \x01(\t\x12\x31\n\x0b\x64\x65scription\x18\x08 \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\x14\n\x05steps\x18\t \x03(\x0b\x32\x05.Step\x12-\n\x07timeout\x18\n \x01(\x0b\x32\x1c.google.protobuf.StringValue\"\xaa\x02\n\x04Step\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x31\n\x0breadable_id\x18\x05 \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\x11\n\ttenant_id\x18\x06 \x01(\t\x12\x0e\n\x06job_id\x18\x07 \x01(\t\x12\x0e\n\x06\x61\x63tion\x18\x08 \x01(\t\x12-\n\x07timeout\x18\t \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\x0f\n\x07parents\x18\n \x03(\t\x12\x10\n\x08\x63hildren\x18\x0b \x03(\t\"?\n\x15\x44\x65leteWorkflowRequest\x12\x11\n\ttenant_id\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\";\n\x18GetWorkflowByNameRequest\x12\x11\n\ttenant_id\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t2\x87\x03\n\x0fWorkflowService\x12>\n\rListWorkflows\x12\x15.ListWorkflowsRequest\x1a\x16.ListWorkflowsResponse\x12\x34\n\x0bPutWorkflow\x12\x13.PutWorkflowRequest\x1a\x10.WorkflowVersion\x12>\n\x10ScheduleWorkflow\x12\x18.ScheduleWorkflowRequest\x1a\x10.WorkflowVersion\x12\x39\n\x11GetWorkflowByName\x12\x19.GetWorkflowByNameRequest\x1a\t.Workflow\x12N\n\x15ListWorkflowsForEvent\x12\x1d.ListWorkflowsForEventRequest\x1a\x16.ListWorkflowsResponse\x12\x33\n\x0e\x44\x65leteWorkflow\x12\x16.DeleteWorkflowRequest\x1a\t.WorkflowBBZ@github.com/hatchet-dev/hatchet/internal/services/admin/contractsb\x06proto3')
_globals = globals()
_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals)
_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'workflows_pb2', _globals)
if _descriptor._USE_C_DESCRIPTORS == False:
_globals['DESCRIPTOR']._options = None
_globals['DESCRIPTOR']._serialized_options = b'Z@github.com/hatchet-dev/hatchet/internal/services/admin/contracts'
_globals['_PUTWORKFLOWREQUEST']._serialized_start=84
_globals['_PUTWORKFLOWREQUEST']._serialized_end=165
_globals['_CREATEWORKFLOWVERSIONOPTS']._serialized_start=168
_globals['_CREATEWORKFLOWVERSIONOPTS']._serialized_end=388
_globals['_CREATEWORKFLOWJOBOPTS']._serialized_start=390
_globals['_CREATEWORKFLOWJOBOPTS']._serialized_end=505
_globals['_CREATEWORKFLOWSTEPOPTS']._serialized_start=507
_globals['_CREATEWORKFLOWSTEPOPTS']._serialized_end=618
_globals['_LISTWORKFLOWSREQUEST']._serialized_start=620
_globals['_LISTWORKFLOWSREQUEST']._serialized_end=661
_globals['_SCHEDULEWORKFLOWREQUEST']._serialized_start=663
_globals['_SCHEDULEWORKFLOWREQUEST']._serialized_end=790
_globals['_LISTWORKFLOWSRESPONSE']._serialized_start=792
_globals['_LISTWORKFLOWSRESPONSE']._serialized_end=845
_globals['_LISTWORKFLOWSFOREVENTREQUEST']._serialized_start=847
_globals['_LISTWORKFLOWSFOREVENTREQUEST']._serialized_end=915
_globals['_WORKFLOW']._serialized_start=918
_globals['_WORKFLOW']._serialized_end=1156
_globals['_WORKFLOWVERSION']._serialized_start=1159
_globals['_WORKFLOWVERSION']._serialized_end=1394
_globals['_WORKFLOWTRIGGERS']._serialized_start=1397
_globals['_WORKFLOWTRIGGERS']._serialized_end=1653
_globals['_WORKFLOWTRIGGEREVENTREF']._serialized_start=1655
_globals['_WORKFLOWTRIGGEREVENTREF']._serialized_end=1718
_globals['_WORKFLOWTRIGGERCRONREF']._serialized_start=1720
_globals['_WORKFLOWTRIGGERCRONREF']._serialized_end=1777
_globals['_JOB']._serialized_start=1780
_globals['_JOB']._serialized_end=2075
_globals['_STEP']._serialized_start=2078
_globals['_STEP']._serialized_end=2376
_globals['_DELETEWORKFLOWREQUEST']._serialized_start=2378
_globals['_DELETEWORKFLOWREQUEST']._serialized_end=2441
_globals['_GETWORKFLOWBYNAMEREQUEST']._serialized_start=2443
_globals['_GETWORKFLOWBYNAMEREQUEST']._serialized_end=2502
_globals['_WORKFLOWSERVICE']._serialized_start=2505
_globals['_WORKFLOWSERVICE']._serialized_end=2896
# @@protoc_insertion_point(module_scope)

View File

@@ -0,0 +1,226 @@
from google.protobuf import timestamp_pb2 as _timestamp_pb2
from google.protobuf import wrappers_pb2 as _wrappers_pb2
from google.protobuf.internal import containers as _containers
from google.protobuf import descriptor as _descriptor
from google.protobuf import message as _message
from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union
DESCRIPTOR: _descriptor.FileDescriptor
class PutWorkflowRequest(_message.Message):
__slots__ = ("tenant_id", "opts")
TENANT_ID_FIELD_NUMBER: _ClassVar[int]
OPTS_FIELD_NUMBER: _ClassVar[int]
tenant_id: str
opts: CreateWorkflowVersionOpts
def __init__(self, tenant_id: _Optional[str] = ..., opts: _Optional[_Union[CreateWorkflowVersionOpts, _Mapping]] = ...) -> None: ...
class CreateWorkflowVersionOpts(_message.Message):
__slots__ = ("name", "description", "version", "event_triggers", "cron_triggers", "scheduled_triggers", "jobs")
NAME_FIELD_NUMBER: _ClassVar[int]
DESCRIPTION_FIELD_NUMBER: _ClassVar[int]
VERSION_FIELD_NUMBER: _ClassVar[int]
EVENT_TRIGGERS_FIELD_NUMBER: _ClassVar[int]
CRON_TRIGGERS_FIELD_NUMBER: _ClassVar[int]
SCHEDULED_TRIGGERS_FIELD_NUMBER: _ClassVar[int]
JOBS_FIELD_NUMBER: _ClassVar[int]
name: str
description: str
version: str
event_triggers: _containers.RepeatedScalarFieldContainer[str]
cron_triggers: _containers.RepeatedScalarFieldContainer[str]
scheduled_triggers: _containers.RepeatedCompositeFieldContainer[_timestamp_pb2.Timestamp]
jobs: _containers.RepeatedCompositeFieldContainer[CreateWorkflowJobOpts]
def __init__(self, name: _Optional[str] = ..., description: _Optional[str] = ..., version: _Optional[str] = ..., event_triggers: _Optional[_Iterable[str]] = ..., cron_triggers: _Optional[_Iterable[str]] = ..., scheduled_triggers: _Optional[_Iterable[_Union[_timestamp_pb2.Timestamp, _Mapping]]] = ..., jobs: _Optional[_Iterable[_Union[CreateWorkflowJobOpts, _Mapping]]] = ...) -> None: ...
class CreateWorkflowJobOpts(_message.Message):
__slots__ = ("name", "description", "timeout", "steps")
NAME_FIELD_NUMBER: _ClassVar[int]
DESCRIPTION_FIELD_NUMBER: _ClassVar[int]
TIMEOUT_FIELD_NUMBER: _ClassVar[int]
STEPS_FIELD_NUMBER: _ClassVar[int]
name: str
description: str
timeout: str
steps: _containers.RepeatedCompositeFieldContainer[CreateWorkflowStepOpts]
def __init__(self, name: _Optional[str] = ..., description: _Optional[str] = ..., timeout: _Optional[str] = ..., steps: _Optional[_Iterable[_Union[CreateWorkflowStepOpts, _Mapping]]] = ...) -> None: ...
class CreateWorkflowStepOpts(_message.Message):
__slots__ = ("readable_id", "action", "timeout", "inputs", "parents")
READABLE_ID_FIELD_NUMBER: _ClassVar[int]
ACTION_FIELD_NUMBER: _ClassVar[int]
TIMEOUT_FIELD_NUMBER: _ClassVar[int]
INPUTS_FIELD_NUMBER: _ClassVar[int]
PARENTS_FIELD_NUMBER: _ClassVar[int]
readable_id: str
action: str
timeout: str
inputs: str
parents: _containers.RepeatedScalarFieldContainer[str]
def __init__(self, readable_id: _Optional[str] = ..., action: _Optional[str] = ..., timeout: _Optional[str] = ..., inputs: _Optional[str] = ..., parents: _Optional[_Iterable[str]] = ...) -> None: ...
class ListWorkflowsRequest(_message.Message):
__slots__ = ("tenant_id",)
TENANT_ID_FIELD_NUMBER: _ClassVar[int]
tenant_id: str
def __init__(self, tenant_id: _Optional[str] = ...) -> None: ...
class ScheduleWorkflowRequest(_message.Message):
__slots__ = ("tenant_id", "workflow_id", "schedules", "input")
TENANT_ID_FIELD_NUMBER: _ClassVar[int]
WORKFLOW_ID_FIELD_NUMBER: _ClassVar[int]
SCHEDULES_FIELD_NUMBER: _ClassVar[int]
INPUT_FIELD_NUMBER: _ClassVar[int]
tenant_id: str
workflow_id: str
schedules: _containers.RepeatedCompositeFieldContainer[_timestamp_pb2.Timestamp]
input: str
def __init__(self, tenant_id: _Optional[str] = ..., workflow_id: _Optional[str] = ..., schedules: _Optional[_Iterable[_Union[_timestamp_pb2.Timestamp, _Mapping]]] = ..., input: _Optional[str] = ...) -> None: ...
class ListWorkflowsResponse(_message.Message):
__slots__ = ("workflows",)
WORKFLOWS_FIELD_NUMBER: _ClassVar[int]
workflows: _containers.RepeatedCompositeFieldContainer[Workflow]
def __init__(self, workflows: _Optional[_Iterable[_Union[Workflow, _Mapping]]] = ...) -> None: ...
class ListWorkflowsForEventRequest(_message.Message):
__slots__ = ("tenant_id", "event_key")
TENANT_ID_FIELD_NUMBER: _ClassVar[int]
EVENT_KEY_FIELD_NUMBER: _ClassVar[int]
tenant_id: str
event_key: str
def __init__(self, tenant_id: _Optional[str] = ..., event_key: _Optional[str] = ...) -> None: ...
class Workflow(_message.Message):
__slots__ = ("id", "created_at", "updated_at", "tenant_id", "name", "description", "versions")
ID_FIELD_NUMBER: _ClassVar[int]
CREATED_AT_FIELD_NUMBER: _ClassVar[int]
UPDATED_AT_FIELD_NUMBER: _ClassVar[int]
TENANT_ID_FIELD_NUMBER: _ClassVar[int]
NAME_FIELD_NUMBER: _ClassVar[int]
DESCRIPTION_FIELD_NUMBER: _ClassVar[int]
VERSIONS_FIELD_NUMBER: _ClassVar[int]
id: str
created_at: _timestamp_pb2.Timestamp
updated_at: _timestamp_pb2.Timestamp
tenant_id: str
name: str
description: _wrappers_pb2.StringValue
versions: _containers.RepeatedCompositeFieldContainer[WorkflowVersion]
def __init__(self, id: _Optional[str] = ..., created_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., updated_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., tenant_id: _Optional[str] = ..., name: _Optional[str] = ..., description: _Optional[_Union[_wrappers_pb2.StringValue, _Mapping]] = ..., versions: _Optional[_Iterable[_Union[WorkflowVersion, _Mapping]]] = ...) -> None: ...
class WorkflowVersion(_message.Message):
__slots__ = ("id", "created_at", "updated_at", "version", "order", "workflow_id", "triggers", "jobs")
ID_FIELD_NUMBER: _ClassVar[int]
CREATED_AT_FIELD_NUMBER: _ClassVar[int]
UPDATED_AT_FIELD_NUMBER: _ClassVar[int]
VERSION_FIELD_NUMBER: _ClassVar[int]
ORDER_FIELD_NUMBER: _ClassVar[int]
WORKFLOW_ID_FIELD_NUMBER: _ClassVar[int]
TRIGGERS_FIELD_NUMBER: _ClassVar[int]
JOBS_FIELD_NUMBER: _ClassVar[int]
id: str
created_at: _timestamp_pb2.Timestamp
updated_at: _timestamp_pb2.Timestamp
version: str
order: int
workflow_id: str
triggers: WorkflowTriggers
jobs: _containers.RepeatedCompositeFieldContainer[Job]
def __init__(self, id: _Optional[str] = ..., created_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., updated_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., version: _Optional[str] = ..., order: _Optional[int] = ..., workflow_id: _Optional[str] = ..., triggers: _Optional[_Union[WorkflowTriggers, _Mapping]] = ..., jobs: _Optional[_Iterable[_Union[Job, _Mapping]]] = ...) -> None: ...
class WorkflowTriggers(_message.Message):
__slots__ = ("id", "created_at", "updated_at", "workflow_version_id", "tenant_id", "events", "crons")
ID_FIELD_NUMBER: _ClassVar[int]
CREATED_AT_FIELD_NUMBER: _ClassVar[int]
UPDATED_AT_FIELD_NUMBER: _ClassVar[int]
WORKFLOW_VERSION_ID_FIELD_NUMBER: _ClassVar[int]
TENANT_ID_FIELD_NUMBER: _ClassVar[int]
EVENTS_FIELD_NUMBER: _ClassVar[int]
CRONS_FIELD_NUMBER: _ClassVar[int]
id: str
created_at: _timestamp_pb2.Timestamp
updated_at: _timestamp_pb2.Timestamp
workflow_version_id: str
tenant_id: str
events: _containers.RepeatedCompositeFieldContainer[WorkflowTriggerEventRef]
crons: _containers.RepeatedCompositeFieldContainer[WorkflowTriggerCronRef]
def __init__(self, id: _Optional[str] = ..., created_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., updated_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., workflow_version_id: _Optional[str] = ..., tenant_id: _Optional[str] = ..., events: _Optional[_Iterable[_Union[WorkflowTriggerEventRef, _Mapping]]] = ..., crons: _Optional[_Iterable[_Union[WorkflowTriggerCronRef, _Mapping]]] = ...) -> None: ...
class WorkflowTriggerEventRef(_message.Message):
__slots__ = ("parent_id", "event_key")
PARENT_ID_FIELD_NUMBER: _ClassVar[int]
EVENT_KEY_FIELD_NUMBER: _ClassVar[int]
parent_id: str
event_key: str
def __init__(self, parent_id: _Optional[str] = ..., event_key: _Optional[str] = ...) -> None: ...
class WorkflowTriggerCronRef(_message.Message):
__slots__ = ("parent_id", "cron")
PARENT_ID_FIELD_NUMBER: _ClassVar[int]
CRON_FIELD_NUMBER: _ClassVar[int]
parent_id: str
cron: str
def __init__(self, parent_id: _Optional[str] = ..., cron: _Optional[str] = ...) -> None: ...
class Job(_message.Message):
__slots__ = ("id", "created_at", "updated_at", "tenant_id", "workflow_version_id", "name", "description", "steps", "timeout")
ID_FIELD_NUMBER: _ClassVar[int]
CREATED_AT_FIELD_NUMBER: _ClassVar[int]
UPDATED_AT_FIELD_NUMBER: _ClassVar[int]
TENANT_ID_FIELD_NUMBER: _ClassVar[int]
WORKFLOW_VERSION_ID_FIELD_NUMBER: _ClassVar[int]
NAME_FIELD_NUMBER: _ClassVar[int]
DESCRIPTION_FIELD_NUMBER: _ClassVar[int]
STEPS_FIELD_NUMBER: _ClassVar[int]
TIMEOUT_FIELD_NUMBER: _ClassVar[int]
id: str
created_at: _timestamp_pb2.Timestamp
updated_at: _timestamp_pb2.Timestamp
tenant_id: str
workflow_version_id: str
name: str
description: _wrappers_pb2.StringValue
steps: _containers.RepeatedCompositeFieldContainer[Step]
timeout: _wrappers_pb2.StringValue
def __init__(self, id: _Optional[str] = ..., created_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., updated_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., tenant_id: _Optional[str] = ..., workflow_version_id: _Optional[str] = ..., name: _Optional[str] = ..., description: _Optional[_Union[_wrappers_pb2.StringValue, _Mapping]] = ..., steps: _Optional[_Iterable[_Union[Step, _Mapping]]] = ..., timeout: _Optional[_Union[_wrappers_pb2.StringValue, _Mapping]] = ...) -> None: ...
class Step(_message.Message):
__slots__ = ("id", "created_at", "updated_at", "readable_id", "tenant_id", "job_id", "action", "timeout", "parents", "children")
ID_FIELD_NUMBER: _ClassVar[int]
CREATED_AT_FIELD_NUMBER: _ClassVar[int]
UPDATED_AT_FIELD_NUMBER: _ClassVar[int]
READABLE_ID_FIELD_NUMBER: _ClassVar[int]
TENANT_ID_FIELD_NUMBER: _ClassVar[int]
JOB_ID_FIELD_NUMBER: _ClassVar[int]
ACTION_FIELD_NUMBER: _ClassVar[int]
TIMEOUT_FIELD_NUMBER: _ClassVar[int]
PARENTS_FIELD_NUMBER: _ClassVar[int]
CHILDREN_FIELD_NUMBER: _ClassVar[int]
id: str
created_at: _timestamp_pb2.Timestamp
updated_at: _timestamp_pb2.Timestamp
readable_id: _wrappers_pb2.StringValue
tenant_id: str
job_id: str
action: str
timeout: _wrappers_pb2.StringValue
parents: _containers.RepeatedScalarFieldContainer[str]
children: _containers.RepeatedScalarFieldContainer[str]
def __init__(self, id: _Optional[str] = ..., created_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., updated_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., readable_id: _Optional[_Union[_wrappers_pb2.StringValue, _Mapping]] = ..., tenant_id: _Optional[str] = ..., job_id: _Optional[str] = ..., action: _Optional[str] = ..., timeout: _Optional[_Union[_wrappers_pb2.StringValue, _Mapping]] = ..., parents: _Optional[_Iterable[str]] = ..., children: _Optional[_Iterable[str]] = ...) -> None: ...
class DeleteWorkflowRequest(_message.Message):
__slots__ = ("tenant_id", "workflow_id")
TENANT_ID_FIELD_NUMBER: _ClassVar[int]
WORKFLOW_ID_FIELD_NUMBER: _ClassVar[int]
tenant_id: str
workflow_id: str
def __init__(self, tenant_id: _Optional[str] = ..., workflow_id: _Optional[str] = ...) -> None: ...
class GetWorkflowByNameRequest(_message.Message):
__slots__ = ("tenant_id", "name")
TENANT_ID_FIELD_NUMBER: _ClassVar[int]
NAME_FIELD_NUMBER: _ClassVar[int]
tenant_id: str
name: str
def __init__(self, tenant_id: _Optional[str] = ..., name: _Optional[str] = ...) -> None: ...

View File

@@ -0,0 +1,233 @@
# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT!
"""Client and server classes corresponding to protobuf-defined services."""
import grpc
from . import workflows_pb2 as workflows__pb2
class WorkflowServiceStub(object):
"""WorkflowService represents a set of RPCs for managing workflows.
"""
def __init__(self, channel):
"""Constructor.
Args:
channel: A grpc.Channel.
"""
self.ListWorkflows = channel.unary_unary(
'/WorkflowService/ListWorkflows',
request_serializer=workflows__pb2.ListWorkflowsRequest.SerializeToString,
response_deserializer=workflows__pb2.ListWorkflowsResponse.FromString,
)
self.PutWorkflow = channel.unary_unary(
'/WorkflowService/PutWorkflow',
request_serializer=workflows__pb2.PutWorkflowRequest.SerializeToString,
response_deserializer=workflows__pb2.WorkflowVersion.FromString,
)
self.ScheduleWorkflow = channel.unary_unary(
'/WorkflowService/ScheduleWorkflow',
request_serializer=workflows__pb2.ScheduleWorkflowRequest.SerializeToString,
response_deserializer=workflows__pb2.WorkflowVersion.FromString,
)
self.GetWorkflowByName = channel.unary_unary(
'/WorkflowService/GetWorkflowByName',
request_serializer=workflows__pb2.GetWorkflowByNameRequest.SerializeToString,
response_deserializer=workflows__pb2.Workflow.FromString,
)
self.ListWorkflowsForEvent = channel.unary_unary(
'/WorkflowService/ListWorkflowsForEvent',
request_serializer=workflows__pb2.ListWorkflowsForEventRequest.SerializeToString,
response_deserializer=workflows__pb2.ListWorkflowsResponse.FromString,
)
self.DeleteWorkflow = channel.unary_unary(
'/WorkflowService/DeleteWorkflow',
request_serializer=workflows__pb2.DeleteWorkflowRequest.SerializeToString,
response_deserializer=workflows__pb2.Workflow.FromString,
)
class WorkflowServiceServicer(object):
"""WorkflowService represents a set of RPCs for managing workflows.
"""
def ListWorkflows(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def PutWorkflow(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def ScheduleWorkflow(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def GetWorkflowByName(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def ListWorkflowsForEvent(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def DeleteWorkflow(self, request, context):
"""Missing associated documentation comment in .proto file."""
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
context.set_details('Method not implemented!')
raise NotImplementedError('Method not implemented!')
def add_WorkflowServiceServicer_to_server(servicer, server):
rpc_method_handlers = {
'ListWorkflows': grpc.unary_unary_rpc_method_handler(
servicer.ListWorkflows,
request_deserializer=workflows__pb2.ListWorkflowsRequest.FromString,
response_serializer=workflows__pb2.ListWorkflowsResponse.SerializeToString,
),
'PutWorkflow': grpc.unary_unary_rpc_method_handler(
servicer.PutWorkflow,
request_deserializer=workflows__pb2.PutWorkflowRequest.FromString,
response_serializer=workflows__pb2.WorkflowVersion.SerializeToString,
),
'ScheduleWorkflow': grpc.unary_unary_rpc_method_handler(
servicer.ScheduleWorkflow,
request_deserializer=workflows__pb2.ScheduleWorkflowRequest.FromString,
response_serializer=workflows__pb2.WorkflowVersion.SerializeToString,
),
'GetWorkflowByName': grpc.unary_unary_rpc_method_handler(
servicer.GetWorkflowByName,
request_deserializer=workflows__pb2.GetWorkflowByNameRequest.FromString,
response_serializer=workflows__pb2.Workflow.SerializeToString,
),
'ListWorkflowsForEvent': grpc.unary_unary_rpc_method_handler(
servicer.ListWorkflowsForEvent,
request_deserializer=workflows__pb2.ListWorkflowsForEventRequest.FromString,
response_serializer=workflows__pb2.ListWorkflowsResponse.SerializeToString,
),
'DeleteWorkflow': grpc.unary_unary_rpc_method_handler(
servicer.DeleteWorkflow,
request_deserializer=workflows__pb2.DeleteWorkflowRequest.FromString,
response_serializer=workflows__pb2.Workflow.SerializeToString,
),
}
generic_handler = grpc.method_handlers_generic_handler(
'WorkflowService', rpc_method_handlers)
server.add_generic_rpc_handlers((generic_handler,))
# This class is part of an EXPERIMENTAL API.
class WorkflowService(object):
"""WorkflowService represents a set of RPCs for managing workflows.
"""
@staticmethod
def ListWorkflows(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/WorkflowService/ListWorkflows',
workflows__pb2.ListWorkflowsRequest.SerializeToString,
workflows__pb2.ListWorkflowsResponse.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
@staticmethod
def PutWorkflow(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/WorkflowService/PutWorkflow',
workflows__pb2.PutWorkflowRequest.SerializeToString,
workflows__pb2.WorkflowVersion.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
@staticmethod
def ScheduleWorkflow(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/WorkflowService/ScheduleWorkflow',
workflows__pb2.ScheduleWorkflowRequest.SerializeToString,
workflows__pb2.WorkflowVersion.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
@staticmethod
def GetWorkflowByName(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/WorkflowService/GetWorkflowByName',
workflows__pb2.GetWorkflowByNameRequest.SerializeToString,
workflows__pb2.Workflow.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
@staticmethod
def ListWorkflowsForEvent(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/WorkflowService/ListWorkflowsForEvent',
workflows__pb2.ListWorkflowsForEventRequest.SerializeToString,
workflows__pb2.ListWorkflowsResponse.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
@staticmethod
def DeleteWorkflow(request,
target,
options=(),
channel_credentials=None,
call_credentials=None,
insecure=False,
compression=None,
wait_for_ready=None,
timeout=None,
metadata=None):
return grpc.experimental.unary_unary(request, target, '/WorkflowService/DeleteWorkflow',
workflows__pb2.DeleteWorkflowRequest.SerializeToString,
workflows__pb2.Workflow.FromString,
options, channel_credentials,
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)

249
python-client/poetry.lock generated Normal file
View File

@@ -0,0 +1,249 @@
# This file is automatically @generated by Poetry 1.7.1 and should not be changed by hand.
[[package]]
name = "grpcio"
version = "1.60.0"
description = "HTTP/2-based RPC framework"
optional = false
python-versions = ">=3.7"
files = [
{file = "grpcio-1.60.0-cp310-cp310-linux_armv7l.whl", hash = "sha256:d020cfa595d1f8f5c6b343530cd3ca16ae5aefdd1e832b777f9f0eb105f5b139"},
{file = "grpcio-1.60.0-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:b98f43fcdb16172dec5f4b49f2fece4b16a99fd284d81c6bbac1b3b69fcbe0ff"},
{file = "grpcio-1.60.0-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:20e7a4f7ded59097c84059d28230907cd97130fa74f4a8bfd1d8e5ba18c81491"},
{file = "grpcio-1.60.0-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:452ca5b4afed30e7274445dd9b441a35ece656ec1600b77fff8c216fdf07df43"},
{file = "grpcio-1.60.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:43e636dc2ce9ece583b3e2ca41df5c983f4302eabc6d5f9cd04f0562ee8ec1ae"},
{file = "grpcio-1.60.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:6e306b97966369b889985a562ede9d99180def39ad42c8014628dd3cc343f508"},
{file = "grpcio-1.60.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:f897c3b127532e6befdcf961c415c97f320d45614daf84deba0a54e64ea2457b"},
{file = "grpcio-1.60.0-cp310-cp310-win32.whl", hash = "sha256:b87efe4a380887425bb15f220079aa8336276398dc33fce38c64d278164f963d"},
{file = "grpcio-1.60.0-cp310-cp310-win_amd64.whl", hash = "sha256:a9c7b71211f066908e518a2ef7a5e211670761651039f0d6a80d8d40054047df"},
{file = "grpcio-1.60.0-cp311-cp311-linux_armv7l.whl", hash = "sha256:fb464479934778d7cc5baf463d959d361954d6533ad34c3a4f1d267e86ee25fd"},
{file = "grpcio-1.60.0-cp311-cp311-macosx_10_10_universal2.whl", hash = "sha256:4b44d7e39964e808b071714666a812049765b26b3ea48c4434a3b317bac82f14"},
{file = "grpcio-1.60.0-cp311-cp311-manylinux_2_17_aarch64.whl", hash = "sha256:90bdd76b3f04bdb21de5398b8a7c629676c81dfac290f5f19883857e9371d28c"},
{file = "grpcio-1.60.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:91229d7203f1ef0ab420c9b53fe2ca5c1fbeb34f69b3bc1b5089466237a4a134"},
{file = "grpcio-1.60.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3b36a2c6d4920ba88fa98075fdd58ff94ebeb8acc1215ae07d01a418af4c0253"},
{file = "grpcio-1.60.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:297eef542156d6b15174a1231c2493ea9ea54af8d016b8ca7d5d9cc65cfcc444"},
{file = "grpcio-1.60.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:87c9224acba0ad8bacddf427a1c2772e17ce50b3042a789547af27099c5f751d"},
{file = "grpcio-1.60.0-cp311-cp311-win32.whl", hash = "sha256:95ae3e8e2c1b9bf671817f86f155c5da7d49a2289c5cf27a319458c3e025c320"},
{file = "grpcio-1.60.0-cp311-cp311-win_amd64.whl", hash = "sha256:467a7d31554892eed2aa6c2d47ded1079fc40ea0b9601d9f79204afa8902274b"},
{file = "grpcio-1.60.0-cp312-cp312-linux_armv7l.whl", hash = "sha256:a7152fa6e597c20cb97923407cf0934e14224af42c2b8d915f48bc3ad2d9ac18"},
{file = "grpcio-1.60.0-cp312-cp312-macosx_10_10_universal2.whl", hash = "sha256:7db16dd4ea1b05ada504f08d0dca1cd9b926bed3770f50e715d087c6f00ad748"},
{file = "grpcio-1.60.0-cp312-cp312-manylinux_2_17_aarch64.whl", hash = "sha256:b0571a5aef36ba9177e262dc88a9240c866d903a62799e44fd4aae3f9a2ec17e"},
{file = "grpcio-1.60.0-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6fd9584bf1bccdfff1512719316efa77be235469e1e3295dce64538c4773840b"},
{file = "grpcio-1.60.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d6a478581b1a1a8fdf3318ecb5f4d0cda41cacdffe2b527c23707c9c1b8fdb55"},
{file = "grpcio-1.60.0-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:77c8a317f0fd5a0a2be8ed5cbe5341537d5c00bb79b3bb27ba7c5378ba77dbca"},
{file = "grpcio-1.60.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:1c30bb23a41df95109db130a6cc1b974844300ae2e5d68dd4947aacba5985aa5"},
{file = "grpcio-1.60.0-cp312-cp312-win32.whl", hash = "sha256:2aef56e85901c2397bd557c5ba514f84de1f0ae5dd132f5d5fed042858115951"},
{file = "grpcio-1.60.0-cp312-cp312-win_amd64.whl", hash = "sha256:e381fe0c2aa6c03b056ad8f52f8efca7be29fb4d9ae2f8873520843b6039612a"},
{file = "grpcio-1.60.0-cp37-cp37m-linux_armv7l.whl", hash = "sha256:92f88ca1b956eb8427a11bb8b4a0c0b2b03377235fc5102cb05e533b8693a415"},
{file = "grpcio-1.60.0-cp37-cp37m-macosx_10_10_universal2.whl", hash = "sha256:e278eafb406f7e1b1b637c2cf51d3ad45883bb5bd1ca56bc05e4fc135dfdaa65"},
{file = "grpcio-1.60.0-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:a48edde788b99214613e440fce495bbe2b1e142a7f214cce9e0832146c41e324"},
{file = "grpcio-1.60.0-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:de2ad69c9a094bf37c1102b5744c9aec6cf74d2b635558b779085d0263166454"},
{file = "grpcio-1.60.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:073f959c6f570797272f4ee9464a9997eaf1e98c27cb680225b82b53390d61e6"},
{file = "grpcio-1.60.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:c826f93050c73e7769806f92e601e0efdb83ec8d7c76ddf45d514fee54e8e619"},
{file = "grpcio-1.60.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:9e30be89a75ee66aec7f9e60086fadb37ff8c0ba49a022887c28c134341f7179"},
{file = "grpcio-1.60.0-cp37-cp37m-win_amd64.whl", hash = "sha256:b0fb2d4801546598ac5cd18e3ec79c1a9af8b8f2a86283c55a5337c5aeca4b1b"},
{file = "grpcio-1.60.0-cp38-cp38-linux_armv7l.whl", hash = "sha256:9073513ec380434eb8d21970e1ab3161041de121f4018bbed3146839451a6d8e"},
{file = "grpcio-1.60.0-cp38-cp38-macosx_10_10_universal2.whl", hash = "sha256:74d7d9fa97809c5b892449b28a65ec2bfa458a4735ddad46074f9f7d9550ad13"},
{file = "grpcio-1.60.0-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:1434ca77d6fed4ea312901122dc8da6c4389738bf5788f43efb19a838ac03ead"},
{file = "grpcio-1.60.0-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e61e76020e0c332a98290323ecfec721c9544f5b739fab925b6e8cbe1944cf19"},
{file = "grpcio-1.60.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:675997222f2e2f22928fbba640824aebd43791116034f62006e19730715166c0"},
{file = "grpcio-1.60.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:5208a57eae445ae84a219dfd8b56e04313445d146873117b5fa75f3245bc1390"},
{file = "grpcio-1.60.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:428d699c8553c27e98f4d29fdc0f0edc50e9a8a7590bfd294d2edb0da7be3629"},
{file = "grpcio-1.60.0-cp38-cp38-win32.whl", hash = "sha256:83f2292ae292ed5a47cdcb9821039ca8e88902923198f2193f13959360c01860"},
{file = "grpcio-1.60.0-cp38-cp38-win_amd64.whl", hash = "sha256:705a68a973c4c76db5d369ed573fec3367d7d196673fa86614b33d8c8e9ebb08"},
{file = "grpcio-1.60.0-cp39-cp39-linux_armv7l.whl", hash = "sha256:c193109ca4070cdcaa6eff00fdb5a56233dc7610216d58fb81638f89f02e4968"},
{file = "grpcio-1.60.0-cp39-cp39-macosx_10_10_universal2.whl", hash = "sha256:676e4a44e740deaba0f4d95ba1d8c5c89a2fcc43d02c39f69450b1fa19d39590"},
{file = "grpcio-1.60.0-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:5ff21e000ff2f658430bde5288cb1ac440ff15c0d7d18b5fb222f941b46cb0d2"},
{file = "grpcio-1.60.0-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4c86343cf9ff7b2514dd229bdd88ebba760bd8973dac192ae687ff75e39ebfab"},
{file = "grpcio-1.60.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0fd3b3968ffe7643144580f260f04d39d869fcc2cddb745deef078b09fd2b328"},
{file = "grpcio-1.60.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:30943b9530fe3620e3b195c03130396cd0ee3a0d10a66c1bee715d1819001eaf"},
{file = "grpcio-1.60.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:b10241250cb77657ab315270b064a6c7f1add58af94befa20687e7c8d8603ae6"},
{file = "grpcio-1.60.0-cp39-cp39-win32.whl", hash = "sha256:79a050889eb8d57a93ed21d9585bb63fca881666fc709f5d9f7f9372f5e7fd03"},
{file = "grpcio-1.60.0-cp39-cp39-win_amd64.whl", hash = "sha256:8a97a681e82bc11a42d4372fe57898d270a2707f36c45c6676e49ce0d5c41353"},
{file = "grpcio-1.60.0.tar.gz", hash = "sha256:2199165a1affb666aa24adf0c97436686d0a61bc5fc113c037701fb7c7fceb96"},
]
[package.extras]
protobuf = ["grpcio-tools (>=1.60.0)"]
[[package]]
name = "grpcio-tools"
version = "1.60.0"
description = "Protobuf code generator for gRPC"
optional = false
python-versions = ">=3.7"
files = [
{file = "grpcio-tools-1.60.0.tar.gz", hash = "sha256:ed30499340228d733ff69fcf4a66590ed7921f94eb5a2bf692258b1280b9dac7"},
{file = "grpcio_tools-1.60.0-cp310-cp310-linux_armv7l.whl", hash = "sha256:6807b7a3f3e6e594566100bd7fe04a2c42ce6d5792652677f1aaf5aa5adaef3d"},
{file = "grpcio_tools-1.60.0-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:857c5351e9dc33a019700e171163f94fcc7e3ae0f6d2b026b10fda1e3c008ef1"},
{file = "grpcio_tools-1.60.0-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:ec0e401e9a43d927d216d5169b03c61163fb52b665c5af2fed851357b15aef88"},
{file = "grpcio_tools-1.60.0-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e68dc4474f30cad11a965f0eb5d37720a032b4720afa0ec19dbcea2de73b5aae"},
{file = "grpcio_tools-1.60.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bbf0ed772d2ae7e8e5d7281fcc00123923ab130b94f7a843eee9af405918f924"},
{file = "grpcio_tools-1.60.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:c771b19dce2bfe06899247168c077d7ab4e273f6655d8174834f9a6034415096"},
{file = "grpcio_tools-1.60.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:e5614cf0960456d21d8a0f4902e3e5e3bcacc4e400bf22f196e5dd8aabb978b7"},
{file = "grpcio_tools-1.60.0-cp310-cp310-win32.whl", hash = "sha256:87cf439178f3eb45c1a889b2e4a17cbb4c450230d92c18d9c57e11271e239c55"},
{file = "grpcio_tools-1.60.0-cp310-cp310-win_amd64.whl", hash = "sha256:687f576d7ff6ce483bc9a196d1ceac45144e8733b953620a026daed8e450bc38"},
{file = "grpcio_tools-1.60.0-cp311-cp311-linux_armv7l.whl", hash = "sha256:2a8a758701f3ac07ed85f5a4284c6a9ddefcab7913a8e552497f919349e72438"},
{file = "grpcio_tools-1.60.0-cp311-cp311-macosx_10_10_universal2.whl", hash = "sha256:7c1cde49631732356cb916ee1710507967f19913565ed5f9991e6c9cb37e3887"},
{file = "grpcio_tools-1.60.0-cp311-cp311-manylinux_2_17_aarch64.whl", hash = "sha256:d941749bd8dc3f8be58fe37183143412a27bec3df8482d5abd6b4ec3f1ac2924"},
{file = "grpcio_tools-1.60.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9ee35234f1da8fba7ddbc544856ff588243f1128ea778d7a1da3039be829a134"},
{file = "grpcio_tools-1.60.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b8f7a5094adb49e85db13ea3df5d99a976c2bdfd83b0ba26af20ebb742ac6786"},
{file = "grpcio_tools-1.60.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:24c4ead4a03037beaeb8ef2c90d13d70101e35c9fae057337ed1a9144ef10b53"},
{file = "grpcio_tools-1.60.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:811abb9c4fb6679e0058dfa123fb065d97b158b71959c0e048e7972bbb82ba0f"},
{file = "grpcio_tools-1.60.0-cp311-cp311-win32.whl", hash = "sha256:bd2a17b0193fbe4793c215d63ce1e01ae00a8183d81d7c04e77e1dfafc4b2b8a"},
{file = "grpcio_tools-1.60.0-cp311-cp311-win_amd64.whl", hash = "sha256:b22b1299b666eebd5752ba7719da536075eae3053abcf2898b65f763c314d9da"},
{file = "grpcio_tools-1.60.0-cp312-cp312-linux_armv7l.whl", hash = "sha256:74025fdd6d1cb7ba4b5d087995339e9a09f0c16cf15dfe56368b23e41ffeaf7a"},
{file = "grpcio_tools-1.60.0-cp312-cp312-macosx_10_10_universal2.whl", hash = "sha256:5a907a4f1ffba86501b2cdb8682346249ea032b922fc69a92f082ba045cca548"},
{file = "grpcio_tools-1.60.0-cp312-cp312-manylinux_2_17_aarch64.whl", hash = "sha256:1fbb9554466d560472f07d906bfc8dcaf52f365c2a407015185993e30372a886"},
{file = "grpcio_tools-1.60.0-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f10ef47460ce3c6fd400f05fe757b90df63486c9b84d1ecad42dcc5f80c8ac14"},
{file = "grpcio_tools-1.60.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:321b18f42a70813545e416ddcb8bf20defa407a8114906711c9710a69596ceda"},
{file = "grpcio_tools-1.60.0-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:081336d8258f1a56542aa8a7a5dec99a2b38d902e19fbdd744594783301b0210"},
{file = "grpcio_tools-1.60.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:addc9b23d6ff729d9f83d4a2846292d4c84f5eb2ec38f08489a6a0d66ac2b91e"},
{file = "grpcio_tools-1.60.0-cp312-cp312-win32.whl", hash = "sha256:e87cabac7969bdde309575edc2456357667a1b28262b2c1f12580ef48315b19d"},
{file = "grpcio_tools-1.60.0-cp312-cp312-win_amd64.whl", hash = "sha256:e70d867c120d9849093b0ac24d861e378bc88af2552e743d83b9f642d2caa7c2"},
{file = "grpcio_tools-1.60.0-cp37-cp37m-linux_armv7l.whl", hash = "sha256:559ce714fe212aaf4abbe1493c5bb8920def00cc77ce0d45266f4fd9d8b3166f"},
{file = "grpcio_tools-1.60.0-cp37-cp37m-macosx_10_10_universal2.whl", hash = "sha256:7a5263a0f2ddb7b1cfb2349e392cfc4f318722e0f48f886393e06946875d40f3"},
{file = "grpcio_tools-1.60.0-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:18976684a931ca4bcba65c78afa778683aefaae310f353e198b1823bf09775a0"},
{file = "grpcio_tools-1.60.0-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e5c519a0d4ba1ab44a004fa144089738c59278233e2010b2cf4527dc667ff297"},
{file = "grpcio_tools-1.60.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6170873b1e5b6580ebb99e87fb6e4ea4c48785b910bd7af838cc6e44b2bccb04"},
{file = "grpcio_tools-1.60.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:fb4df80868b3e397d5fbccc004c789d2668b622b51a9d2387b4c89c80d31e2c5"},
{file = "grpcio_tools-1.60.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:dba6e32c87b4af29b5f475fb2f470f7ee3140bfc128644f17c6c59ddeb670680"},
{file = "grpcio_tools-1.60.0-cp37-cp37m-win_amd64.whl", hash = "sha256:f610384dee4b1ca705e8da66c5b5fe89a2de3d165c5282c3d1ddf40cb18924e4"},
{file = "grpcio_tools-1.60.0-cp38-cp38-linux_armv7l.whl", hash = "sha256:4041538f55aad5b3ae7e25ab314d7995d689e968bfc8aa169d939a3160b1e4c6"},
{file = "grpcio_tools-1.60.0-cp38-cp38-macosx_10_10_universal2.whl", hash = "sha256:2fb4cf74bfe1e707cf10bc9dd38a1ebaa145179453d150febb121c7e9cd749bf"},
{file = "grpcio_tools-1.60.0-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:2fd1671c52f96e79a2302c8b1c1f78b8a561664b8b3d6946f20d8f1cc6b4225a"},
{file = "grpcio_tools-1.60.0-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:dd1e68c232fe01dd5312a8dbe52c50ecd2b5991d517d7f7446af4ba6334ba872"},
{file = "grpcio_tools-1.60.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:17a32b3da4fc0798cdcec0a9c974ac2a1e98298f151517bf9148294a3b1a5742"},
{file = "grpcio_tools-1.60.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:9970d384fb0c084b00945ef57d98d57a8d32be106d8f0bd31387f7cbfe411b5b"},
{file = "grpcio_tools-1.60.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:5ce6bbd4936977ec1114f2903eb4342781960d521b0d82f73afedb9335251f6f"},
{file = "grpcio_tools-1.60.0-cp38-cp38-win32.whl", hash = "sha256:2e00de389729ca8d8d1a63c2038703078a887ff738dc31be640b7da9c26d0d4f"},
{file = "grpcio_tools-1.60.0-cp38-cp38-win_amd64.whl", hash = "sha256:6192184b1f99372ff1d9594bd4b12264e3ff26440daba7eb043726785200ff77"},
{file = "grpcio_tools-1.60.0-cp39-cp39-linux_armv7l.whl", hash = "sha256:eae27f9b16238e2aaee84c77b5923c6924d6dccb0bdd18435bf42acc8473ae1a"},
{file = "grpcio_tools-1.60.0-cp39-cp39-macosx_10_10_universal2.whl", hash = "sha256:b96981f3a31b85074b73d97c8234a5ed9053d65a36b18f4a9c45a2120a5b7a0a"},
{file = "grpcio_tools-1.60.0-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:1748893efd05cf4a59a175d7fa1e4fbb652f4d84ccaa2109f7869a2be48ed25e"},
{file = "grpcio_tools-1.60.0-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7a6fe752205caae534f29fba907e2f59ff79aa42c6205ce9a467e9406cbac68c"},
{file = "grpcio_tools-1.60.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3456df087ea61a0972a5bc165aed132ed6ddcc63f5749e572f9fff84540bdbad"},
{file = "grpcio_tools-1.60.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:f3d916606dcf5610d4367918245b3d9d8cd0d2ec0b7043d1bbb8c50fe9815c3a"},
{file = "grpcio_tools-1.60.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:fc01bc1079279ec342f0f1b6a107b3f5dc3169c33369cf96ada6e2e171f74e86"},
{file = "grpcio_tools-1.60.0-cp39-cp39-win32.whl", hash = "sha256:2dd01257e4feff986d256fa0bac9f56de59dc735eceeeb83de1c126e2e91f653"},
{file = "grpcio_tools-1.60.0-cp39-cp39-win_amd64.whl", hash = "sha256:1b93ae8ffd18e9af9a965ebca5fa521e89066267de7abdde20721edc04e42721"},
]
[package.dependencies]
grpcio = ">=1.60.0"
protobuf = ">=4.21.6,<5.0dev"
setuptools = "*"
[[package]]
name = "protobuf"
version = "4.25.2"
description = ""
optional = false
python-versions = ">=3.8"
files = [
{file = "protobuf-4.25.2-cp310-abi3-win32.whl", hash = "sha256:b50c949608682b12efb0b2717f53256f03636af5f60ac0c1d900df6213910fd6"},
{file = "protobuf-4.25.2-cp310-abi3-win_amd64.whl", hash = "sha256:8f62574857ee1de9f770baf04dde4165e30b15ad97ba03ceac65f760ff018ac9"},
{file = "protobuf-4.25.2-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:2db9f8fa64fbdcdc93767d3cf81e0f2aef176284071507e3ede160811502fd3d"},
{file = "protobuf-4.25.2-cp37-abi3-manylinux2014_aarch64.whl", hash = "sha256:10894a2885b7175d3984f2be8d9850712c57d5e7587a2410720af8be56cdaf62"},
{file = "protobuf-4.25.2-cp37-abi3-manylinux2014_x86_64.whl", hash = "sha256:fc381d1dd0516343f1440019cedf08a7405f791cd49eef4ae1ea06520bc1c020"},
{file = "protobuf-4.25.2-cp38-cp38-win32.whl", hash = "sha256:33a1aeef4b1927431d1be780e87b641e322b88d654203a9e9d93f218ee359e61"},
{file = "protobuf-4.25.2-cp38-cp38-win_amd64.whl", hash = "sha256:47f3de503fe7c1245f6f03bea7e8d3ec11c6c4a2ea9ef910e3221c8a15516d62"},
{file = "protobuf-4.25.2-cp39-cp39-win32.whl", hash = "sha256:5e5c933b4c30a988b52e0b7c02641760a5ba046edc5e43d3b94a74c9fc57c1b3"},
{file = "protobuf-4.25.2-cp39-cp39-win_amd64.whl", hash = "sha256:d66a769b8d687df9024f2985d5137a337f957a0916cf5464d1513eee96a63ff0"},
{file = "protobuf-4.25.2-py3-none-any.whl", hash = "sha256:a8b7a98d4ce823303145bf3c1a8bdb0f2f4642a414b196f04ad9853ed0c8f830"},
{file = "protobuf-4.25.2.tar.gz", hash = "sha256:fe599e175cb347efc8ee524bcd4b902d11f7262c0e569ececcb89995c15f0a5e"},
]
[[package]]
name = "python-dotenv"
version = "1.0.0"
description = "Read key-value pairs from a .env file and set them as environment variables"
optional = false
python-versions = ">=3.8"
files = [
{file = "python-dotenv-1.0.0.tar.gz", hash = "sha256:a8df96034aae6d2d50a4ebe8216326c61c3eb64836776504fcca410e5937a3ba"},
{file = "python_dotenv-1.0.0-py3-none-any.whl", hash = "sha256:f5971a9226b701070a4bf2c38c89e5a3f0d64de8debda981d1db98583009122a"},
]
[package.extras]
cli = ["click (>=5.0)"]
[[package]]
name = "pyyaml"
version = "6.0.1"
description = "YAML parser and emitter for Python"
optional = false
python-versions = ">=3.6"
files = [
{file = "PyYAML-6.0.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d858aa552c999bc8a8d57426ed01e40bef403cd8ccdd0fc5f6f04a00414cac2a"},
{file = "PyYAML-6.0.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:fd66fc5d0da6d9815ba2cebeb4205f95818ff4b79c3ebe268e75d961704af52f"},
{file = "PyYAML-6.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:69b023b2b4daa7548bcfbd4aa3da05b3a74b772db9e23b982788168117739938"},
{file = "PyYAML-6.0.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:81e0b275a9ecc9c0c0c07b4b90ba548307583c125f54d5b6946cfee6360c733d"},
{file = "PyYAML-6.0.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ba336e390cd8e4d1739f42dfe9bb83a3cc2e80f567d8805e11b46f4a943f5515"},
{file = "PyYAML-6.0.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:326c013efe8048858a6d312ddd31d56e468118ad4cdeda36c719bf5bb6192290"},
{file = "PyYAML-6.0.1-cp310-cp310-win32.whl", hash = "sha256:bd4af7373a854424dabd882decdc5579653d7868b8fb26dc7d0e99f823aa5924"},
{file = "PyYAML-6.0.1-cp310-cp310-win_amd64.whl", hash = "sha256:fd1592b3fdf65fff2ad0004b5e363300ef59ced41c2e6b3a99d4089fa8c5435d"},
{file = "PyYAML-6.0.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:6965a7bc3cf88e5a1c3bd2e0b5c22f8d677dc88a455344035f03399034eb3007"},
{file = "PyYAML-6.0.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f003ed9ad21d6a4713f0a9b5a7a0a79e08dd0f221aff4525a2be4c346ee60aab"},
{file = "PyYAML-6.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:42f8152b8dbc4fe7d96729ec2b99c7097d656dc1213a3229ca5383f973a5ed6d"},
{file = "PyYAML-6.0.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:062582fca9fabdd2c8b54a3ef1c978d786e0f6b3a1510e0ac93ef59e0ddae2bc"},
{file = "PyYAML-6.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d2b04aac4d386b172d5b9692e2d2da8de7bfb6c387fa4f801fbf6fb2e6ba4673"},
{file = "PyYAML-6.0.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:e7d73685e87afe9f3b36c799222440d6cf362062f78be1013661b00c5c6f678b"},
{file = "PyYAML-6.0.1-cp311-cp311-win32.whl", hash = "sha256:1635fd110e8d85d55237ab316b5b011de701ea0f29d07611174a1b42f1444741"},
{file = "PyYAML-6.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:bf07ee2fef7014951eeb99f56f39c9bb4af143d8aa3c21b1677805985307da34"},
{file = "PyYAML-6.0.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:855fb52b0dc35af121542a76b9a84f8d1cd886ea97c84703eaa6d88e37a2ad28"},
{file = "PyYAML-6.0.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:40df9b996c2b73138957fe23a16a4f0ba614f4c0efce1e9406a184b6d07fa3a9"},
{file = "PyYAML-6.0.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6c22bec3fbe2524cde73d7ada88f6566758a8f7227bfbf93a408a9d86bcc12a0"},
{file = "PyYAML-6.0.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:8d4e9c88387b0f5c7d5f281e55304de64cf7f9c0021a3525bd3b1c542da3b0e4"},
{file = "PyYAML-6.0.1-cp312-cp312-win32.whl", hash = "sha256:d483d2cdf104e7c9fa60c544d92981f12ad66a457afae824d146093b8c294c54"},
{file = "PyYAML-6.0.1-cp312-cp312-win_amd64.whl", hash = "sha256:0d3304d8c0adc42be59c5f8a4d9e3d7379e6955ad754aa9d6ab7a398b59dd1df"},
{file = "PyYAML-6.0.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:50550eb667afee136e9a77d6dc71ae76a44df8b3e51e41b77f6de2932bfe0f47"},
{file = "PyYAML-6.0.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1fe35611261b29bd1de0070f0b2f47cb6ff71fa6595c077e42bd0c419fa27b98"},
{file = "PyYAML-6.0.1-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:704219a11b772aea0d8ecd7058d0082713c3562b4e271b849ad7dc4a5c90c13c"},
{file = "PyYAML-6.0.1-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:afd7e57eddb1a54f0f1a974bc4391af8bcce0b444685d936840f125cf046d5bd"},
{file = "PyYAML-6.0.1-cp36-cp36m-win32.whl", hash = "sha256:fca0e3a251908a499833aa292323f32437106001d436eca0e6e7833256674585"},
{file = "PyYAML-6.0.1-cp36-cp36m-win_amd64.whl", hash = "sha256:f22ac1c3cac4dbc50079e965eba2c1058622631e526bd9afd45fedd49ba781fa"},
{file = "PyYAML-6.0.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:b1275ad35a5d18c62a7220633c913e1b42d44b46ee12554e5fd39c70a243d6a3"},
{file = "PyYAML-6.0.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:18aeb1bf9a78867dc38b259769503436b7c72f7a1f1f4c93ff9a17de54319b27"},
{file = "PyYAML-6.0.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:596106435fa6ad000c2991a98fa58eeb8656ef2325d7e158344fb33864ed87e3"},
{file = "PyYAML-6.0.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:baa90d3f661d43131ca170712d903e6295d1f7a0f595074f151c0aed377c9b9c"},
{file = "PyYAML-6.0.1-cp37-cp37m-win32.whl", hash = "sha256:9046c58c4395dff28dd494285c82ba00b546adfc7ef001486fbf0324bc174fba"},
{file = "PyYAML-6.0.1-cp37-cp37m-win_amd64.whl", hash = "sha256:4fb147e7a67ef577a588a0e2c17b6db51dda102c71de36f8549b6816a96e1867"},
{file = "PyYAML-6.0.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1d4c7e777c441b20e32f52bd377e0c409713e8bb1386e1099c2415f26e479595"},
{file = "PyYAML-6.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a0cd17c15d3bb3fa06978b4e8958dcdc6e0174ccea823003a106c7d4d7899ac5"},
{file = "PyYAML-6.0.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:28c119d996beec18c05208a8bd78cbe4007878c6dd15091efb73a30e90539696"},
{file = "PyYAML-6.0.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7e07cbde391ba96ab58e532ff4803f79c4129397514e1413a7dc761ccd755735"},
{file = "PyYAML-6.0.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:49a183be227561de579b4a36efbb21b3eab9651dd81b1858589f796549873dd6"},
{file = "PyYAML-6.0.1-cp38-cp38-win32.whl", hash = "sha256:184c5108a2aca3c5b3d3bf9395d50893a7ab82a38004c8f61c258d4428e80206"},
{file = "PyYAML-6.0.1-cp38-cp38-win_amd64.whl", hash = "sha256:1e2722cc9fbb45d9b87631ac70924c11d3a401b2d7f410cc0e3bbf249f2dca62"},
{file = "PyYAML-6.0.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:9eb6caa9a297fc2c2fb8862bc5370d0303ddba53ba97e71f08023b6cd73d16a8"},
{file = "PyYAML-6.0.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:c8098ddcc2a85b61647b2590f825f3db38891662cfc2fc776415143f599bb859"},
{file = "PyYAML-6.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5773183b6446b2c99bb77e77595dd486303b4faab2b086e7b17bc6bef28865f6"},
{file = "PyYAML-6.0.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b786eecbdf8499b9ca1d697215862083bd6d2a99965554781d0d8d1ad31e13a0"},
{file = "PyYAML-6.0.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bc1bf2925a1ecd43da378f4db9e4f799775d6367bdb94671027b73b393a7c42c"},
{file = "PyYAML-6.0.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:04ac92ad1925b2cff1db0cfebffb6ffc43457495c9b3c39d3fcae417d7125dc5"},
{file = "PyYAML-6.0.1-cp39-cp39-win32.whl", hash = "sha256:faca3bdcf85b2fc05d06ff3fbc1f83e1391b3e724afa3feba7d13eeab355484c"},
{file = "PyYAML-6.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:510c9deebc5c0225e8c96813043e62b680ba2f9c50a08d3724c7f28a747d1486"},
{file = "PyYAML-6.0.1.tar.gz", hash = "sha256:bfdf460b1736c775f2ba9f6a92bca30bc2095067b8a9d77876d1fad6cc3b4a43"},
]
[[package]]
name = "setuptools"
version = "69.0.3"
description = "Easily download, build, install, upgrade, and uninstall Python packages"
optional = false
python-versions = ">=3.8"
files = [
{file = "setuptools-69.0.3-py3-none-any.whl", hash = "sha256:385eb4edd9c9d5c17540511303e39a147ce2fc04bc55289c322b9e5904fe2c05"},
{file = "setuptools-69.0.3.tar.gz", hash = "sha256:be1af57fc409f93647f2e8e4573a142ed38724b8cdd389706a867bb4efcf1e78"},
]
[package.extras]
docs = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "rst.linker (>=1.9)", "sphinx (<7.2.5)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (>=1,<2)", "sphinx-reredirects", "sphinxcontrib-towncrier"]
testing = ["build[virtualenv]", "filelock (>=3.4.0)", "flake8-2020", "ini2toml[lite] (>=0.9)", "jaraco.develop (>=7.21)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pip (>=19.1)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-mypy (>=0.9.1)", "pytest-perf", "pytest-ruff", "pytest-timeout", "pytest-xdist", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel"]
testing-integration = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "packaging (>=23.1)", "pytest", "pytest-enabler", "pytest-xdist", "tomli", "virtualenv (>=13.0.0)", "wheel"]
[metadata]
lock-version = "2.0"
python-versions = "^3.8"
content-hash = "e552286ee1c1bf75f33fb747f3380cc865da3e6a0bc5a25fd25e2525d328fd40"

17
python-client/publish.sh Normal file
View File

@@ -0,0 +1,17 @@
#!/bin/bash
# This scripts generates and publishes the python package.
# env name is required
if [ -z "$POETRY_PYPI_TOKEN_PYPI" ]; then
echo "Please set POETRY_PYPI_TOKEN_PYPI variable"
exit 1
fi
# version is required
if [ -z "$VERSION" ]; then
echo "Please set VERSION variable"
exit 1
fi
poetry build
poetry publish

View File

@@ -0,0 +1,19 @@
[tool.poetry]
name = "hatchet-sdk"
version = "0.3.0"
description = ""
authors = ["Alexander Belanger <alexander@hatchet.run>"]
readme = "README.md"
[tool.poetry.dependencies]
python = "^3.8"
grpcio = "^1.60.0"
python-dotenv = "^1.0.0"
protobuf = "^4.25.2"
pyyaml = "^6.0.1"
grpcio-tools = "^1.60.0"
[build-system]
requires = ["poetry-core"]
build-backend = "poetry.core.masonry.api"

View File