GraphQL API#

Dagster exposes a GraphQL API that allows clients to interact with Dagster programmatically. The API allows users to:

  • Query information about Dagster runs (both historical and currently executing)
  • Retrieve metadata about repositories, jobs, and ops (such as dependency structure and config schemas)
  • Launch job executions and re-executions, allowing users to trigger executions on custom events

Note about breaking changes#

The GraphQL API is still evolving and is subject to breaking changes. A large portion of the API is primarily for internal use by Dagit. For any of the queries below, we will be clear about breaking changes in release notes.

Using the GraphQL API#

The GraphQL API is served alongside the Dagit web server. To start the server, run dagit in your Dagster workspace.

dagit

Dagit serves the GraphQL endpoint at the /graphql endpoint. If you are running Dagit locally on port 3000, you can access the API at localhost:3000/graphql

Using the GraphQL Playground#

You can access the GraphQL Playground by navigating to the /graphql route in your browser. The GraphQL Playground contains the full GraphQL Schema and an interactive playground to write and test queries and mutations.

GraphQL Playground

Exploring the GraphQL Schema and Documentation#

Clicking on the Docs tab on the right edge of the playground opens up interactive documentation for the GraphQL API. The interactive documentation is the best way to explore the API and get information about which fields are available on the queries and mutations.

GraphQL Docs

Python Client (Experimental)#

Dagster also provides a Python Client

Experimental
to interface with Dagster's GraphQL API from Python.

Example Queries#

Get a list of Dagster runs#

To retrieve a list of all runs, use the runsOrError query.

query RunsQuery {
  runsOrError {
    __typename
    ... on Runs {
      results {
        runId
        jobName
        status
        runConfigYaml
        stats {
          ... on RunStatsSnapshot {
            startTime
            endTime
            stepsFailed
          }
        }
      }
    }
  }
}

Pagination#

You may eventually accumulate too many runs to return in one query. The runsOrError query takes in optional cursor and limit arguments for pagination:

query PaginatedRunsQuery {
  runsOrError(
    cursor: "7fd2e5ef-5591-43db-be15-1ebbbbed8bb5"
    limit: 10
  ) {
    __typename
    ... on Runs {
      results {
        runId
        jobName
        status
        runConfigYaml
        stats {
          ... on RunStatsSnapshot {
            startTime
            endTime
            stepsFailed
          }
        }
      }
    }
  }
}

Filters#

The runsOrError query also takes in an optional filter argument, of type RunsFilter. This query allows you to filter runs by:

  • run ID
  • job name
  • tags
  • statuses

For example, the following query will return all failed runs:

query FilteredRunsQuery {
  runsOrError(filter: { statuses: [FAILURE] }) {
    __typename
    ... on Runs {
      results {
        runId
        jobName
        status
        runConfigYaml
        stats {
          ... on RunStatsSnapshot {
            startTime
            endTime
            stepsFailed
          }
        }
      }
    }
  }
}

Get a list of repositories#

This query returns the names and location names of all the repositories currently loaded.

query RepositoriesQuery {
  repositoriesOrError {
    ... on RepositoryConnection {
      nodes {
        name
        location {
          name
        }
      }
    }
  }
}

Get a list of jobs within a repository:#

Given a repository, this query returns the names of all the jobs in the repository.

This query takes a selector, which is of type RepositorySelector. A repository selector consists of both the repository location name and repository name.

query JobsQuery(
  $repositoryLocationName: String!
  $repositoryName: String!
) {
  repositoryOrError(
    repositorySelector: {
      repositoryLocationName: $repositoryLocationName
      repositoryName: $repositoryName
    }
  ) {
    ... on Repository {
      jobs {
        name
      }
    }
  }
}

Launch a run#

To launch a run, we can use the launchRun mutation. Here, we define LaunchRunMutation to wrap our mutation and allow us to pass in the required arguments as query variables.

For this query, the required arguments are:

selector:

  • This is a dictionary that contains the repository location name, repository name, and job name

runConfigData:

  • This is where you define the run config for the job execution.
  • Note that runConfigData is of type RunConfigData. This type is used when passing in an arbitrary object for run config. This is any-typed in the GraphQL type system but must conform to the constraints of the config schema for this job. If it does not, the mutation returns a RunConfigValidationInvalid response
mutation LaunchRunMutation(
  $repositoryLocationName: String!
  $repositoryName: String!
  $jobName: String!
  $runConfigData: RunConfigData!
) {
  launchRun(
    executionParams: {
      selector: {
        repositoryLocationName: $repositoryLocationName
        repositoryName: $repositoryName
        jobName: $jobName
      }
      runConfigData: $runConfigData
    }
  ) {
    __typename
    ... on LaunchRunSuccess {
      run {
        runId
      }
    }
    ... on RunConfigValidationInvalid {
      errors {
        message
        reason
      }
    }
    ... on PythonError {
      message
    }
  }
}

Terminate an in-progress run#

If you want to stop execution of an in-progress run, use the terminateRun mutation. The only required argument for this mutation is the ID of the run.

mutation TerminateRun($runId: String!) {
  terminateRun(runId: $runId){
    __typename
    ... on TerminateRunSuccess{
      run {
        runId
      }
    }
    ... on TerminateRunFailure {
      message
    }
    ... on RunNotFoundError {
      runId
    }
    ... on PythonError {
      message
      stack
    }
  }
}