Definitions
- class dagster.Definitions
A set of definitions explicitly available and loadable by Dagster tools.
Parameters:
- assets (Optional[Iterable[Union[AssetsDefinition, SourceAsset, CacheableAssetsDefinition]]]) – A list of assets. Assets can be created by annotating
- asset_checks (Optional[Iterable[AssetChecksDefinition]]) – A list of asset checks.
- schedules (Optional[Iterable[Union[ScheduleDefinition, UnresolvedPartitionedAssetScheduleDefinition]]]) – List of schedules.
- sensors (Optional[Iterable[SensorDefinition]]) – List of sensors, typically created with
@sensor
. - jobs (Optional[Iterable[Union[JobDefinition, UnresolvedAssetJobDefinition]]]) – List of jobs. Typically created with
define_asset_job
- resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets.
- executor (Optional[Union[ExecutorDefinition, Executor]]) – Default executor for jobs. Individual jobs can override this and define their own executors
- loggers (Optional[Mapping[str, LoggerDefinition]) – Default loggers for jobs. Individual jobs
- metadata (Optional[MetadataMapping]) – Arbitrary metadata for the Definitions. Not displayed in the UI but accessible on
Example usage:
defs = Definitions(
assets=[asset_one, asset_two],
schedules=[a_schedule],
sensors=[a_sensor],
jobs=[a_job],
resources=\{
"a_resource": some_resource,
},
asset_checks=[asset_one_check_one]
)Dagster separates user-defined code from system tools such the web server and the daemon. Rather than loading code directly into process, a tool such as the webserver interacts with user-defined code over a serialization boundary.
These tools must be able to locate and load this code when they start. Via CLI arguments or config, they specify a Python module to inspect.
A Python module is loadable by Dagster tools if there is a top-level variable that is an instance of
Definitions
.- get_all_asset_specs
- experimental
This API may break in future versions, even between dot releases.
Returns an AssetSpec object for every asset contained inside the Definitions object.
- get_asset_value_loader
Returns an object that can load the contents of assets as Python objects.
Invokes load_input on the
IOManager
associated with the assets. Avoids spinning up resources separately for each asset.Usage:
with defs.get_asset_value_loader() as loader:
asset1 = loader.load_asset_value("asset1")
asset2 = loader.load_asset_value("asset2")
- get_job_def
Get a job definition by name. If you passed in a an
UnresolvedAssetJobDefinition
(return value ofdefine_asset_job()
) it will be resolved to aJobDefinition
when returned from this function, with all resource dependencies fully resolved.
- get_schedule_def
Get a
ScheduleDefinition
by name. If your passed-in schedule had resource dependencies, or the job targeted by the schedule had resource dependencies, those resource dependencies will be fully resolved on the returned object.
- get_sensor_def
Get a
SensorDefinition
by name. If your passed-in sensor had resource dependencies, or the job targeted by the sensor had resource dependencies, those resource dependencies will be fully resolved on the returned object.
- load_asset_value
Load the contents of an asset as a Python object.
Invokes load_input on the
IOManager
associated with the asset.If you want to load the values of multiple assets, it’s more efficient to use
get_asset_value_loader()
, which avoids spinning up resources separately for each asset.Parameters:
- asset_key (Union[AssetKey, Sequence[str], str]) – The key of the asset to load.
- python_type (Optional[Type]) – The python type to load the asset as. This is what will
- partition_key (Optional[str]) – The partition of the asset to load.
- metadata (Optional[Dict[str, Any]]) – Input metadata to pass to the
IOManager
Returns: The contents of an asset as a Python object.
- static merge
- experimental
This API may break in future versions, even between dot releases.
Merges multiple Definitions objects into a single Definitions object.
The returned Definitions object has the union of all the definitions in the input Definitions objects.
Raises an error if the Definitions objects to be merged contain conflicting values for the same resource key or logger key, or if they have different executors defined.
Examples:
import submodule1
import submodule2
defs = Definitions.merge(submodule1.defs, submodule2.defs)Returns: The merged definitions.Return type: Definitions
- static validate_loadable
Validates that the enclosed definitions will be loadable by Dagster:
- No assets have conflicting keys.
- No jobs, sensors, or schedules have conflicting names.
- All asset jobs can be resolved.
- All resource requirements are satisfied.
Meant to be used in unit tests.
Raises an error if any of the above are not true.
- dagster.create_repository_using_definitions_args
- experimental
This API may break in future versions, even between dot releases.
Create a named repository using the same arguments as
Definitions
. In older versions of Dagster, repositories were the mechanism for organizing assets, schedules, sensors, and jobs. There could be many repositories per code location. This was a complicated ontology but gave users a way to organize code locations that contained large numbers of heterogenous definitions.As a stopgap for those who both want to 1) use the new
Definitions
API and 2) but still want multiple logical groups of assets in the same code location, we have introduced this function.Example usage:
named_repo = create_repository_using_definitions_args(
name="a_repo",
assets=[asset_one, asset_two],
schedules=[a_schedule],
sensors=[a_sensor],
jobs=[a_job],
resources=\{
"a_resource": some_resource,
}
)
- dagster.load_definitions_from_current_module
- experimental
This API may break in future versions, even between dot releases.
Constructs the
dagster.Definitions
from the module where this function is called.Parameters:
- resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded
dagster.Definitions
. - loggers (Optional[Mapping[str, LoggerDefinition]]) – Default loggers for jobs in the loaded
dagster.Definitions
. Individual jobs - executor (Optional[Union[Executor, ExecutorDefinition]]) – Default executor for jobs in the loaded
dagster.Definitions
. Individual jobs
Returns: The
dagster.Definitions
defined in the current module.Return type: Definitions - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded
- dagster.load_definitions_from_module
- experimental
This API may break in future versions, even between dot releases.
Constructs the
dagster.Definitions
from the given module.Parameters:
- module (ModuleType) – The Python module to look for
dagster.Definitions
inside. - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded
dagster.Definitions
. - loggers (Optional[Mapping[str, LoggerDefinition]]) – Default loggers for jobs in the loaded
dagster.Definitions
. Individual jobs - executor (Optional[Union[Executor, ExecutorDefinition]]) – Default executor for jobs in the loaded
dagster.Definitions
. Individual jobs
Returns: The
dagster.Definitions
defined in the given module.Return type: Definitions - module (ModuleType) – The Python module to look for
- dagster.load_definitions_from_modules
- experimental
This API may break in future versions, even between dot releases.
Constructs the
dagster.Definitions
from the given modules.Parameters:
- modules (Iterable[ModuleType]) – The Python modules to look for
dagster.Definitions
inside. - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded
dagster.Definitions
. - loggers (Optional[Mapping[str, LoggerDefinition]]) – Default loggers for jobs in the loaded
dagster.Definitions
. Individual jobs - executor (Optional[Union[Executor, ExecutorDefinition]]) – Default executor for jobs in the loaded
dagster.Definitions
. Individual jobs
Returns: The
dagster.Definitions
defined in the given modules.Return type: Definitions - modules (Iterable[ModuleType]) – The Python modules to look for
- dagster.load_definitions_from_package_module
- experimental
This API may break in future versions, even between dot releases.
Constructs the
dagster.Definitions
from the given package module.Parameters:
- package_module (ModuleType) – The package module to look for
dagster.Definitions
inside. - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded
dagster.Definitions
. - loggers (Optional[Mapping[str, LoggerDefinition]]) – Default loggers for jobs in the loaded
dagster.Definitions
. Individual jobs - executor (Optional[Union[Executor, ExecutorDefinition]]) – Default executor for jobs in the loaded
dagster.Definitions
. Individual jobs
Returns: The
dagster.Definitions
defined in the given package module.Return type: Definitions - package_module (ModuleType) – The package module to look for
- dagster.load_definitions_from_package_name
- experimental
This API may break in future versions, even between dot releases.
Constructs the
dagster.Definitions
from the package module for the given package name.Parameters:
- package_name (str) – The name of the package module to look for
dagster.Definitions
inside. - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded
dagster.Definitions
. - loggers (Optional[Mapping[str, LoggerDefinition]]) – Default loggers for jobs in the loaded
dagster.Definitions
. Individual jobs - executor (Optional[Union[Executor, ExecutorDefinition]]) – Default executor for jobs in the loaded
dagster.Definitions
. Individual jobs
Returns: The
dagster.Definitions
defined in the package module for the given package name.Return type: Definitions - package_name (str) – The name of the package module to look for