Running GitHub Actions through Temporal: A complete guide

AUTHORS
Lily Baginski Doar
DATE
Aug 06, 2025
DURATION
0 MIN

For those who aren’t aware, Temporal is made up of an open-source project as well as a hosted Cloud offering. Our engineers work hard to maintain 100% feature parity between these two. This means that any release of the Cloud product requires, at a minimum, communication across two repositories. As Temporal has grown, and our CI/CD pipelines grew with it, we ran into a familiar set of problems: GitHub Actions can do the tasks, but their orchestration doesn’t scale to complex release pipelines.

We needed more control over how actions run and the reliability to match.

That’s what inspired me to write this guide. Throughout, I’ll walk through how we used Temporal to build a durable orchestration layer for GitHub Actions. It covers the full stack: GitHub App authentication, dispatch ID tracking, Temporal Workflows and Activities, retry logic, long-polling with heartbeats, and even helpful production test tips. Basically, breaking down everything I wish I knew when facing this problem myself.

If you’ve ever wanted to treat CI/CD like a proper system, something observable, debuggable, and recoverable, then join me as we walk through building this system from the ground up.

Why add orchestration to GitHub Actions?#

GitHub Actions do a great job running CI/CD tasks (e.g. testing, building, deploying, etc.) but once you need more coordination or visibility, things start to deteriorate. Dispatching across multiple repos, handling long-running jobs, or recovering gracefully from failure aren't built in.

That's why you need orchestration. It’s a way to trigger, track, and manage actions as part of a bigger, more reliable system.

By combining GitHub Actions and Temporal, we created a CI/CD system that scales with our needs and survives failure without slapdash efforts to patch things up.

Architecture overview#

To start, here’s a high-level look at the system we built. This setup calls GitHub's Actions API through Temporal's Workflow orchestration to create reliable, observable CI/CD pipelines.

Naming conventions#

Before we get too deep into things, let’s clarify some overlapping terminology that can be confusing:

  • Temporal: Has "Workflows" (orchestration logic) and "Activities" (individual tasks)
  • GitHub: Has "actions" (YAML workflow files) and "workflows" (individual runs of those actions)

Throughout this guide, we'll use these naming conventions:

  • “Action" refers to GitHub concepts (the YAML files and their runs).
  • "Temporal Workflow" and "Temporal Activity" when referring to Temporal concepts.
  • "GitHub Action" when we need to be explicit about GitHub's side.

For example: "Our Temporal Workflow triggers a GitHub Action, then uses Temporal Activities to monitor the action run's status."

High-level components#

With that naming cleared up, let’s walk through the building blocks. To run GitHub Actions through Temporal, you need four main components:

  • GitHub API Client: Handles authentication via GitHub Apps and provides methods to trigger actions, search for action runs, and monitor their status
  • Temporal Workflow: Orchestrates the three-phase process of triggering and monitoring actions with proper error handling and retry logic
  • Temporal Activities: Execute the actual GitHub API calls with appropriate timeout and retry configurations for each operation type
  • Worker/Client Infrastructure: Runs the Temporal workers and provides a client interface for triggering workflows running-github-actions-through-Temporal

GitHub app authentication model#

The system uses GitHub Apps for secure, installation-scoped authentication. Unlike personal access tokens, GitHub Apps provide:

  • Fine-grained permissions per repository
  • Installation-level access control
  • Audit trails for all API operations
  • Automatic token rotation
  1. The app authenticates itself within your GitHub organization using a JSON Web Token (JWT).
  2. The app will then generate an installation access token with specific repository access.

You can go here if you want to learn more about authentication with GitHub Apps.

The dispatch ID challenge#

Now that we've discussed authentication, it’s time to move to orchestration. One critical challenge when dispatching actions through GitHub's API is that it doesn't provide the ID of the dispatched action run. This creates a coordination problem: how do you track an action you just triggered when you don't know its ID?

Our solution uses the community-accepted workaround of injecting our own unique identifier into the Action as an input parameter. Later in the process, we can search the step names of action runs to identify our specific run.

name: wait-then-echo
on:
  workflow_dispatch:
    inputs:
      dispatch_id:
        description: An ID used to identify the workflow run
        required: true
        type: string
jobs:
  dispatch:
    runs-on: ubuntu-latest
    steps:
      # Put the dispatch ID in the step name so we can find it later
      - name: Dispatch ${{ inputs.dispatch_id }}
        run: echo "Dispatch ${{ inputs.dispatch_id }}"

Temporal Workflow vs GitHub Actions relationship#

The integration creates a clear separation of concerns:

  • GitHub Actions: Handle the actual CI/CD work (e.g. testing, building, deploying).
  • Temporal Workflows: Orchestrate when and how actions run, with reliable execution guarantees.

This relationship enables:

  • Complex coordination: Chain multiple actions across repositories.
  • Reliable execution: Automatic retries and failure handling.
  • Observability: Complete audit trail of all orchestration decisions.
  • Long-running processes: Wait for actions that take hours without timeouts.

Setting up GitHub app authentication#

With our architecture mapped out, let’s look at the code. To trigger actions programmatically, we need to authenticate as a GitHub App. This provides secure, scoped access to repositories and their workflows.

We can use the ghinstallation library by bradleyfalzon to handle app and repository authentication. Once authenticated, we can use the go-github library to create a client for interacting with the GitHub API.

The authentication setup uses a two-step process. First, authenticate as the app using a credential obtained from an external source. Then, generate credentials specific to the repository you want to run the action in.

import (
	"github.com/bradleyfalzon/ghinstallation/v2"
	"github.com/google/go-github/v68/github"
)

type GitHubApp struct {
	Org        string
	ID         int64
	PrivateKey string
}

type Client struct {
	*github.Client
}

func NewClient(ctx context.Context, app GitHubApp, repo string) (*Client, error) {
	// Create app-level transport
	appTransport, err := ghinstallation.NewAppsTransport(
		http.DefaultTransport,
		app.ID,
		[]byte(app.PrivateKey),
	)
	if err != nil {
		return nil, err
	}

	// Find the installation for this repository
	appClient := github.NewClient(&http.Client{Transport: appTransport})
	installation, _, err := appClient.Apps.FindRepositoryInstallation(ctx, app.Org, repo)
	if err != nil {
		return nil, err
	}

	// Create installation-specific client
	installationTransport, err := ghinstallation.New(
		http.DefaultTransport,
		app.ID,
		installation.GetID(),
		[]byte(app.PrivateKey),
	)

	return &Client{Client: github.NewClient(&http.Client{Transport: installationTransport})}, nil
}

Core GitHub API Operations#

Once authenticated, we need three core operations: triggering actions, finding their IDs, and monitoring status.

Triggering an action:

func (client *Client) TriggerAction(ctx context.Context, org, repo, workflowFile, ref string, inputs map[string]any) error {
	_, err := client.Actions.CreateWorkflowDispatchEventByFileName(
		ctx, org, repo, workflowFile,
		github.CreateWorkflowDispatchEventRequest{
			Ref:    ref,
			Inputs: inputs,
		},
	)
	return err
}

Finding the action ID:

Since GitHub doesn’t return the run ID when triggering an action, we search for it by looking for our unique dispatch ID in the job’s step names.

func (client *Client) GetActionID(ctx context.Context, org, repo, workflowFile, ref, dispatchID string) (int64, error) {
	for runItem := range client.iterateActionRuns(ctx, org, repo, workflowFile) {
		for jobItem := range client.iterateRunJobs(ctx, org, repo, runItem.value.GetID()) {
			for _, step := range jobItem.value.Steps {
				if strings.Contains(step.GetName(), dispatchID) {
					return runItem.value.GetID(), nil
				}
			}
		}
	}
	return -1, errors.New("Action ID not found")
}

Getting the Action status:

type Action struct {
	Status     string
	Conclusion string
	URL        string
}

func (client *Client) GetActionStatus(ctx context.Context, org, repo string, actionID int64) (*Action, error) {
	run, _, err := client.Actions.GetWorkflowRunByID(ctx, org, repo, actionID)
	if err != nil {
		return nil, err
	}

	return &Action{
		Status:     run.GetStatus(),
		Conclusion: run.GetConclusion(),
		URL:        run.GetURL(),
	}, nil
}

Designing the Temporal Workflow#

At the center of the system is our main Workflow. This Temporal Workflow orchestrates the three-step process: trigger, discover, and monitor.

type GitHubActionRequest struct {
	Org          string // GitHub organization name
	Repo         string // Repository name
	Ref          string // Git reference (branch, tag, or commit SHA. Most of the time this will be main)
	WorkflowFile string // Action file name (e.g. deploy.yml)
	Inputs       []struct {
		Key   string // Input parameter name
		Value string // Input parameter value
	}
}

type GitHubActionResponse struct {
	Status     string // Action status: "queued", "in_progress", "completed"
	Conclusion string // Final result: "success", "failure", "cancelled", etc.
	URL        string // GitHub URL to view the action run
}

func RunGitHubAction(ctx workflow.Context, request GitHubActionRequest) (*GitHubActionResponse, error) {
	// Create a dispatch ID to track the GH action we are running
	dispatchID, err := uuidSideEffect(ctx)
	if err != nil {
		return nil, err
	}

	// Start the GH action
	if err = triggerGitHubAction(ctx, request, dispatchID); err != nil {
		return nil, err
	}

	// Use the dispatch ID we added to get the action's ID
	actionID, err := getActionID(ctx, request, dispatchID)
	if err != nil {
		return nil, err
	}

	// Wait for the action to finish and return its final status
	return awaitActionCompletion(ctx, request, actionID)
}

It’s worth pointing out that the Workflow uses a SideEffect to generate a UUID that will remain consistent across Workflow Replays. This is so that our Activities can freely retry using a consistent dispatchID for the entire Workflow execution.

func uuidSideEffect(ctx workflow.Context) (string, error) {
	sideEffect := workflow.SideEffect(ctx, func(ctx workflow.Context) any {
		return uuid.New().String()
	})
	var uuid string
	return uuid, sideEffect.Get(&uuid)
}

Implementing Temporal Activities#

Now, let’s zoom in on the activities themselves. In Temporal, Activities handle any code that can fail. For us, that means the GitHub API calls. Each Activity has specific timeout and retry configuration based on its expected behavior.

Triggering the action

You may have noticed that we’re passing a slice for a variable representing a map. This is because Go’s map type is non-deterministic and Temporal Activities require deterministic inputs so that they can be Replayed. We can handle this simply by converting the slice to a map before making the client call.

func (a *Activities) TriggerGitHubActionActivity(ctx context.Context,request GitHubActionRequest, dispatchID string) error {
	inputs := mapFromSlice(request.Inputs)

	// Check if the user is using the dispatch_id input key
	if _, ok := inputs["dispatch_id"]; ok {
		err := ReservedInputKeyError{}
		return temporal.NewNonRetryableApplicationError(err.Error(), err.Name(), nil)
	}

	// Add the dispatch ID to the inputs
	inputs["dispatch_id"] = dispatchID

	client, err := github.NewClient(ctx, a.App, request.Repo)
	if err != nil {
		return err
	}

	return client.TriggerAction(ctx, request.Org, request.Repo, request.WorkflowFile, request.Ref, inputs)
}

Finding the action ID

func (a *Activities) GetActionIDActivity(ctx context.Context, request GitHubActionRequest, dispatchID string) (int64, error) {
	client, err := github.NewClient(ctx, a.App, request.Repo)
	if err != nil {
		return -1, err
	}

	return client.GetActionID(ctx, request.Org, request.Repo, request.WorkflowFile, request.Ref, dispatchID)
}

Waiting for action completion

This Activity implements a polling pattern with heartbeats to keep the Activity alive while monitoring long-running actions.

The key here is using activity.RecordHeartbeat() to prevent timeouts and provide status updates to the workflow execution history. Additionally, adding the status as a detail in the heartbeat call greatly improves developer visibility of the action's status during execution.

This Activity implements a polling pattern with heartbeats to keep the Activity alive while monitoring long-running actions.

The key here is using activity.RecordHeartbeat() to prevent timeouts and provide status updates to the workflow execution history. Additionally, adding the current status of the running Action as a detail in the heartbeat call greatly improves developer visibility of the action's status during execution.

func (a *Activities) AwaitActionCompletionActivity(ctx context.Context, request GitHubActionRequest, actionID int64, pollRate time.Duration) (*GitHubActionResponse, error) {
	client, err := github.NewClient(ctx, a.App, request.Repo)
	if err != nil {
		return nil, err
	}

	for {
		status, err := client.GetActionStatus(ctx, request.Org, request.Repo, actionID)
		if err != nil {
			return nil, err
		}

		if status.IsRunning() {
			select {
			// The ctx timed out or was cancelled before the action completed
			case <-ctx.Done():
				return nil, ctx.Err()
			// Wait for the specified poll rate before checking again
			case <-time.After(pollRate):
				activity.RecordHeartbeat(ctx, status)
				continue
			}
		}

		resp := &GitHubActionResponse{
			Status:     status.Status,
			Conclusion: status.Conclusion,
			URL:        status.URL,
		}

		// Return a non-retryable error if the action failed since this activity cannot retry the action
    if status.IsFailure() {
			err := GithubActionConclusionError{}
			return resp, temporal.NewNonRetryableApplicationError(err.Error(), err.Name(), nil, status.Conclusion)
		}

		// The action completed successfully
		return resp, nil
	}
}

func (action Action) IsRunning() bool {
	terminalStates := []string{"completed", "cancelled", "failure", "neutral", "skipped", "success", "timed_out"}
	return !slices.Contains(terminalStates, action.Status)
}

func (action Action) IsFailure() bool {
  var failureStates = []string{"failure", "timed_out"}
	return slices.Contains(failureStates, action.Conclusion)
}

Error Handling#

At Temporal, we always stress reliability so proper error handling is essential. This means we must differentiate between transient errors (typically network issues) which should be retried and errors which should not be retried (an invalid request, for example).

Transient Errors:

Transient errors within Temporal Activities can be treated exactly like normal Go errors. If your activity returns an error, the activity will be retried according to its RetryPolicy.

Non-Retryable Errors:

Errors which should not be retried can be created with temporal.NewNonRetryableApplicationError(). For organization we create custom error types and configure the activity with its possible errors.

type ReservedInputKeyError struct{}

func (e ReservedInputKeyError) Error() string {
  return "dispatch_id input is a reserved input key"
}

func (e ReservedInputKeyError) Name() string {
	return "ReservedInputKeyError"
}

func triggerGitHubAction(ctx workflow.Context, request GitHubActionRequest, dispatchID string) error {
	...

	ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{
		RetryPolicy: &temporal.RetryPolicy{
			...
			// Errors that will prevent the activity from being retried.
			NonRetryableErrorTypes: []string{
				ReservedInputKeyError{}.Name(),
			},
		},
	})

	activity := (*Activities).TriggerGitHubActionActivity
	future := workflow.ExecuteActivity(ctx, activity, request, dispatchID)
	if err := future.Get(ctx, nil); err != nil {
		return err
	}

	return nil
}

Handling action retries:

Now we’re handling API errors and invalid requests, but what if the action itself fails?

If our activities had a one-to-one correspondence with an Action run, we could return an error and let Temporal automatically handle the retries. But that's not our case. Since we interact with our actions through multiple Activities, we must retry at the workflow level. By default, Temporal does not retry Workflows when they fail, but we can add a retry policy to the Workflow in the exact same way we would add one to an Activity.

func runWorkflow(ctx context.Context, temporalClient client.Client) error {
  opts := client.StartWorkflowOptions{
    TaskQueue: "my-gh-actions-task-queue",
    RetryPolicy: &temporal.RetryPolicy{
      // 0 means unlimited attempts
      MaximumAttempts: 0,
    },
  }
  ...

  future, err := temporalClient.ExecuteWorkflow(ctx, opts, wf, args)
  ...
}

Now we’re handling API errors and invalid requests, but what if the action itself fails?

If our activities had a one-to-one correspondence with an Action run, we could return an error and let Temporal automatically handle the retries. But that's not our case. Since we interact with our actions through multiple Activities, we must retry at the workflow level. By default, Temporal does not retry Workflows when they fail, but we can add a retry policy to the Workflow in the exact same way we would add one to an Activity.

func runWorkflow(ctx context.Context, temporalClient client.Client) error {
  opts := client.StartWorkflowOptions{
    TaskQueue: "my-gh-actions-task-queue",
    RetryPolicy: &temporal.RetryPolicy{
      // 0 means unlimited attempts
      MaximumAttempts: 0,
      ...
    },
  }
  ...

  future, err := temporalClient.ExecuteWorkflow(ctx, opts, wf, args)
  ...
}

Worker and client implementation#

The final step is setting up the worker to execute activities and a client to trigger workflows.

Worker setup:

func main() {
	// GitHub App configuration
	app := github.GitHubApp{
		Org:        "my-org",
		ID:         1234567,
		PrivateKey: "my-private-key",
	}

	// Create a Temporal client
	temporalClient, err := client.Dial(client.Options{})
	if err != nil {
		fmt.Println("Failed to create Temporal client:", err)
		os.Exit(1)
	}

	// Create a Temporal worker
	taskQueue := "my-task-queue"
	temporalWorker := worker.New(temporalClient, taskQueue, worker.Options{})

	// Register workflows and activities
	temporalWorker.RegisterWorkflow(workflows.RunGitHubAction)

	activities := workflows.Activities{App: app}
	temporalWorker.RegisterActivity(activities)

	// Run the worker
	if err = temporalWorker.Run(worker.InterruptCh()); err != nil {
		fmt.Println("Worker failure:", err)
		os.Exit(1)
	}
}

Client usage:

func main() {
	temporalClient, err := client.Dial(client.Options{})
	if err != nil {
		fmt.Println("Failed to create Temporal client:", err)
		os.Exit(1)
	}

	// Execute the workflow
	opts := client.StartWorkflowOptions{TaskQueue: "my-task-queue"}
	request := workflows.GitHubActionRequest{
		Org:          "my-org",
		Repo:         "my-repo",
		Ref:          "main",
		WorkflowFile: "wait-and-echo.yaml",
		Inputs: []struct {
			Key   string
			Value string
		}{
			{Key: "wait_time", Value: "100"},
			{Key: "message", Value: "my custom message"},
		},
	}

	future, err := temporalClient.ExecuteWorkflow(ctx, opts, workflows.RunGitHubAction, request)
	if err != nil {
		fmt.Println("Failed to execute workflow:", err)
		os.Exit(1)
	}

	var result workflows.GitHubActionResponse
	if err = future.Get(ctx, &result); err != nil {
		fmt.Println("Failed to get workflow result:", err)
		os.Exit(1)
	}

	fmt.Printf("Action completed with status: %s\n", result.Status)
}

Running it:

After you have set everything up, you can run the Temporal Server, the Worker, and the client.

# Start a Temporal server in the background
temporal server start-dev &

# Start the worker in the background
go run ./cmd/worker/main.go &

# Run the client to trigger an action
go run ./cmd/client/main.go

Go to http://localhost:7233 to see the Temporal Web UI and monitor the Workflow. Temporal Web UI Workflow

Go to your repository's Actions tab to see the action run. Repository Action tab

As you can see, we successfully ran our action, with custom inputs, completely through our Temporal Workflow.

Testing strategies#

Now we can orchestrate the running of any action we want, but we still need a way to test our CI/CD pipelines.

By default, actions don't have any form of dry run. This makes developing actions with side effects (creating artifacts, cutting branches, etc.) potentially dangerous. One wrong change can destroy a release branch. To combat this possibility, we've set up "mirror" repositories that are synced against the contents of our “real” repositories. With this, we can freely run integration tests of the entire CI/CD process.

name: Mirror Respostory
on:
  workflow_dispatch:
permissions:
  contents: read
  id-token: write
jobs:
  synchronize:
    runs-on: ubuntu-latest
    steps:
      - uses: actions/create-github-app-token@v2
        id: app-token
        with:
          app-id: ${{ secrets.MY_APP_ID }}
          private-key: ${{ secrets.MY_PRIVATE_KEY }}
          owner: ${{ github.repository_owner }}
          repositories: my-repo
      - uses: actions/checkout@v4
        with:
          repository: my-org/mirror-my-repo
          ref: main
          token: ${{ steps.app-token.outputs.token }}
      - name: Hard reset mirror-my-repo to match my-repo
        run: |
          git config --unset-all http.https://github.com/.extraheader
          git remote set-url origin https://token:${{ steps.app-token.outputs.token }}@github.com/my-org/mirror-my-repo
          git remote add upstream https://token:${{ steps.app-token.outputs.token }}@github.com/my-org/my-repo
          git remote set-url --push upstream NO_PUSH
          git fetch upstream main
          git reset --hard upstream/main
          git push --force origin main

This strategy does require careful consideration when the repositories you’re mirroring have actions that trigger on events like branch creation, or release creation.

One way of dealing with this problem is to use a repository level variable that defines when these side effects should be executed. Since this variable is not present in the mirror repository, we can safely interact with the mirror repositories without fear of triggering unexpected side effects.

Production considerations#

At this point, you can orchestrate the running of actions and you can safely test them, but there are a few additional tips that go a long way in production.

Action idempotency

Temporal recommends that all activities performed by a workflow be idempotent. Since Temporal cannot validate the tasks performed by the GitHub Actions you choose to run, it is up to you to write your actions with this in mind.

Running Actions from the marketplace

The GitHub API only allows you to trigger actions defined in your own repositories. If you specifically want to run actions from the marketplace, you can write a simple wrapper that just calls the marketplace action.

Debugging

Debugging failing actions often involves a slow iteration loop, since you have to navigate many pages and buttons to find the information you need. To help with this, we recommend outputting the action URL when heartbeating, or returning from an error. The developer can then check for action inputs, status, and logs all in one place through the Temporal Workflow UI.

Conclusion#

With Temporal and GitHub Actions, we got exactly what we needed: dependable CI, total transparency, predictable failures, and reliable scaling.

Our system reflects how we approach developer productivity at Temporal — as more than just running tasks — leading to inherently resilient designs. If you’re looking to make your CI workflows more predictable and debuggable, I hope this guide helps you get started faster.

You can view the complete source code for this blog here.

If you’re a person who is curious like I am, you can always join our Community Slack and chat with devs or even members of Temporal about topics like this. If you want to get your hands dirty, then you can start with a free trial of Temporal Cloud with $1,000 in free credits.

Temporal Cloud

Ready to see for yourself?

Sign up for Temporal Cloud today and get $1,000 in free credits.