Internals
Note that APIs imported from Dagster submodules are not considered stable, and are potentially subject to change in the future.
If you find yourself consulting these docs because you are writing custom components and plug-ins, please get in touch with the core team on our Slack. We’re curious what you’re up to, happy to help, excited for new community contributions, and eager to make the system as easy to work with as possible – including for teams who are looking to customize it.
Executors (Experimental)
APIs for constructing custom executors. This is considered advanced experimental usage. Please note that using Dagster-provided executors is considered stable, common usage.
- @dagster.executor
Define an executor.
The decorated function should accept an InitExecutorContext
InitExecutorContext
and return an instance of ExecutorExecutor
.Parameters:
- name (Optional[str]) – The name of the executor.
- config_schema (Optional[ConfigSchemaConfigSchema]) – The schema for the config. Configuration data available in
- requirements (Optional[List[ExecutorRequirement]]) – Any requirements that must
- class dagster.ExecutorDefinition
An executor is responsible for executing the steps of a job.
Parameters:
- name (str) – The name of the executor.
- config_schema (Optional[ConfigSchemaConfigSchema]) – The schema for the config. Configuration data
- requirements (Optional[List[ExecutorRequirement]]) – Any requirements that must
- executor_creation_fn (Optional[Callable]) – Should accept an InitExecutorContext
InitExecutorContext
- required_resource_keys (Optional[Set[str]]) – Keys for the resources required by the
- description (Optional[str]) – A description of the executor.
- configured
Wraps this object in an object of the same type that provides configuration to the inner object.
Using
configured
may result in config values being displayed in the Dagster UI, so it is not recommended to use this API with sensitive values, such as secrets.Parameters:
- config_or_config_fn (Union[Any, Callable[[Any], Any]]) – Either (1) Run configuration
- name (Optional[str]) – Name of the new definition. If not provided, the emitted
- config_schema (Optional[ConfigSchemaConfigSchema]) – If config_or_config_fn is a function, the config
- description (Optional[str]) – Description of the new definition. If not specified,
Returns (ConfigurableDefinition): A configured version of this object.
- property description
Description of executor, if provided.
- property executor_creation_fn
Callable that takes an InitExecutorContext
InitExecutorContext
and returns an instance of ExecutorExecutor
.
- property name
Name of the executor.
- class dagster.InitExecutorContext
Executor-specific initialization context.
- job
The job to be executed.
Type: IJob
- executor_def
The definition of the executor currently being constructed.
Type: ExecutorDefinition
- executor_config
The parsed config passed to the executor.
Type: dict
- instance
The current instance.
Type: DagsterInstance
- class dagster.Executor
- abstract execute
For the given context and execution plan, orchestrate a series of sub plan executions in a way that satisfies the whole plan being executed.
Parameters:
- plan_context (PlanOrchestrationContext) – The plan’s orchestration context.
- execution_plan (ExecutionPlan) – The plan to execute.
Returns: A stream of dagster events.
- abstract property retries
Whether retries are enabled or disabled for this instance of the executor.
Executors should allow this to be controlled via configuration if possible.
Returns: RetryMode
File Manager (Experimental)
- class dagster._core.storage.file_manager.FileManager
Base class for all file managers in dagster.
The file manager is an interface that can be implemented by resources to provide abstract access to a file system such as local disk, S3, or other cloud storage.
For examples of usage, see the documentation of the concrete file manager implementations.
- abstract copy_handle_to_local_temp
Copy a file represented by a file handle to a temp file.
In an implementation built around an object store such as S3, this method would be expected to download the file from S3 to local filesystem in a location assigned by the standard library’s
python:tempfile
module.Temp files returned by this method are not guaranteed to be reusable across solid boundaries. For files that must be available across solid boundaries, use the read()
read()
, read_data()read_data()
, write()write()
, and write_data()write_data()
methods.Parameters: file_handle (FileHandleFileHandle) – The handle to the file to make available as a local temp file.Returns: Path to the local temp file.Return type: str
- abstract delete_local_temp
Delete all local temporary files created by previous calls to copy_handle_to_local_temp()
copy_handle_to_local_temp()
.Should typically only be called by framework implementors.
- abstract read
Return a file-like stream for the file handle.
This may incur an expensive network call for file managers backed by object stores such as S3.
Parameters:
- file_handle (FileHandleFileHandle) – The file handle to make available as a stream.
- mode (str) – The mode in which to open the file. Default:
"rb"
.
Returns: A file-like stream.Return type: Union[TextIO, BinaryIO]
- abstract read_data
Return the bytes for a given file handle. This may incur an expensive network call for file managers backed by object stores such as s3.
Parameters: file_handle (FileHandleFileHandle) – The file handle for which to return bytes.Returns: Bytes for a given file handle.Return type: bytes
- abstract write
Write the bytes contained within the given file object into the file manager.
Parameters:
- file_obj (Union[TextIO, StringIO]) – A file-like object.
- mode (Optional[str]) – The mode in which to write the file into the file manager.
- ext (Optional[str]) – For file managers that support file extensions, the extension with
Returns: A handle to the newly created file.Return type: FileHandle
- abstract write_data
Write raw bytes into the file manager.
Parameters:
- data (bytes) – The bytes to write into the file manager.
- ext (Optional[str]) – For file managers that support file extensions, the extension with
Returns: A handle to the newly created file.Return type: FileHandle
- dagster.local_file_manager ResourceDefinition
FileManager that provides abstract access to a local filesystem.
By default, files will be stored in <local_artifact_storage>/storage/file_manager where <local_artifact_storage> can be configured the
dagster.yaml
file in$DAGSTER_HOME
.Implements the FileManager
FileManager
API.Examples:
import tempfile
from dagster import job, local_file_manager, op
@op(required_resource_keys=\{"file_manager"})
def write_files(context):
fh_1 = context.resources.file_manager.write_data(b"foo")
with tempfile.NamedTemporaryFile("w+") as fd:
fd.write("bar")
fd.seek(0)
fh_2 = context.resources.file_manager.write(fd, mode="w", ext=".txt")
return (fh_1, fh_2)
@op(required_resource_keys=\{"file_manager"})
def read_files(context, file_handles):
fh_1, fh_2 = file_handles
assert context.resources.file_manager.read_data(fh_2) == b"bar"
fd = context.resources.file_manager.read(fh_2, mode="r")
assert fd.read() == "foo"
fd.close()
@job(resource_defs=\{"file_manager": local_file_manager})
def files_pipeline():
read_files(write_files())Or to specify the file directory:
@job(
resource_defs=\{
"file_manager": local_file_manager.configured(\{"base_dir": "/my/base/dir"})
}
)
def files_pipeline():
read_files(write_files())
- class dagster.FileHandle
A reference to a file as manipulated by a FileManager.
Subclasses may handle files that are resident on the local file system, in an object store, or in any arbitrary place where a file can be stored.
This exists to handle the very common case where you wish to write a computation that reads, transforms, and writes files, but where you also want the same code to work in local development as well as on a cluster where the files will be stored in a globally available object store such as S3.
- abstract property path_desc
A representation of the file path for display purposes only.
- class dagster.LocalFileHandle
A reference to a file on a local filesystem.
- property path
The file’s path.
- property path_desc
A representation of the file path for display purposes only.
Instance
- class dagster.DagsterInstance
Core abstraction for managing Dagster’s access to storage and other resources.
Use DagsterInstance.get() to grab the current DagsterInstance which will load based on the values in the
dagster.yaml
file in$DAGSTER_HOME
.Alternatively, DagsterInstance.ephemeral() can use used which provides a set of transient in-memory components.
Configuration of this class should be done by setting values in
$DAGSTER_HOME/dagster.yaml
. For example, to use Postgres for dagster storage, you can write adagster.yaml
such as the following:dagster.yaml
storage:
postgres:
postgres_db:
username: my_username
password: my_password
hostname: my_hostname
db_name: my_database
port: 5432Parameters:
- instance_type (InstanceType) – Indicates whether the instance is ephemeral or persistent.
- local_artifact_storage (LocalArtifactStorageLocalArtifactStorage) – The local artifact storage is used to
- run_storage (RunStorageRunStorage) – The run storage is used to store metadata about ongoing and past
- event_storage (EventLogStorageEventLogStorage) – Used to store the structured event logs generated by
- compute_log_manager (Optional[ComputeLogManagerComputeLogManager]) – The compute log manager handles stdout
- run_coordinator (Optional[RunCoordinator]) – A runs coordinator may be used to manage the execution
- run_launcher (Optional[RunLauncherRunLauncher]) – Optionally, a run launcher may be used to enable
- settings (Optional[Dict]) – Specifies certain per-instance settings,
- ref (Optional[InstanceRefInstanceRef]) – Used by internal machinery to pass instances across process
- add_dynamic_partitions
Add partitions to the specified DynamicPartitionsDefinition
DynamicPartitionsDefinition
idempotently. Does not add any partitions that already exist.Parameters:
- partitions_def_name (str) – The name of the DynamicPartitionsDefinition.
- partition_keys (Sequence[str]) – Partition keys to add.
- delete_dynamic_partition
Delete a partition for the specified DynamicPartitionsDefinition
DynamicPartitionsDefinition
. If the partition does not exist, exits silently.Parameters:
- partitions_def_name (str) – The name of the DynamicPartitionsDefinition.
- partition_key (Sequence[str]) – Partition key to delete.
- delete_run
Delete a run and all events generated by that from storage.
Parameters: run_id (str) – The id of the run to delete.
- static ephemeral
Create a DagsterInstance suitable for ephemeral execution, useful in test contexts. An ephemeral instance uses mostly in-memory components. Use local_temp to create a test instance that is fully persistent.
Parameters:
- tempdir (Optional[str]) – The path of a directory to be used for local artifact storage.
- preload (Optional[Sequence[DebugRunPayload]]) – A sequence of payloads to load into the
- settings (Optional[Dict]) – Settings for the instance.
Returns: An ephemeral DagsterInstance.Return type: DagsterInstance
- fetch_materializations
Return a list of materialization records stored in the event log storage.
Parameters:
- records_filter (Union[AssetKeyAssetKey, AssetRecordsFilter]) – the filter by which to
- limit (int) – Number of results to get.
- cursor (Optional[str]) – Cursor to use for pagination. Defaults to None.
- ascending (Optional[bool]) – Sort the result in ascending order if True, descending
Returns: Object containing a list of event log records and a cursor stringReturn type: EventRecordsResult
- fetch_observations
Return a list of observation records stored in the event log storage.
Parameters:
- records_filter (Optional[Union[AssetKeyAssetKey, AssetRecordsFilter]]) – the filter by which to
- limit (int) – Number of results to get.
- cursor (Optional[str]) – Cursor to use for pagination. Defaults to None.
- ascending (Optional[bool]) – Sort the result in ascending order if True, descending
Returns: Object containing a list of event log records and a cursor stringReturn type: EventRecordsResult
- fetch_run_status_changes
Return a list of run_status_event records stored in the event log storage.
Parameters:
- records_filter (Optional[Union[DagsterEventTypeDagsterEventType, RunStatusChangeRecordsFilter]]) – the
- limit (int) – Number of results to get.
- cursor (Optional[str]) – Cursor to use for pagination. Defaults to None.
- ascending (Optional[bool]) – Sort the result in ascending order if True, descending
Returns: Object containing a list of event log records and a cursor stringReturn type: EventRecordsResult
- static get
Get the current DagsterInstance as specified by the
DAGSTER_HOME
environment variable.Returns: The current DagsterInstance.Return type: DagsterInstance
- get_asset_keys
Return a filtered subset of asset keys managed by this instance.
Parameters:
- prefix (Optional[Sequence[str]]) – Return only assets having this key prefix.
- limit (Optional[int]) – Maximum number of keys to return.
- cursor (Optional[str]) – Cursor to use for pagination.
Returns: List of asset keys.Return type: Sequence[AssetKey]
- get_asset_records
Return an AssetRecord for each of the given asset keys.
Parameters: asset_keys (Optional[Sequence[AssetKeyAssetKey]]) – List of asset keys to retrieve records for.Returns: List of asset records.Return type: Sequence[AssetRecord]
- get_dynamic_partitions
Get the set of partition keys for the specified DynamicPartitionsDefinition
DynamicPartitionsDefinition
.Parameters: partitions_def_name (str) – The name of the DynamicPartitionsDefinition.
- get_event_records
Return a list of event records stored in the event log storage.
Parameters:
- event_records_filter (Optional[EventRecordsFilterEventRecordsFilter]) – the filter by which to filter event
- limit (Optional[int]) – Number of results to get. Defaults to infinite.
- ascending (Optional[bool]) – Sort the result in ascending order if True, descending
Returns: List of event log records stored in the event log storage.Return type: List[EventLogRecord]
- get_latest_materialization_code_versions
Returns the code version used for the latest materialization of each of the provided assets.
Parameters: asset_keys (Iterable[AssetKeyAssetKey]) – The asset keys to find latest materialization code versions for.Returns: A dictionary with a key for each of the provided asset keys. The values will be None if the asset has no materializations. If an asset does not have a code version explicitly assigned to its definitions, but was materialized, Dagster assigns the run ID as its code version.
Return type: Mapping[AssetKey, Optional[str]]
- get_latest_materialization_event
Fetch the latest materialization event for the given asset key.
Parameters: asset_key (AssetKeyAssetKey) – Asset key to return materialization for.Returns: The latest materialization event for the given asset key, or None if the asset has not been materialized.
Return type: Optional[EventLogEntry]
- get_run_by_id
Get a DagsterRun
DagsterRun
matching the provided run_id.Parameters: run_id (str) – The id of the run to retrieve.Returns: The run corresponding to the given id. If no run matching the id is found, return None.
Return type: Optional[DagsterRun]
- get_run_record_by_id
Get a
RunRecord
matching the provided run_id.Parameters: run_id (str) – The id of the run record to retrieve.Returns: The run record corresponding to the given id. If no run matching the id is found, return None.
Return type: Optional[RunRecord]
- get_run_records
Return a list of run records stored in the run storage, sorted by the given column in given order.
Parameters:
- filters (Optional[RunsFilterRunsFilter]) – the filter by which to filter runs.
- limit (Optional[int]) – Number of results to get. Defaults to infinite.
- order_by (Optional[str]) – Name of the column to sort by. Defaults to id.
- ascending (Optional[bool]) – Sort the result in ascending order if True, descending
Returns: List of run records stored in the run storage.Return type: List[RunRecord]
- get_status_by_partition
Get the current status of provided partition_keys for the provided asset.
Parameters:
- asset_key (AssetKeyAssetKey) – The asset to get per-partition status for.
- partition_keys (Sequence[str]) – The partitions to get status for.
- partitions_def (PartitionsDefinitionPartitionsDefinition) – The PartitionsDefinition of the asset to get
Returns: status for each partition keyReturn type: Optional[Mapping[str, AssetPartitionStatus]]
- has_asset_key
Return true if this instance manages the given asset key.
Parameters: asset_key (AssetKeyAssetKey) – Asset key to check.
- has_dynamic_partition
Check if a partition key exists for the DynamicPartitionsDefinition
DynamicPartitionsDefinition
.Parameters:
- partitions_def_name (str) – The name of the DynamicPartitionsDefinition.
- partition_key (Sequence[str]) – Partition key to check.
- static local_temp
Create a DagsterInstance that uses a temporary directory for local storage. This is a regular, fully persistent instance. Use ephemeral to get an ephemeral instance with in-memory components.
Parameters:
- tempdir (Optional[str]) – The path of a directory to be used for local artifact storage.
- overrides (Optional[DagsterInstanceOverrides]) – Override settings for the instance.
Returns: DagsterInstance
- report_runless_asset_event
- experimental
This API may break in future versions, even between dot releases.
Record an event log entry related to assets that does not belong to a Dagster run.
- wipe_assets
Wipes asset event history from the event log for the given asset keys.
Parameters: asset_keys (Sequence[AssetKeyAssetKey]) – Asset keys to wipe.
- class dagster._core.instance.InstanceRef
Serializable representation of a DagsterInstance
DagsterInstance
.Users should not instantiate this class directly.
- class dagster._serdes.ConfigurableClass
Abstract mixin for classes that can be loaded from config.
This supports a powerful plugin pattern which avoids both a) a lengthy, hard-to-synchronize list of conditional imports / optional extras_requires in dagster core and b) a magic directory or file in which third parties can place plugin packages. Instead, the intention is to make, e.g., run storage, pluggable with a config chunk like:
run_storage:
module: very_cool_package.run_storage
class: SplendidRunStorage
config:
magic_word: "quux"This same pattern should eventually be viable for other system components, e.g. engines.
The
ConfigurableClass
mixin provides the necessary hooks for classes to be instantiated from an instance ofConfigurableClassData
.Pieces of the Dagster system which we wish to make pluggable in this way should consume a config type such as:
\{'module': str, 'class': str, 'config': Field(Permissive())}
- class dagster._serdes.ConfigurableClassData
Serializable tuple describing where to find a class and the config fragment that should be used to instantiate it.
Users should not instantiate this class directly.
Classes intended to be serialized in this way should implement the
dagster.serdes.ConfigurableClass
mixin.
Storage
- class dagster._core.storage.base_storage.DagsterStorage
Abstract base class for Dagster persistent storage, for reading and writing data for runs, events, and schedule/sensor state.
Users should not directly instantiate concrete subclasses of this class; they are instantiated by internal machinery when
dagster-webserver
anddagster-daemon
load, based on the values in thedagster.yaml
file in$DAGSTER_HOME
. Configuration of concrete subclasses of this class should be done by setting values in that file.
Run storage
- class dagster.DagsterRun
Serializable internal representation of a dagster run, as stored in a RunStorage
RunStorage
.- job_name
The name of the job executed in this run
Type: str
- run_id
The ID of the run
Type: str
- run_config
The config for the run
Type: Mapping[str, object]
- tags
The tags applied to the run
Type: Mapping[str, str]
- property is_cancelable
If this run an be canceled.
Type: bool
- property is_failure
If this run has failed.
Type: bool
- property is_failure_or_canceled
If this run has either failed or was canceled.
Type: bool
- property is_finished
If this run has completely finished execution.
Type: bool
- property is_resume_retry
If this run was created from retrying another run from the point of failure.
Type: bool
- property is_success
If this run has successfully finished executing.
Type: bool
- class dagster.DagsterRunStatus
The status of run execution.
- class dagster.RunsFilter
Defines a filter across job runs, for use when querying storage directly.
Each field of the RunsFilter represents a logical AND with each other. For example, if you specify job_name and tags, then you will receive only runs with the specified job_name AND the specified tags. If left blank, then all values will be permitted for that field.
Parameters:
- run_ids (Optional[List[str]]) – A list of job run_id values.
- job_name (Optional[str]) – Name of the job to query for. If blank, all job_names will be accepted.
- statuses (Optional[List[DagsterRunStatusDagsterRunStatus]]) – A list of run statuses to filter by. If blank, all run statuses will be allowed.
- tags (Optional[Dict[str, Union[str, List[str]]]]) – A dictionary of run tags to query by. All tags specified here must be present for a given run to pass the filter.
- snapshot_id (Optional[str]) – The ID of the job snapshot to query for. Intended for internal use.
- updated_after (Optional[DateTime]) – Filter by runs that were last updated before this datetime.
- created_before (Optional[DateTime]) – Filter by runs that were created before this datetime.
- exclude_subruns (Optional[bool]) – If true, runs that were launched to backfill historical data will be excluded from results.
- class dagster._core.storage.runs.RunStorage
Abstract base class for storing pipeline run history.
Note that run storages using SQL databases as backing stores should implement SqlRunStorage
SqlRunStorage
.Users should not directly instantiate concrete subclasses of this class; they are instantiated by internal machinery when
dagster-webserver
anddagster-graphql
load, based on the values in thedagster.yaml
file in$DAGSTER_HOME
. Configuration of concrete subclasses of this class should be done by setting values in that file.
- class dagster._core.storage.runs.SqlRunStorage
Base class for SQL based run storages.
- class dagster._core.storage.runs.SqliteRunStorage
SQLite-backed run storage.
Users should not directly instantiate this class; it is instantiated by internal machinery when
dagster-webserver
anddagster-graphql
load, based on the values in thedagster.yaml
file in$DAGSTER_HOME
. Configuration of this class should be done by setting values in that file.This is the default run storage when none is specified in the
dagster.yaml
.To explicitly specify SQLite for run storage, you can add a block such as the following to your
dagster.yaml
:run_storage:
module: dagster._core.storage.runs
class: SqliteRunStorage
config:
base_dir: /path/to/dirThe
base_dir
param tells the run storage where on disk to store the database.
- class dagster._core.storage.dagster_run.RunRecord
Internal representation of a run record, as stored in a RunStorage
RunStorage
.Users should not invoke this class directly.
See also: dagster_postgres.PostgresRunStoragedagster_postgres.PostgresRunStorage
and dagster_mysql.MySQLRunStoragedagster_mysql.MySQLRunStorage
.
Event log storage
- class dagster.EventLogEntry
Entries in the event log.
Users should not instantiate this object directly. These entries may originate from the logging machinery (DagsterLogManager/context.log), from framework events (e.g. EngineEvent), or they may correspond to events yielded by user code (e.g. Output).
Parameters:
- error_info (Optional[SerializableErrorInfo]) – Error info for an associated exception, if
- level (Union[str, int]) – The Python log level at which to log this event. Note that
- user_message (str) – For log messages, this is the user-generated message.
- run_id (str) – The id of the run which generated this event.
- timestamp (float) – The Unix timestamp of this event.
- step_key (Optional[str]) – The step key for the step which generated this event. Some events
- job_name (Optional[str]) – The job which generated this event. Some events are
- dagster_event (Optional[DagsterEventDagsterEvent]) – For framework and user events, the associated
- get_dagster_event
DagsterEvent: Returns the DagsterEvent contained within this entry. If this entry does not contain a DagsterEvent, an error will be raised.
- property dagster_event_type
The type of the DagsterEvent contained by this entry, if any.
Type: Optional[DagsterEventType]
- property is_dagster_event
If this entry contains a DagsterEvent.
Type: bool
- property message
Return the message from the structured DagsterEvent if present, fallback to user_message.
- class dagster.EventLogRecord
Internal representation of an event record, as stored in a EventLogStorage
EventLogStorage
.Users should not instantiate this class directly.
- class dagster.EventRecordsFilter
Defines a set of filter fields for fetching a set of event log entries or event log records.
Parameters:
- event_type (DagsterEventTypeDagsterEventType) – Filter argument for dagster event type
- asset_key (Optional[AssetKeyAssetKey]) – Asset key for which to get asset materialization event
- asset_partitions (Optional[List[str]]) – Filter parameter such that only asset
- after_cursor (Optional[EventCursor]) – Filter parameter such that only
- before_cursor (Optional[EventCursor]) – Filter parameter such that
- after_timestamp (Optional[float]) – Filter parameter such that only event records for
- before_timestamp (Optional[float]) – Filter parameter such that only event records for
- class dagster.RunShardedEventsCursor
Pairs an id-based event log cursor with a timestamp-based run cursor, for improved performance on run-sharded event log storages (e.g. the default SqliteEventLogStorage). For run-sharded storages, the id field is ignored, since they may not be unique across shards.
- class dagster._core.storage.event_log.EventLogStorage
Abstract base class for storing structured event logs from pipeline runs.
Note that event log storages using SQL databases as backing stores should implement SqlEventLogStorage
SqlEventLogStorage
.Users should not directly instantiate concrete subclasses of this class; they are instantiated by internal machinery when
dagster-webserver
anddagster-graphql
load, based on the values in thedagster.yaml
file in$DAGSTER_HOME
. Configuration of concrete subclasses of this class should be done by setting values in that file.
- class dagster._core.storage.event_log.SqlEventLogStorage
Base class for SQL backed event log storages.
Distinguishes between run-based connections and index connections in order to support run-level sharding, while maintaining the ability to do cross-run queries
- class dagster._core.storage.event_log.SqliteEventLogStorage
SQLite-backed event log storage.
Users should not directly instantiate this class; it is instantiated by internal machinery when
dagster-webserver
anddagster-graphql
load, based on the values in thedagster.yaml
file insqliteve$DAGSTER_HOME
. Configuration of this class should be done by setting values in that file.This is the default event log storage when none is specified in the
dagster.yaml
.To explicitly specify SQLite for event log storage, you can add a block such as the following to your
dagster.yaml
:event_log_storage:
module: dagster._core.storage.event_log
class: SqliteEventLogStorage
config:
base_dir: /path/to/dirThe
base_dir
param tells the event log storage where on disk to store the databases. To improve concurrent performance, event logs are stored in a separate SQLite database for each run.
- class dagster._core.storage.event_log.ConsolidatedSqliteEventLogStorage
SQLite-backed consolidated event log storage intended for test cases only.
Users should not directly instantiate this class; it is instantiated by internal machinery when
dagster-webserver
anddagster-graphql
load, based on the values in thedagster.yaml
file in$DAGSTER_HOME
. Configuration of this class should be done by setting values in that file.To explicitly specify the consolidated SQLite for event log storage, you can add a block such as the following to your
dagster.yaml
:run_storage:
module: dagster._core.storage.event_log
class: ConsolidatedSqliteEventLogStorage
config:
base_dir: /path/to/dirThe
base_dir
param tells the event log storage where on disk to store the database.
- class dagster._core.storage.event_log.AssetRecord
Internal representation of an asset record, as stored in a EventLogStorage
EventLogStorage
.Users should not invoke this class directly.
See also: dagster_postgres.PostgresEventLogStoragedagster_postgres.PostgresEventLogStorage
and dagster_mysql.MySQLEventLogStoragedagster_mysql.MySQLEventLogStorage
.
Compute log manager
- class dagster._core.storage.compute_log_manager.ComputeLogManager
Abstract base class for capturing the unstructured logs (stdout/stderr) in the current process, stored / retrieved with a provided log_key.
- class dagster._core.storage.local_compute_log_manager.LocalComputeLogManager
Stores copies of stdout & stderr for each compute step locally on disk.
- class dagster._core.storage.noop_compute_log_manager.NoOpComputeLogManager
When enabled for a Dagster instance, stdout and stderr will not be available for any step.
See also: dagster_aws.S3ComputeLogManager
.
Run launcher
- class dagster._core.launcher.DefaultRunLauncher
Launches runs against running GRPC servers.
Run coordinator
- class dagster._core.run_coordinator.DefaultRunCoordinator
Immediately send runs to the run launcher.
- dagster._core.run_coordinator.QueuedRunCoordinator RunCoordinator
Enqueues runs via the run storage, to be deqeueued by the Dagster Daemon process. Requires the Dagster Daemon process to be alive in order for runs to be launched.
Scheduling
- class dagster._core.scheduler.Scheduler
Abstract base class for a scheduler. This component is responsible for interfacing with an external system such as cron to ensure scheduled repeated execution according.
- class dagster._core.storage.schedules.ScheduleStorage
Abstract class for managing persistance of scheduler artifacts.
- class dagster._core.storage.schedules.SqlScheduleStorage
Base class for SQL backed schedule storage.
- class dagster._core.storage.schedules.SqliteScheduleStorage
Local SQLite backed schedule storage.
see also: dagster_postgres.PostgresScheduleStoragedagster_postgres.PostgresScheduleStorage
and dagster_mysql.MySQLScheduleStoragedagster_mysql.MySQLScheduleStorage
.
Exception handling
- dagster._core.errors.user_code_error_boundary
Wraps the execution of user-space code in an error boundary. This places a uniform policy around any user code invoked by the framework. This ensures that all user errors are wrapped in an exception derived from DagsterUserCodeExecutionError, and that the original stack trace of the user error is preserved, so that it can be reported without confusing framework code in the stack trace, if a tool author wishes to do so.
Examples: .. code-block:: python
with user_code_error_boundary(
Pass a class that inherits from DagsterUserCodeExecutionError
DagsterExecutionStepExecutionError,
Pass a function that produces a message
“Error occurred during step execution”
): call_user_provided_function()
Step Launchers (Superseded)
Learn how to migrate from Step Launchers to Dagster Pipes in the migration guide.
- class dagster.StepLauncher
- superseded
This API has been superseded and it's usage is discouraged. Consider using Dagster Pipes instead. Learn more here: https://docs.dagster.io/concepts/dagster-pipes.
A StepLauncher is responsible for executing steps, either in-process or in an external process.
- class dagster.StepRunRef
A serializable object that specifies what’s needed to hydrate a step so that it can be executed in a process outside the plan process.
Users should not instantiate this class directly.
- class dagster.StepExecutionContext
Context for the execution of a step. Users should not instantiate this class directly.
This context assumes that user code can be run directly, and thus includes resource and information.