diff --git a/docs/docs-beta/.gitignore b/docs/docs-beta/.gitignore
index 1e7302174ead6..a7c3a9cf904b4 100644
--- a/docs/docs-beta/.gitignore
+++ b/docs/docs-beta/.gitignore
@@ -4,6 +4,7 @@
# Production
/build
!/docs/guides/build
+!/versioned_docs/*/guides/build
!/static/images/guides/build
# Generated files
diff --git a/docs/docs-beta/docs/api/python-api/asset-checks.mdx b/docs/docs-beta/docs/api/python-api/asset-checks.mdx
new file mode 100644
index 0000000000000..c2f34459a33b6
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/asset-checks.mdx
@@ -0,0 +1,603 @@
+---
+title: 'asset checks'
+title_meta: 'asset checks API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'asset checks Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Asset Checks
+
+Dagster allows you to define and execute checks on your software-defined assets. Each asset check verifies some property of a data asset, e.g. that is has no null values in a particular column.
+
+
+
@dagster.asset_check
+
+
+ Create a definition for how to execute an asset check.
+
+ Parameters:
+ - asset (Union[[*AssetKey*](assets.mdx#dagster.AssetKey), Sequence[str], str, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]) – The
+ - name (Optional[str]) – The name of the check. If not specified, the name of the decorated
+ - description (Optional[str]) – The description of the check.
+ - blocking (bool) – When enabled, runs that include this check and any downstream assets that
+ - additional_ins (Optional[Mapping[str, [*AssetIn*](assets.mdx#dagster.AssetIn)]]) – A mapping from input name to
+ - additional_deps (Optional[Iterable[CoercibleToAssetDep]]) – Assets that are upstream
+ - required_resource_keys (Optional[Set[str]]) – A set of keys for resources that are required
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The configuration schema for the check’s underlying
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that executes the check.
+ - compute_kind (Optional[str]) – A string to represent the kind of computation that executes
+ - retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that executes the check.
+ - metadata (Optional[Mapping[str, Any]]) – A dictionary of static metadata for the check.
+ - automation_condition (Optional[[*AutomationCondition*](assets.mdx#dagster.AutomationCondition)]) – An AutomationCondition which determines
+
+
+ Produces an [`AssetChecksDefinition`](#dagster.AssetChecksDefinition) object.
+
+ Example:
+
+ ```python
+ from dagster import asset, asset_check, AssetCheckResult
+
+ @asset
+ def my_asset() -> None:
+ ...
+
+ @asset_check(asset=my_asset, description="Check that my asset has enough rows")
+ def my_asset_has_enough_rows() -> AssetCheckResult:
+ num_rows = ...
+ return AssetCheckResult(passed=num_rows > 5, metadata={"num_rows": num_rows})
+ ```
+ Example with a DataFrame Output:
+ ```python
+ from dagster import asset, asset_check, AssetCheckResult
+ from pandas import DataFrame
+
+ @asset
+ def my_asset() -> DataFrame:
+ ...
+
+ @asset_check(asset=my_asset, description="Check that my asset has enough rows")
+ def my_asset_has_enough_rows(my_asset: DataFrame) -> AssetCheckResult:
+ num_rows = my_asset.shape[0]
+ return AssetCheckResult(passed=num_rows > 5, metadata={"num_rows": num_rows})
+ ```
+
+
+
+
+
+
class dagster.AssetCheckResult
+
+
+ The result of an asset check.
+
+
+
asset_key
+
+
+ The asset key that was checked.
+
+ Type: Optional[[AssetKey](assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
check_name
+
+
+ The name of the check.
+
+ Type: Optional[str]
+
+
+
+
+
+
passed
+
+
+ The pass/fail result of the check.
+
+ Type: bool
+
+
+
+
+
+
metadata
+
+
+ Arbitrary metadata about the asset. Keys are displayed string labels, and values are
+ one of the following: string, float, int, JSON-serializable dict, JSON-serializable
+ list, and one of the data classes returned by a MetadataValue static method.
+
+ Type: Optional[Dict[str, RawMetadataValue]]
+
+
+
+
+
+
severity
+
+
+ Severity of the check. Defaults to ERROR.
+
+ Type: [AssetCheckSeverity](#dagster.AssetCheckSeverity)
+
+
+
+
+
+
description
+
+
+ A text description of the result of the check evaluation.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.AssetCheckSpec
+
+
+ Defines information about an asset check, except how to execute it.
+
+ AssetCheckSpec is often used as an argument to decorators that decorator a function that can
+ execute multiple checks - e.g. @asset, and @multi_asset. It defines one of the checks that
+ will be executed inside that function.
+
+ Parameters:
+ - name (str) – Name of the check.
+ - asset (Union[[*AssetKey*](assets.mdx#dagster.AssetKey), Sequence[str], str, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]) – The asset that
+ - description (Optional[str]) – Description for the check.
+ - additional_deps (Optional[Iterable[[*AssetDep*](assets.mdx#dagster.AssetDep)]]) – Additional dependencies for the check. The
+ - metadata (Optional[Mapping[str, Any]]) – A dict of static metadata for this asset check.
+
+
+
+
+
+
+
+
class dagster.AssetCheckSeverity
+
+
+ Severity level for an AssetCheckResult.
+
+ - WARN: a potential issue with the asset
+ - ERROR: a definite issue with the asset
+
+
+ Severity does not impact execution of the asset or downstream assets.
+
+
+
+
+
+
+
class dagster.AssetCheckKey
+
+ Check names are expected to be unique per-asset. Thus, this combination of asset key and
+ check name uniquely identifies an asset check within a deployment.
+
+
+
+
+
@dagster.multi_asset_check
+
+
+ Defines a set of asset checks that can be executed together with the same op.
+
+ Parameters:
+ - specs (Sequence[[*AssetCheckSpec*](#dagster.AssetCheckSpec)]) – Specs for the asset checks.
+ - name (Optional[str]) – The name of the op. If not specified, the name of the decorated
+ - description (Optional[str]) – Description of the op.
+ - required_resource_keys (Optional[Set[str]]) – A set of keys for resources that are required
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The configuration schema for the asset checks’ underlying
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that executes the checks.
+ - compute_kind (Optional[str]) – A string to represent the kind of computation that executes
+ - retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that executes the checks.
+ - can_subset (bool) – Whether the op can emit results for a subset of the asset checks
+ - ins (Optional[Mapping[str, Union[[*AssetKey*](assets.mdx#dagster.AssetKey), [*AssetIn*](assets.mdx#dagster.AssetIn)]]]) – A mapping from input name to AssetIn depended upon by
+
+
+ Examples:
+
+ ```python
+ @multi_asset_check(
+ specs=[
+ AssetCheckSpec("enough_rows", asset="asset1"),
+ AssetCheckSpec("no_dupes", asset="asset1"),
+ AssetCheckSpec("enough_rows", asset="asset2"),
+ ],
+ )
+ def checks():
+ yield AssetCheckResult(passed=True, asset_key="asset1", check_name="enough_rows")
+ yield AssetCheckResult(passed=False, asset_key="asset1", check_name="no_dupes")
+ yield AssetCheckResult(passed=True, asset_key="asset2", check_name="enough_rows")
+ ```
+
+
+
+
+
+
dagster.load_asset_checks_from_modules
+
+
+ Constructs a list of asset checks from the given modules. This is most often used in
+ conjunction with a call to load_assets_from_modules.
+
+ Parameters:
+ - modules (Iterable[ModuleType]) – The Python modules to look for checks inside.
+ - asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the
+
+
+ Returns: A list containing asset checks defined in the given modules.Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
dagster.load_asset_checks_from_current_module
+
+
+ Constructs a list of asset checks from the module where this function is called. This is most
+ often used in conjunction with a call to load_assets_from_current_module.
+
+ Parameters: asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the
+ key_prefix argument to load_assets_from_current_module.Returns: A list containing asset checks defined in the current module.Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
dagster.load_asset_checks_from_package_module
+
+
+ Constructs a list of asset checks from all sub-modules of the given package module. This is
+ most often used in conjunction with a call to load_assets_from_package_module.
+
+ Parameters:
+ - package_module (ModuleType) – The Python module to look for checks inside.
+ - asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the
+
+
+ Returns: A list containing asset checks defined in the package.Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
dagster.load_asset_checks_from_package_name
+
+
+ Constructs a list of asset checks from all sub-modules of the given package. This is most
+ often used in conjunction with a call to load_assets_from_package_name.
+
+ Parameters:
+ - package_name (str) – The name of the Python package to look for checks inside.
+ - asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the
+
+
+ Returns: A list containing asset checks defined in the package.Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
class dagster.AssetChecksDefinition
+
+
+ Defines a set of checks that are produced by the same op or op graph.
+
+ AssetChecksDefinition should not be instantiated directly, but rather produced using the @asset_check decorator or AssetChecksDefinition.create method.
+
+
+
+
+
+
+
dagster.build_last_update_freshness_checks
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs an AssetChecksDefinition that checks the freshness of the provided assets.
+
+ This check passes if the asset is found to be “fresh”, and fails if the asset is found to be
+ “overdue”. An asset is considered fresh if a record (i.e. a materialization or observation)
+ exists with a timestamp greater than the “lower bound” derived from the parameters of this
+ function.
+
+ deadline_cron is a cron schedule that defines the deadline for when we should expect the asset
+ to arrive by; if not provided, we consider the deadline to be the execution time of the check.
+ lower_bound_delta is a timedelta that defines the lower bound for when a record could have
+ arrived by. If the most recent recent record’s timestamp is earlier than
+ deadline-lower_bound_delta, the asset is considered overdue.
+
+ Let’s use two examples, one with a deadline_cron set and one without.
+ Let’s say I have an asset which runs on a schedule every day at 8:00 AM UTC, and usually takes
+ around 45 minutes to complete. To account for operational delays, I would expect the asset to be
+ done materializing every day by 9:00 AM UTC. I would set the deadline_cron to “0 9 * * *”, and
+ the lower_bound_delta to “45 minutes”. This would mean that starting at 9:00 AM, this check
+ will expect a materialization record to have been created no earlier than 8:15 AM. Note that if
+ the check runs at 8:59 AM, the deadline has not yet passed, and we’ll instead be checking for
+ the most recently passed deadline, which is yesterday.
+ Let’s say I have an observable source asset on a data source which I expect should never be more
+ than 3 hours out of date. In this case, there’s no fixed schedule for when the data should be
+ updated, so I would not provide a deadline_cron. Instead, I would set the lower_bound_delta
+ parameter to “3 hours”. This would mean that the check will expect the most recent observation
+ record to indicate data no older than 3 hours, relative to the current time, regardless of when it runs.
+
+ The check result will contain the following metadata:
+ “dagster/freshness_params”: A dictionary containing the parameters used to construct the
+ check
+ “dagster/last_updated_time”: The time of the most recent update to the asset
+ “dagster/overdue_seconds”: (Only present if asset is overdue) The number of seconds that the
+ asset is overdue by.
+ “dagster/overdue_deadline_timestamp”: The timestamp that we are expecting the asset to have
+ arrived by. In the case of a provided deadline_cron, this is the timestamp of the most recent
+ tick of the cron schedule. In the case of no deadline_cron, this is the current time.
+
+ Examples:
+
+ ```python
+ # Example 1: Assets that are expected to be updated every day within 45 minutes of
+ # 9:00 AM UTC
+ from dagster import build_last_update_freshness_checks, AssetKey
+ from .somewhere import my_daily_scheduled_assets_def
+
+ checks_def = build_last_update_freshness_checks(
+ [my_daily_scheduled_assets_def, AssetKey("my_other_daily_asset_key")],
+ lower_bound_delta=datetime.timedelta(minutes=45),
+ deadline_cron="0 9 * * *",
+ )
+
+ # Example 2: Assets that are expected to be updated within 3 hours of the current time
+ from dagster import build_last_update_freshness_checks, AssetKey
+ from .somewhere import my_observable_source_asset
+
+ checks_def = build_last_update_freshness_checks(
+ [my_observable_source_asset, AssetKey("my_other_observable_asset_key")],
+ lower_bound_delta=datetime.timedelta(hours=3),
+ )
+ ```
+ Parameters:
+ - assets (Sequence[Union[CoercibleToAssetKey, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]) – The assets to
+ - lower_bound_delta (datetime.timedelta) – The check will pass if the asset was updated within
+ - deadline_cron (Optional[str]) – Defines the deadline for when we should start checking
+ - timezone (Optional[str]) – The timezone to use when calculating freshness and deadline. If
+
+
+ Returns:
+ AssetChecksDefinition objects which execute freshness checks
+ for the provided assets.
+
+ Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
dagster.build_time_partition_freshness_checks
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Construct an AssetChecksDefinition that checks the freshness of the provided assets.
+
+ This check passes if the asset is considered “fresh” by the time that execution begins. We
+ consider an asset to be “fresh” if there exists a record for the most recent partition, once
+ the deadline has passed.
+
+ deadline_cron is a cron schedule that defines the deadline for when we should expect the most
+ recent partition to arrive by. Once a tick of the cron schedule has passed, this check will fail
+ if the most recent partition has not been observed/materialized.
+
+ Let’s say I have a daily-partitioned asset which runs every day at 8:00 AM UTC, and takes around
+ 45 minutes to complete. To account for operational delays, I would expect the asset to be done
+ materializing every day by 9:00 AM UTC. I would set the deadline_cron to “0 9 * * *”. This
+ means that starting at 9:00 AM, this check will expect a record to exist for the previous day’s
+ partition. Note that if the check runs at 8:59 AM, the deadline has not yet passed, and we’ll
+ instead be checking for the most recently passed deadline, which is yesterday (meaning the
+ partition representing the day before yesterday).
+
+ The timestamp of an observation record is the timestamp indicated by the
+ “dagster/last_updated_timestamp” metadata key. The timestamp of a materialization record is the
+ timestamp at which that record was created.
+
+ The check will fail at runtime if a non-time-window partitioned asset is passed in.
+
+ The check result will contain the following metadata:
+ “dagster/freshness_params”: A dictionary containing the parameters used to construct the
+ check.
+ “dagster/last_updated_time”: (Only present if the asset has been observed/materialized before)
+ The time of the most recent update to the asset.
+ “dagster/overdue_seconds”: (Only present if asset is overdue) The number of seconds that the
+ asset is overdue by.
+ “dagster/overdue_deadline_timestamp”: The timestamp that we are expecting the asset to have
+ arrived by. This is the timestamp of the most recent tick of the cron schedule.
+
+ Examples:
+
+ ```python
+ from dagster import build_time_partition_freshness_checks, AssetKey
+ # A daily partitioned asset that is expected to be updated every day within 45 minutes
+ # of 9:00 AM UTC
+ from .somewhere import my_daily_scheduled_assets_def
+
+ checks_def = build_time_partition_freshness_checks(
+ [my_daily_scheduled_assets_def],
+ deadline_cron="0 9 * * *",
+ )
+ ```
+ Parameters:
+ - assets (Sequence[Union[CoercibleToAssetKey, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]) – The assets to
+ - deadline_cron (str) – The check will pass if the partition time window most recently
+ - timezone (Optional[str]) – The timezone to use when calculating freshness and deadline. If
+
+
+ Returns:
+ AssetChecksDefinition objects which execute freshness
+ checks for the provided assets.
+
+ Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
dagster.build_sensor_for_freshness_checks
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Builds a sensor which kicks off evaluation of freshness checks.
+
+ This sensor will kick off an execution of a check in the following cases:
+ - The check has never been executed before.
+ - The check has been executed before, and the previous result was a success, but it is again
+ possible for the check to be overdue based on the dagster/fresh_until_timestamp metadata
+ on the check result.
+
+ Note that we will not execute if:
+ - The freshness check has been executed before, and the previous result was a failure. This is
+ because whichever run materializes/observes the run to bring the check back to a passing
+ state will end up also running the check anyway, so until that run occurs, there’s no point
+ in evaluating the check.
+ - The freshness check has been executed before, and the previous result was a success, but it is
+ not possible for the check to be overdue based on the dagster/fresh_until_timestamp
+ metadata on the check result. Since the check cannot be overdue, we know the check
+ result would not change with an additional execution.
+
+ Parameters:
+ - freshness_checks (Sequence[[*AssetChecksDefinition*](#dagster.AssetChecksDefinition)]) – The freshness checks to evaluate.
+ - minimum_interval_seconds (Optional[int]) – The duration in seconds between evaluations of the sensor.
+ - name (Optional[str]) – The name of the sensor. Defaults to “freshness_check_sensor”, but a
+ - default_status (Optional[DefaultSensorStatus]) – The default status of the sensor. Defaults
+
+
+ Returns: The sensor that kicks off freshness evaluations.Return type: [SensorDefinition](schedules-sensors.mdx#dagster.SensorDefinition)
+
+
+
+
+
+
dagster.build_column_schema_change_checks
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns asset checks that pass if the column schema of the asset’s latest materialization
+ is the same as the column schema of the asset’s previous materialization.
+
+ The underlying materializations are expected to have a metadata entry with key dagster/column_schema and type [`TableSchema`](metadata.mdx#dagster.TableSchema).
+ To learn more about how to add column schema metadata and other forms of tabular metadata to assets, see
+ [https://docs.dagster.io/guides/build/assets/metadata-and-tags/table-metadata#attaching-column-schema](https://docs.dagster.io/guides/build/assets/metadata-and-tags/table-metadata#attaching-column-schema).
+
+ The resulting checks will fail if any changes are detected in the column schema between
+ materializations, including:
+ - Added columns
+ - Removed columns
+ - Changes to column types
+
+ The check failure message will detail exactly what changed in the schema.
+
+ Parameters:
+ - assets (Sequence[Union[[*AssetKey*](assets.mdx#dagster.AssetKey), str, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]]) – The assets to create
+ - severity ([*AssetCheckSeverity*](#dagster.AssetCheckSeverity)) – The severity if the check fails. Defaults to WARN.
+
+
+ Returns: Sequence[AssetsChecksDefinition]
+ Examples:
+
+ First, define an asset with column schema metadata. You can attach schema metadata either as
+ definition metadata (when schema is known at definition time) or as materialization metadata
+ (when schema is only known at runtime):
+
+ ```python
+ import dagster as dg
+
+ # Using definition metadata when schema is known upfront
+ @dg.asset
+ def people_table():
+ column_names = ...
+ column_types = ...
+
+ columns = [
+ dg.TableColumn(name, column_type)
+ for name, column_type in zip(column_names, column_types)
+ ]
+
+ yield dg.MaterializeResult(
+ metadata={"dagster/column_schema": dg.TableSchema(columns=columns)}
+ )
+ ```
+ Once you have assets with column schema metadata, you can create schema change checks to monitor
+ for changes in the schema between materializations:
+
+ ```python
+ # Create schema change checks for one or more assets
+ schema_checks = dg.build_column_schema_change_checks(
+ assets=[people_table]
+ )
+ ```
+
+
+
+
+
+
dagster.build_metadata_bounds_checks
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns asset checks that pass if the metadata value of the asset’s latest materialization
+ is within the specified range.
+
+ Parameters:
+ - assets (Sequence[Union[[*AssetKey*](assets.mdx#dagster.AssetKey), str, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]]) – The assets to create
+ - severity ([*AssetCheckSeverity*](#dagster.AssetCheckSeverity)) – The severity if the check fails. Defaults to WARN.
+ - metadata_key (str) – The metadata key to check.
+ - min_value (Optional[Union[int, float]]) – The minimum value to check for. If None, no minimum
+ - max_value (Optional[Union[int, float]]) – The maximum value to check for. If None, no maximum
+ - exclusive_min (bool) – If True, the check will fail if the metadata value is equal to min_value.
+ - exclusive_max (bool) – If True, the check will fail if the metadata value is equal to max_value.
+
+
+ Returns: Sequence[AssetsChecksDefinition]
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/assets.mdx b/docs/docs-beta/docs/api/python-api/assets.mdx
new file mode 100644
index 0000000000000..ca89381c2e102
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/assets.mdx
@@ -0,0 +1,2417 @@
+---
+title: 'assets'
+title_meta: 'assets API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'assets Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Assets
+
+An asset is an object in persistent storage, such as a table, file, or persisted machine learning model. An asset definition is a description, in code, of an asset that should exist and how to produce and update that asset.
+
+
+
+
+## Asset definitions
+
+Refer to the [Asset definitions](https://docs.dagster.io/guides/build/assets/defining-assets) documentation for more information.
+
+
+
@dagster.asset
+
+
+ Create a definition for how to compute an asset.
+
+ A software-defined asset is the combination of:
+ 1. An asset key, e.g. the name of a table.
+ 2. A function, which can be run to compute the contents of the asset.
+ 3. A set of upstream assets that are provided as inputs to the function when computing the asset.
+ Unlike an op, whose dependencies are determined by the graph it lives inside, an asset knows
+ about the upstream assets it depends on. The upstream assets are inferred from the arguments
+ to the decorated function. The name of the argument designates the name of the upstream asset.
+
+ An asset has an op inside it to represent the function that computes it. The name of the op
+ will be the segments of the asset key, separated by double-underscores.
+
+ Parameters:
+ - name (Optional[str]) – The name of the asset. If not provided, defaults to the name of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the
+ - ins (Optional[Mapping[str, [*AssetIn*](#dagster.AssetIn)]]) – A dictionary that maps input names to information
+ - deps (Optional[Sequence[Union[[*AssetDep*](#dagster.AssetDep), [*AssetsDefinition*](#dagster.AssetsDefinition), [*SourceAsset*](#dagster.SourceAsset), [*AssetKey*](#dagster.AssetKey), str]]]) – The assets that are upstream dependencies, but do not correspond to a parameter of the
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The configuration schema for the asset’s underlying
+ - metadata (Optional[Dict[str, Any]]) – A dict of metadata entries for the asset.
+ - tags (Optional[Mapping[str, str]]) – Tags for filtering and organizing. These tags are not
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the op.
+ - io_manager_key (Optional[str]) – The resource key of the IOManager used
+ - io_manager_def (Optional[object]) – experimental
+ - dagster_type (Optional[[*DagsterType*](types.mdx#dagster.DagsterType)]) – Allows specifying type validation functions that
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset.
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided,
+ - resource_defs (Optional[Mapping[str, object]]) – experimental
+ - output_required (bool) – Whether the decorated function will always materialize an asset.
+ - automation_condition ([*AutomationCondition*](#dagster.AutomationCondition)) – A condition describing when Dagster should materialize this asset.
+ - backfill_policy ([*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)) – experimental
+ - retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that computes the asset.
+ - code_version (Optional[str]) – Version of the code that generates this asset. In
+ - check_specs (Optional[Sequence[[*AssetCheckSpec*](asset-checks.mdx#dagster.AssetCheckSpec)]]) – Specs for asset checks that
+ - key (Optional[CoeercibleToAssetKey]) – The key for this asset. If provided, cannot specify key_prefix or name.
+ - owners (Optional[Sequence[str]]) – experimentalteam:,
+ - kinds (Optional[Set[str]]) – A list of strings representing the kinds of the asset. These
+ - pool (Optional[str]) – A string that identifies the concurrency pool that governs this asset’s execution.
+ - non_argument_deps (Optional[Union[Set[[*AssetKey*](#dagster.AssetKey)], Set[str]]]) – deprecateddeps instead.) Deprecated, use deps instead.
+
+
+ Examples:
+
+ ```python
+ @asset
+ def my_upstream_asset() -> int:
+ return 5
+
+ @asset
+ def my_asset(my_upstream_asset: int) -> int:
+ return my_upstream_asset + 1
+
+ should_materialize = True
+
+ @asset(output_required=False)
+ def conditional_asset():
+ if should_materialize:
+ yield Output(5) # you must `yield`, not `return`, the result
+
+ # Will also only materialize if `should_materialize` is `True`
+ @asset
+ def downstream_asset(conditional_asset):
+ return conditional_asset + 1
+ ```
+
+
+
+
+
+
class dagster.MaterializeResult
+
+
+ An object representing a successful materialization of an asset. These can be returned from
+ @asset and @multi_asset decorated functions to pass metadata or specify specific assets were
+ materialized.
+
+
+
asset_key
+
+
+ Optional in @asset, required in @multi_asset to discern which asset this refers to.
+
+ Type: Optional[[AssetKey](#dagster.AssetKey)]
+
+
+
+
+
+
metadata
+
+
+ Metadata to record with the corresponding AssetMaterialization event.
+
+ Type: Optional[RawMetadataMapping]
+
+
+
+
+
+
check_results
+
+
+ Check results to record with the
+ corresponding AssetMaterialization event.
+
+ Type: Optional[Sequence[[AssetCheckResult](asset-checks.mdx#dagster.AssetCheckResult)]]
+
+
+
+
+
+
data_version
+
+
+ The data version of the asset that was observed.
+
+ Type: Optional[DataVersion]
+
+
+
+
+
+
tags
+
+
+ Tags to record with the corresponding
+ AssetMaterialization event.
+
+ Type: Optional[Mapping[str, str]]
+
+
+
+
+
+
+
+
+
+
class dagster.AssetSpec
+
+
+ Specifies the core attributes of an asset, except for the function that materializes or
+ observes it.
+
+ An asset spec plus any materialization or observation function for the asset constitutes an
+ “asset definition”.
+
+
+
key
+
+
+ The unique identifier for this asset.
+
+ Type: [AssetKey](#dagster.AssetKey)
+
+
+
+
+
+
deps
+
+
+ The asset keys for the upstream assets that
+ materializing this asset depends on.
+
+ Type: Optional[AbstractSet[[AssetKey](#dagster.AssetKey)]]
+
+
+
+
+
+
description
+
+
+ Human-readable description of this asset.
+
+ Type: Optional[str]
+
+
+
+
+
+
metadata
+
+
+ A dict of static metadata for this asset.
+ For example, users can provide information about the database table this
+ asset corresponds to.
+
+ Type: Optional[Dict[str, Any]]
+
+
+
+
+
+
skippable
+
+
+ Whether this asset can be omitted during materialization, causing downstream
+ dependencies to skip.
+
+ Type: bool
+
+
+
+
+
+
group_name
+
+
+ A string name used to organize multiple assets into groups. If
+ not provided, the name “default” is used.
+
+ Type: Optional[str]
+
+
+
+
+
+
code_version
+
+
+ The version of the code for this specific asset,
+ overriding the code version of the materialization function
+
+ Type: Optional[str]
+
+
+
+
+
+
backfill_policy
+
+
+ BackfillPolicy to apply to the specified asset.
+
+ Type: Optional[[BackfillPolicy](partitions.mdx#dagster.BackfillPolicy)]
+
+
+
+
+
+
owners
+
+
+ A list of strings representing owners of the asset. Each
+ string can be a user’s email address, or a team name prefixed with team:,
+ e.g. team:finops.
+
+ Type: Optional[Sequence[str]]
+
+
+
+
+
+
tags
+
+
+ Tags for filtering and organizing. These tags are not
+ attached to runs of the asset.
+
+ Type: Optional[Mapping[str, str]]
+
+
+
+
+
+
kinds
+
+ (Optional[Set[str]]): A list of strings representing the kinds of the asset. These
+ will be made visible in the Dagster UI.
+
+
+
+
+
partitions_def
+
+
+ Defines the set of partition keys that
+ compose the asset.
+
+ Type: Optional[[PartitionsDefinition](partitions.mdx#dagster.PartitionsDefinition)]
+
+
+
+
+
+
merge_attributes
+
+
+ Returns a new AssetSpec with the specified attributes merged with the current attributes.
+
+ Parameters:
+ - deps (Optional[Iterable[CoercibleToAssetDep]]) – A set of asset dependencies to add to
+ - metadata (Optional[Mapping[str, Any]]) – A set of metadata to add to the asset self.
+ - owners (Optional[Sequence[str]]) – A set of owners to add to the asset self.
+ - tags (Optional[Mapping[str, str]]) – A set of tags to add to the asset self.
+ - kinds (Optional[Set[str]]) – A set of kinds to add to the asset self.
+
+
+ Returns: AssetSpec
+
+
+
+
+
+
replace_attributes
+
+ Returns a new AssetSpec with the specified attributes replaced.
+
+
+
+
+
with_io_manager_key
+
+
+ Returns a copy of this AssetSpec with an extra metadata value that dictates which I/O
+ manager to use to load the contents of this asset in downstream computations.
+
+ Parameters: io_manager_key (str) – The I/O manager key. This will be used as the value for the
+ “dagster/io_manager_key” metadata key.Returns: AssetSpec
+
+
+
+
+
+
+
+
+
+
class dagster.AssetsDefinition
+
+
+ Defines a set of assets that are produced by the same op or graph.
+
+ AssetsDefinitions are typically not instantiated directly, but rather produced using the
+ [`@asset`](#dagster.asset) or [`@multi_asset`](#dagster.multi_asset) decorators.
+
+
+
static from_graph
+
+
+ Constructs an AssetsDefinition from a GraphDefinition.
+
+ Parameters:
+ - graph_def ([*GraphDefinition*](graphs.mdx#dagster.GraphDefinition)) – The GraphDefinition that is an asset.
+ - keys_by_input_name (Optional[Mapping[str, [*AssetKey*](#dagster.AssetKey)]]) – A mapping of the input
+ - keys_by_output_name (Optional[Mapping[str, [*AssetKey*](#dagster.AssetKey)]]) – A mapping of the output
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, key_prefix will be prepended
+ - internal_asset_deps (Optional[Mapping[str, Set[[*AssetKey*](#dagster.AssetKey)]]]) – By default, it is assumed
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - partition_mappings (Optional[Mapping[str, [*PartitionMapping*](partitions.mdx#dagster.PartitionMapping)]]) – Defines how to map partition
+ - resource_defs (Optional[Mapping[str, [*ResourceDefinition*](resources.mdx#dagster.ResourceDefinition)]]) – experimental
+ - group_name (Optional[str]) – A group name for the constructed asset. Assets without a
+ - group_names_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a group name to be
+ - descriptions_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a description to be
+ - metadata_by_output_name (Optional[Mapping[str, Optional[RawMetadataMapping]]]) – Defines metadata to
+ - tags_by_output_name (Optional[Mapping[str, Optional[Mapping[str, str]]]]) – Defines
+ - freshness_policies_by_output_name (Optional[Mapping[str, Optional[FreshnessPolicy]]]) – Defines a
+ - automation_conditions_by_output_name (Optional[Mapping[str, Optional[[*AutomationCondition*](#dagster.AutomationCondition)]]]) – Defines an
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – Defines this asset’s BackfillPolicy
+ - owners_by_key (Optional[Mapping[[*AssetKey*](#dagster.AssetKey), Sequence[str]]]) – Defines
+
+
+
+
+
+
+
+
static from_op
+
+
+ Constructs an AssetsDefinition from an OpDefinition.
+
+ Parameters:
+ - op_def ([*OpDefinition*](ops.mdx#dagster.OpDefinition)) – The OpDefinition that is an asset.
+ - keys_by_input_name (Optional[Mapping[str, [*AssetKey*](#dagster.AssetKey)]]) – A mapping of the input
+ - keys_by_output_name (Optional[Mapping[str, [*AssetKey*](#dagster.AssetKey)]]) – A mapping of the output
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, key_prefix will be prepended
+ - internal_asset_deps (Optional[Mapping[str, Set[[*AssetKey*](#dagster.AssetKey)]]]) – By default, it is assumed
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - partition_mappings (Optional[Mapping[str, [*PartitionMapping*](partitions.mdx#dagster.PartitionMapping)]]) – Defines how to map partition
+ - group_name (Optional[str]) – A group name for the constructed asset. Assets without a
+ - group_names_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a group name to be
+ - descriptions_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a description to be
+ - metadata_by_output_name (Optional[Mapping[str, Optional[RawMetadataMapping]]]) – Defines metadata to
+ - tags_by_output_name (Optional[Mapping[str, Optional[Mapping[str, str]]]]) – Defines
+ - freshness_policies_by_output_name (Optional[Mapping[str, Optional[FreshnessPolicy]]]) – Defines a
+ - automation_conditions_by_output_name (Optional[Mapping[str, Optional[[*AutomationCondition*](#dagster.AutomationCondition)]]]) – Defines an
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – Defines this asset’s BackfillPolicy
+
+
+
+
+
+
+
+
get_asset_spec
+
+
+ Returns a representation of this asset as an [`AssetSpec`](#dagster.AssetSpec).
+
+ If this is a multi-asset, the “key” argument allows selecting which asset to return the
+ spec for.
+
+ Parameters: key (Optional[[*AssetKey*](#dagster.AssetKey)]) – If this is a multi-asset, select which asset to return its
+ AssetSpec. If not a multi-asset, this can be left as None.Returns: AssetSpec
+
+
+
+
+
+
get_partition_mapping
+
+ Returns the partition mapping between keys in this AssetsDefinition and a given input
+ asset key (if any).
+
+
+
+
+
to_source_asset
+
+
+ Returns a representation of this asset as a [`SourceAsset`](#dagster.SourceAsset).
+
+ If this is a multi-asset, the “key” argument allows selecting which asset to return a
+ SourceAsset representation of.
+
+ Parameters: key (Optional[Union[str, Sequence[str], [*AssetKey*](#dagster.AssetKey)]]]) – If this is a multi-asset, select
+ which asset to return a SourceAsset representation of. If not a multi-asset, this
+ can be left as None.Returns: SourceAsset
+
+
+
+
+
+
to_source_assets
+
+
+ Returns a SourceAsset for each asset in this definition.
+
+ Each produced SourceAsset will have the same key, metadata, io_manager_key, etc. as the
+ corresponding asset
+
+
+
+
+
+
+
property asset_deps
+
+ Maps assets that are produced by this definition to assets that they depend on. The
+ dependencies can be either “internal”, meaning that they refer to other assets that are
+ produced by this definition, or “external”, meaning that they refer to assets that aren’t
+ produced by this definition.
+
+
+
+
+
property can_subset
+
+
+ If True, indicates that this AssetsDefinition may materialize any subset of its
+ asset keys in a given computation (as opposed to being required to materialize all asset
+ keys).
+
+ Type: bool
+
+
+
+
+
+
property check_specs
+
+
+ Returns the asset check specs defined on this AssetsDefinition, i.e. the checks that can
+ be executed while materializing the assets.
+
+ Return type: Iterable[AssetsCheckSpec]
+
+
+
+
+
+
property dependency_keys
+
+
+ The asset keys which are upstream of any asset included in this
+ AssetsDefinition.
+
+ Type: Iterable[[AssetKey](#dagster.AssetKey)]
+
+
+
+
+
+
property descriptions_by_key
+
+
+ Returns a mapping from the asset keys in this AssetsDefinition
+ to the descriptions assigned to them. If there is no assigned description for a given AssetKey,
+ it will not be present in this dictionary.
+
+ Type: Mapping[[AssetKey](#dagster.AssetKey), str]
+
+
+
+
+
+
property group_names_by_key
+
+
+ Returns a mapping from the asset keys in this AssetsDefinition
+ to the group names assigned to them. If there is no assigned group name for a given AssetKey,
+ it will not be present in this dictionary.
+
+ Type: Mapping[[AssetKey](#dagster.AssetKey), str]
+
+
+
+
+
+
property key
+
+
+ The asset key associated with this AssetsDefinition. If this AssetsDefinition
+ has more than one asset key, this will produce an error.
+
+ Type: [AssetKey](#dagster.AssetKey)
+
+
+
+
+
+
property keys
+
+
+ The asset keys associated with this AssetsDefinition.
+
+ Type: AbstractSet[[AssetKey](#dagster.AssetKey)]
+
+
+
+
+
+
property node_def
+
+
+ Returns the OpDefinition or GraphDefinition that is used to materialize
+ the assets in this AssetsDefinition.
+
+ Type: NodeDefinition
+
+
+
+
+
+
property op
+
+
+ Returns the OpDefinition that is used to materialize the assets in this
+ AssetsDefinition.
+
+ Type: [OpDefinition](ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
property partitions_def
+
+
+ The PartitionsDefinition for this AssetsDefinition (if any).
+
+ Type: Optional[[PartitionsDefinition](partitions.mdx#dagster.PartitionsDefinition)]
+
+
+
+
+
+
property required_resource_keys
+
+
+ The set of keys for resources that must be provided to this AssetsDefinition.
+
+ Type: Set[str]
+
+
+
+
+
+
property resource_defs
+
+
+ A mapping from resource name to ResourceDefinition for
+ the resources bound to this AssetsDefinition.
+
+ Type: Mapping[str, [ResourceDefinition](resources.mdx#dagster.ResourceDefinition)]
+
+
+
+
+
+
+
+
+
+
class dagster.AssetKey
+
+
+ Object representing the structure of an asset key. Takes in a sanitized string, list of
+ strings, or tuple of strings.
+
+ Example usage:
+
+ ```python
+ from dagster import AssetKey
+
+ AssetKey("asset1")
+ AssetKey(["asset1"]) # same as the above
+ AssetKey(["prefix", "asset1"])
+ AssetKey(["prefix", "subprefix", "asset1"])
+ ```
+ Parameters: path (Union[str, Sequence[str]]) – String, list of strings, or tuple of strings. A list of
+ strings represent the hierarchical structure of the asset_key.
+
+
property path
+
+
+
+
+
+
+
+
+
+
+
dagster.map_asset_specs
+
+
+ Map a function over a sequence of AssetSpecs or AssetsDefinitions, replacing specs in the sequence
+ or specs in an AssetsDefinitions with the result of the function.
+
+ Parameters:
+ - func (Callable[[[*AssetSpec*](#dagster.AssetSpec)], [*AssetSpec*](#dagster.AssetSpec)]) – The function to apply to each AssetSpec.
+ - iterable (Iterable[Union[[*AssetsDefinition*](#dagster.AssetsDefinition), [*AssetSpec*](#dagster.AssetSpec)]]) – The sequence of AssetSpecs or AssetsDefinitions.
+
+
+ Returns:
+ A sequence of AssetSpecs or AssetsDefinitions with the function applied
+ to each spec.
+
+ Return type: Sequence[Union[[AssetsDefinition](#dagster.AssetsDefinition), [AssetSpec](#dagster.AssetSpec)]]
+ Examples:
+
+ ```python
+ from dagster import AssetSpec, map_asset_specs
+
+ asset_specs = [
+ AssetSpec(key="my_asset"),
+ AssetSpec(key="my_asset_2"),
+ ]
+
+ mapped_specs = map_asset_specs(lambda spec: spec.replace_attributes(owners=["nelson@hooli.com"]), asset_specs)
+ ```
+
+
+
+
+
+
+
+
+
+
+## Graph-backed asset definitions
+
+Refer to the [Graph-backed asset](https://docs.dagster.io/guides/build/assets/defining-assets#graph-asset) documentation for more information.
+
+
+
@dagster.graph_asset
+
+
+ Creates a software-defined asset that’s computed using a graph of ops.
+
+ This decorator is meant to decorate a function that composes a set of ops or graphs to define
+ the dependencies between them.
+
+ Parameters:
+ - name (Optional[str]) – The name of the asset. If not provided, defaults to the name of the
+ - description (Optional[str]) – A human-readable description of the asset.
+ - ins (Optional[Mapping[str, [*AssetIn*](#dagster.AssetIn)]]) – A dictionary that maps input names to information
+ - config (Optional[Union[[*ConfigMapping*](config.mdx#dagster.ConfigMapping)], Mapping[str, Any]) –
+
+ Describes how the graph underlying the asset is configured at runtime.
+
+ If a [`ConfigMapping`](config.mdx#dagster.ConfigMapping) object is provided, then the graph takes on the config
+ schema of this object. The mapping will be applied at runtime to generate the config for
+ the graph’s constituent nodes.
+
+ If a dictionary is provided, then it will be used as the default run config for the
+ graph. This means it must conform to the config schema of the underlying nodes. Note
+ that the values provided will be viewable and editable in the Dagster UI, so be careful
+ with secrets.
+
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. If
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - metadata (Optional[RawMetadataMapping]) – Dictionary of metadata to be associated with
+ - tags (Optional[Mapping[str, str]]) – (Experimental) Tags for filtering and organizing. These tags are not
+ - owners (Optional[Sequence[str]]) – experimentalteam:,
+ - kinds (Optional[Set[str]]) – A list of strings representing the kinds of the asset. These
+ - automation_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – The AutomationCondition to use
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – The BackfillPolicy to use for this asset.
+ - code_version (Optional[str]) – Version of the code that generates this asset. In
+ - key (Optional[CoeercibleToAssetKey]) – The key for this asset. If provided, cannot specify key_prefix or name.
+
+
+ Examples:
+
+ ```python
+ @op
+ def fetch_files_from_slack(context) -> pd.DataFrame:
+ ...
+
+ @op
+ def store_files(files) -> None:
+ files.to_sql(name="slack_files", con=create_db_connection())
+
+ @graph_asset
+ def slack_files_table():
+ return store_files(fetch_files_from_slack())
+ ```
+
+
+
+
+
+
@dagster.graph_multi_asset
+
+
+ Create a combined definition of multiple assets that are computed using the same graph of
+ ops, and the same upstream assets.
+
+ Each argument to the decorated function references an upstream asset that this asset depends on.
+ The name of the argument designates the name of the upstream asset.
+
+ Parameters:
+ - name (Optional[str]) – The name of the graph.
+ - outs – (Optional[Dict[str, AssetOut]]): The AssetOuts representing the produced assets.
+ - ins (Optional[Mapping[str, [*AssetIn*](#dagster.AssetIn)]]) – A dictionary that maps input names to information
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – The backfill policy for the asset.
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. This
+ - can_subset (bool) – Whether this asset’s computation can emit a subset of the asset
+ - config (Optional[Union[[*ConfigMapping*](config.mdx#dagster.ConfigMapping)], Mapping[str, Any]) –
+
+ Describes how the graph underlying the asset is configured at runtime.
+
+ If a [`ConfigMapping`](config.mdx#dagster.ConfigMapping) object is provided, then the graph takes on the config
+ schema of this object. The mapping will be applied at runtime to generate the config for
+ the graph’s constituent nodes.
+
+ If a dictionary is provided, then it will be used as the default run config for the
+ graph. This means it must conform to the config schema of the underlying nodes. Note
+ that the values provided will be viewable and editable in the Dagster UI, so be careful
+ with secrets.
+
+ If no value is provided, then the config schema for the graph is the default (derived
+
+
+
+
+
+
+
+
+
+
+
+
+## Multi-asset definitions
+
+Refer to the [Multi-asset](https://docs.dagster.io/guides/build/assets/defining-assets#multi-asset) documentation for more information.
+
+
+
@dagster.multi_asset
+
+
+ Create a combined definition of multiple assets that are computed using the same op and same
+ upstream assets.
+
+ Each argument to the decorated function references an upstream asset that this asset depends on.
+ The name of the argument designates the name of the upstream asset.
+
+ You can set I/O managers keys, auto-materialize policies, freshness policies, group names, etc.
+ on an individual asset within the multi-asset by attaching them to the [`AssetOut`](#dagster.AssetOut)
+ corresponding to that asset in the outs parameter.
+
+ Parameters:
+ - name (Optional[str]) – The name of the op.
+ - outs – (Optional[Dict[str, AssetOut]]): The AssetOuts representing the assets materialized by
+ - ins (Optional[Mapping[str, [*AssetIn*](#dagster.AssetIn)]]) – A dictionary that maps input names to information
+ - deps (Optional[Sequence[Union[[*AssetsDefinition*](#dagster.AssetsDefinition), [*SourceAsset*](#dagster.SourceAsset), [*AssetKey*](#dagster.AssetKey), str]]]) – The assets that are upstream dependencies, but do not correspond to a parameter of the
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The configuration schema for the asset’s underlying
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the underlying op.
+ - internal_asset_deps (Optional[Mapping[str, Set[[*AssetKey*](#dagster.AssetKey)]]]) – By default, it is assumed
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – The backfill policy for the op that computes the asset.
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset.
+ - can_subset (bool) – If this asset’s computation can emit a subset of the asset
+ - resource_defs (Optional[Mapping[str, object]]) – experimental
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. This
+ - retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that computes the asset.
+ - code_version (Optional[str]) – Version of the code encapsulated by the multi-asset. If set,
+ - specs (Optional[Sequence[[*AssetSpec*](#dagster.AssetSpec)]]) – The specifications for the assets materialized
+ - check_specs (Optional[Sequence[[*AssetCheckSpec*](asset-checks.mdx#dagster.AssetCheckSpec)]]) – Specs for asset checks that
+ - pool (Optional[str]) – A string that identifies the concurrency pool that governs this
+ - non_argument_deps (Optional[Union[Set[[*AssetKey*](#dagster.AssetKey)], Set[str]]]) – deprecateddeps instead.) Deprecated, use deps instead.
+
+
+ Examples:
+
+ ```python
+ @multi_asset(
+ specs=[
+ AssetSpec("asset1", deps=["asset0"]),
+ AssetSpec("asset2", deps=["asset0"]),
+ ]
+ )
+ def my_function():
+ asset0_value = load(path="asset0")
+ asset1_result, asset2_result = do_some_transformation(asset0_value)
+ write(asset1_result, path="asset1")
+ write(asset2_result, path="asset2")
+
+ # Or use IO managers to handle I/O:
+ @multi_asset(
+ outs={
+ "asset1": AssetOut(),
+ "asset2": AssetOut(),
+ }
+ )
+ def my_function(asset0):
+ asset1_value = do_some_transformation(asset0)
+ asset2_value = do_some_other_transformation(asset0)
+ return asset1_value, asset2_value
+ ```
+
+
+
+
+
+
@dagster.multi_observable_source_asset
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Defines a set of assets that can be observed together with the same function.
+
+ Parameters:
+ - name (Optional[str]) – The name of the op.
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - can_subset (bool) – If this asset’s computation can emit a subset of the asset
+ - resource_defs (Optional[Mapping[str, object]]) – (Experimental) A mapping of resource keys to resources. These resources
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. This
+ - specs (Optional[Sequence[[*AssetSpec*](#dagster.AssetSpec)]]) – (Experimental) The specifications for the assets
+ - check_specs (Optional[Sequence[[*AssetCheckSpec*](asset-checks.mdx#dagster.AssetCheckSpec)]]) – (Experimental) Specs for asset checks that
+
+
+ Examples:
+
+ ```python
+ @multi_observable_source_asset(
+ specs=[AssetSpec("asset1"), AssetSpec("asset2")],
+ )
+ def my_function():
+ yield ObserveResult(asset_key="asset1", metadata={"foo": "bar"})
+ yield ObserveResult(asset_key="asset2", metadata={"baz": "qux"})
+ ```
+
+
+
+
+
+
class dagster.AssetOut
+
+
+ Defines one of the assets produced by a [`@multi_asset`](#dagster.multi_asset).
+
+
+
static from_spec
+
+
+ Builds an AssetOut from the passed spec.
+
+ Parameters:
+ - spec ([*AssetSpec*](#dagster.AssetSpec)) – The spec to build the AssetOut from.
+ - dagster_type (Optional[Union[Type, [*DagsterType*](types.mdx#dagster.DagsterType)]]) – The type of this output. Should only
+ - is_required (bool) – Whether the presence of this field is required. (default: True)
+ - io_manager_key (Optional[str]) – The resource key of the IO manager used for this output.
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – BackfillPolicy to apply to the specified
+
+
+ Returns: The AssetOut built from the spec.Return type: [AssetOut](#dagster.AssetOut)
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Source assets
+
+Refer to the [External asset dependencies](https://docs.dagster.io/guides/build/assets/external-assets) documentation for more information.
+
+
+
class dagster.SourceAsset
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.0.
+ Use AssetSpec instead. If using the SourceAsset io_manager_key property, use AssetSpec(...).with_io_manager_key(...)..
+
+ :::
+
+ A SourceAsset represents an asset that will be loaded by (but not updated by) Dagster.
+
+
+
key
+
+
+ The key of the asset.
+
+ Type: Union[[AssetKey](#dagster.AssetKey), Sequence[str], str]
+
+
+
+
+
+
metadata
+
+
+ Metadata associated with the asset.
+
+ Type: Mapping[str, [MetadataValue](metadata.mdx#dagster.MetadataValue)]
+
+
+
+
+
+
io_manager_key
+
+
+ The key for the IOManager that will be used to load the contents of
+ the asset when it’s used as an input to other assets inside a job.
+
+ Type: Optional[str]
+
+
+
+
+
+
io_manager_def
+
+
+ (Experimental) The definition of the IOManager that will be used to load the contents of
+ the asset when it’s used as an input to other assets inside a job.
+
+ Type: Optional[[IOManagerDefinition](io-managers.mdx#dagster.IOManagerDefinition)]
+
+
+
+
+
+
resource_defs
+
+
+ (Experimental) resource definitions that may be required by the [`dagster.IOManagerDefinition`](io-managers.mdx#dagster.IOManagerDefinition) provided in the io_manager_def argument.
+
+ Type: Optional[Mapping[str, [ResourceDefinition](resources.mdx#dagster.ResourceDefinition)]]
+
+
+
+
+
+
description
+
+
+ The description of the asset.
+
+ Type: Optional[str]
+
+
+
+
+
+
partitions_def
+
+
+ Defines the set of partition keys that
+ compose the asset.
+
+ Type: Optional[[PartitionsDefinition](partitions.mdx#dagster.PartitionsDefinition)]
+
+
+
+ A dictionary of tags for the op that computes the asset.
+ Frameworks may expect and require certain metadata to be attached to a op. Values that
+ are not strings will be json encoded and must meet the criteria that
+ json.loads(json.dumps(value)) == value.
+
+ Type: Optional[Dict[str, Any]]
+
+
+
+
+
+
auto_observe_interval_minutes
+
+
+ While the asset daemon is turned on, a run
+ of the observation function for this asset will be launched at this interval. observe_fn
+ must be provided.
+
+ Type: Optional[float]
+
+
+
+
+
+
freshness_policy
+
+
+ A constraint telling Dagster how often this asset is intended to be updated
+ with respect to its root data.
+
+ Type: FreshnessPolicy
+
+
+
+
+
+
tags
+
+
+ Tags for filtering and organizing. These tags are not
+ attached to runs of the asset.
+
+ Type: Optional[Mapping[str, str]]
+
+
+
+
+
+
property is_observable
+
+
+ Whether the asset is observable.
+
+ Type: bool
+
+
+
+
+
+
property op
+
+
+ The OpDefinition associated with the observation function of an observable
+ source asset.
+
+ Throws an error if the asset is not observable.
+
+ Type: [OpDefinition](ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
+
+
+
+
@dagster.observable_source_asset
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Create a SourceAsset with an associated observation function.
+
+ The observation function of a source asset is wrapped inside of an op and can be executed as
+ part of a job. Each execution generates an AssetObservation event associated with the source
+ asset. The source asset observation function should return a `DataVersion`,
+ a ~dagster.DataVersionsByPartition, or an [`ObserveResult`](#dagster.ObserveResult).
+
+ Parameters:
+ - name (Optional[str]) – The name of the source asset. If not provided, defaults to the name of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the source asset’s key is the
+ - metadata (Mapping[str, RawMetadataValue]) – Metadata associated with the asset.
+ - io_manager_key (Optional[str]) – The key for the IOManager that will be used to load the contents of
+ - io_manager_def (Optional[[*IOManagerDefinition*](io-managers.mdx#dagster.IOManagerDefinition)]) – (Experimental) The definition of the IOManager that will be used to load the contents of
+ - description (Optional[str]) – The description of the asset.
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided,
+ - required_resource_keys (Optional[Set[str]]) – Set of resource keys required by the observe op.
+ - resource_defs (Optional[Mapping[str, [*ResourceDefinition*](resources.mdx#dagster.ResourceDefinition)]]) – (Experimental) resource
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset.
+ - tags (Optional[Mapping[str, str]]) – Tags for filtering and organizing. These tags are not
+ - observe_fn (Optional[SourceAssetObserveFunction]) – Observation function for the source asset.
+ - automation_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – A condition describing when Dagster
+
+
+
+
+
+
+
+
class dagster.ObserveResult
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ An object representing a successful observation of an asset. These can be returned from an
+ @observable_source_asset decorated function to pass metadata.
+
+
+
asset_key
+
+
+ The asset key. Optional to include.
+
+ Type: Optional[[AssetKey](#dagster.AssetKey)]
+
+
+
+
+
+
metadata
+
+
+ Metadata to record with the corresponding
+ AssetObservation event.
+
+ Type: Optional[RawMetadataMapping]
+
+
+
+
+
+
check_results
+
+
+ Check results to record with the
+ corresponding AssetObservation event.
+
+ Type: Optional[Sequence[[AssetCheckResult](asset-checks.mdx#dagster.AssetCheckResult)]]
+
+
+
+
+
+
data_version
+
+
+ The data version of the asset that was observed.
+
+ Type: Optional[DataVersion]
+
+
+
+
+
+
tags
+
+
+ Tags to record with the corresponding AssetObservation
+ event.
+
+ Type: Optional[Mapping[str, str]]
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Dependencies
+
+
+
class dagster.AssetDep
+
+
+ Specifies a dependency on an upstream asset.
+
+
+
asset
+
+
+ The upstream asset to depend on.
+
+ Type: Union[[AssetKey](#dagster.AssetKey), str, [AssetSpec](#dagster.AssetSpec), [AssetsDefinition](#dagster.AssetsDefinition), [SourceAsset](#dagster.SourceAsset)]
+
+
+
+
+
+
partition_mapping
+
+
+ Defines what partitions to depend on in
+ the upstream asset. If not provided and the upstream asset is partitioned, defaults to
+ the default partition mapping for the partitions definition, which is typically maps
+ partition keys to the same partition keys in upstream assets.
+
+ Type: Optional[[PartitionMapping](partitions.mdx#dagster.PartitionMapping)]
+
+
+
+ If provided, the asset’s key is the
+ concatenation of the key_prefix and the input name. Only one of the “key_prefix” and
+ “key” arguments should be provided.
+
+ Type: Optional[Union[str, Sequence[str]]]
+
+
+
+
+
+
key
+
+
+ The asset’s key. Only one of the
+ “key_prefix” and “key” arguments should be provided.
+
+ Type: Optional[Union[str, Sequence[str], [AssetKey](#dagster.AssetKey)]]
+
+
+
+
+
+
metadata
+
+
+ A dict of the metadata for the input.
+ For example, if you only need a subset of columns from an upstream table, you could
+ include that in metadata and the IO manager that loads the upstream table could use the
+ metadata to determine which columns to load.
+
+ Type: Optional[Dict[str, Any]]
+
+
+
+
+
+
partition_mapping
+
+
+ Defines what partitions to depend on in
+ the upstream asset. If not provided, defaults to the default partition mapping for the
+ partitions definition, which is typically maps partition keys to the same partition keys
+ in upstream assets.
+
+ Type: Optional[[PartitionMapping](partitions.mdx#dagster.PartitionMapping)]
+
+
+
+
+
+
dagster_type
+
+
+ Allows specifying type validation functions that
+ will be executed on the input of the decorated function before it runs.
+
+ Type: [DagsterType](types.mdx#dagster.DagsterType)
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Asset jobs
+
+[Asset jobs](https://docs.dagster.io/guides/build/assets/asset-jobs) enable the automation of asset materializations. Dagster’s [asset selection syntax](https://docs.dagster.io/guides/build/assets/asset-selection-syntax) can be used to select assets and assign them to a job.
+
+
+
dagster.define_asset_job
+
+
+ Creates a definition of a job which will either materialize a selection of assets or observe
+ a selection of source assets. This will only be resolved to a JobDefinition once placed in a
+ code location.
+
+ Parameters:
+ - name (str) – The name for the job.
+ - selection (Union[str, Sequence[str], Sequence[[*AssetKey*](#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](#dagster.AssetsDefinition), [*SourceAsset*](#dagster.SourceAsset)]], [*AssetSelection*](#dagster.AssetSelection)]) –
+
+ The assets that will be materialized or observed when the job is run.
+
+ The selected assets must all be included in the assets that are passed to the assets
+ argument of the Definitions object that this job is included on.
+
+ The string “my_asset*” selects my_asset and all downstream assets within the code
+ location. A list of strings represents the union of all assets selected by strings
+ within the list.
+
+ - config –
+
+ Describes how the Job is parameterized at runtime.
+
+ If no value is provided, then the schema for the job’s run config is a standard
+ format based on its ops and resources.
+
+ If a dictionary is provided, then it must conform to the standard config schema, and
+ it will be used as the job’s run config for the job whenever the job is executed.
+ The values provided will be viewable and editable in the Dagster UI, so be
+ careful with secrets.
+
+ - tags (Optional[Mapping[str, object]]) – A set of key-value tags that annotate the job and can
+ - run_tags (Optional[Mapping[str, object]]) – A set of key-value tags that will be automatically attached to runs launched by this
+ - metadata (Optional[Mapping[str, RawMetadataValue]]) – Arbitrary metadata about the job.
+ - description (Optional[str]) – A description for the Job.
+ - executor_def (Optional[[*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]) – How this Job will be executed. Defaults to [`multi_or_in_process_executor`](execution.mdx#dagster.multi_or_in_process_executor),
+ - op_retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The default retry policy for all ops that compute assets in this job.
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – deprecated
+
+
+ Returns: The job, which can be placed inside a code location.Return type: UnresolvedAssetJobDefinition
+ Examples:
+
+ ```python
+ # A job that targets all assets in the code location:
+ @asset
+ def asset1():
+ ...
+
+ defs = Definitions(
+ assets=[asset1],
+ jobs=[define_asset_job("all_assets")],
+ )
+
+ # A job that targets a single asset
+ @asset
+ def asset1():
+ ...
+
+ defs = Definitions(
+ assets=[asset1],
+ jobs=[define_asset_job("all_assets", selection=[asset1])],
+ )
+
+ # A job that targets all the assets in a group:
+ defs = Definitions(
+ assets=assets,
+ jobs=[define_asset_job("marketing_job", selection=AssetSelection.groups("marketing"))],
+ )
+
+ @observable_source_asset
+ def source_asset():
+ ...
+
+ # A job that observes a source asset:
+ defs = Definitions(
+ assets=assets,
+ jobs=[define_asset_job("observation_job", selection=[source_asset])],
+ )
+
+ # Resources are supplied to the assets, not the job:
+ @asset(required_resource_keys={"slack_client"})
+ def asset1():
+ ...
+
+ defs = Definitions(
+ assets=[asset1],
+ jobs=[define_asset_job("all_assets")],
+ resources={"slack_client": prod_slack_client},
+ )
+ ```
+
+
+
+
+
+
class dagster.AssetSelection
+
+
+ An AssetSelection defines a query over a set of assets and asset checks, normally all that are defined in a code location.
+
+ You can use the “|”, “&”, and “-” operators to create unions, intersections, and differences of selections, respectively.
+
+ AssetSelections are typically used with [`define_asset_job()`](#dagster.define_asset_job).
+
+ By default, selecting assets will also select all of the asset checks that target those assets.
+
+ Examples:
+
+ ```python
+ # Select all assets in group "marketing":
+ AssetSelection.groups("marketing")
+
+ # Select all assets in group "marketing", as well as the asset with key "promotion":
+ AssetSelection.groups("marketing") | AssetSelection.assets("promotion")
+
+ # Select all assets in group "marketing" that are downstream of asset "leads":
+ AssetSelection.groups("marketing") & AssetSelection.assets("leads").downstream()
+
+ # Select a list of assets:
+ AssetSelection.assets(*my_assets_list)
+
+ # Select all assets except for those in group "marketing"
+ AssetSelection.all() - AssetSelection.groups("marketing")
+
+ # Select all assets which are materialized by the same op as "projections":
+ AssetSelection.assets("projections").required_multi_asset_neighbors()
+
+ # Select all assets in group "marketing" and exclude their asset checks:
+ AssetSelection.groups("marketing") - AssetSelection.all_asset_checks()
+
+ # Select all asset checks that target a list of assets:
+ AssetSelection.checks_for_assets(*my_assets_list)
+
+ # Select a specific asset check:
+ AssetSelection.checks(my_asset_check)
+ ```
+
+
static all
+
+
+ Returns a selection that includes all assets and their asset checks.
+
+ Parameters: include_sources (bool) – experimental
+
+
+
+
+
+
static all_asset_checks
+
+ Returns a selection that includes all asset checks.
+
+
+
+
+
static assets
+
+
+ Returns a selection that includes all of the provided assets and asset checks that target
+ them.
+
+ Parameters: *assets_defs (Union[[*AssetsDefinition*](#dagster.AssetsDefinition), str, Sequence[str], [*AssetKey*](#dagster.AssetKey)]) – The assets to
+ select.
+ Examples:
+
+ ```python
+ AssetSelection.assets(AssetKey(["a"]))
+
+ AssetSelection.assets("a")
+
+ AssetSelection.assets(AssetKey(["a"]), AssetKey(["b"]))
+
+ AssetSelection.assets("a", "b")
+
+ @asset
+ def asset1():
+ ...
+
+ AssetSelection.assets(asset1)
+
+ asset_key_list = [AssetKey(["a"]), AssetKey(["b"])]
+ AssetSelection.assets(*asset_key_list)
+ ```
+
+
+
+
+
+
static checks
+
+ Returns a selection that includes all of the provided asset checks or check keys.
+
+
+
+
+
static checks_for_assets
+
+
+ Returns a selection with the asset checks that target the provided assets.
+
+ Parameters: *assets_defs (Union[[*AssetsDefinition*](#dagster.AssetsDefinition), str, Sequence[str], [*AssetKey*](#dagster.AssetKey)]) – The assets to
+ select checks for.
+
+
+
+
+
+
downstream
+
+
+ Returns a selection that includes all assets that are downstream of any of the assets in
+ this selection, selecting the assets in this selection by default. Includes the asset checks targeting the returned assets. Iterates through each
+ asset in this selection and returns the union of all downstream assets.
+
+ depth (Optional[int]): If provided, then only include assets to the given depth. A depth
+ of 2 means all assets that are children or grandchildren of the assets in this
+ selection.
+
+ include_self (bool): If True, then include the assets in this selection in the result.
+ If the include_self flag is False, return each downstream asset that is not part of the
+ original selection. By default, set to True.
+
+
+
+
+
+
+
static groups
+
+
+ Returns a selection that includes materializable assets that belong to any of the
+ provided groups and all the asset checks that target them.
+
+ Parameters: include_sources (bool) – If True, then include source assets matching the group in the
+ selection.
+
+
+
+
+
+
static key_prefixes
+
+
+ Returns a selection that includes assets that match any of the provided key prefixes and all the asset checks that target them.
+
+ Parameters: include_sources (bool) – If True, then include source assets matching the key prefix(es)
+ in the selection.
+ Examples:
+
+ ```python
+ # match any asset key where the first segment is equal to "a" or "b"
+ # e.g. AssetKey(["a", "b", "c"]) would match, but AssetKey(["abc"]) would not.
+ AssetSelection.key_prefixes("a", "b")
+
+ # match any asset key where the first two segments are ["a", "b"] or ["a", "c"]
+ AssetSelection.key_prefixes(["a", "b"], ["a", "c"])
+ ```
+
+
+
+
+
+
static keys
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use AssetSelection.assets instead..
+
+ :::
+
+ Returns a selection that includes assets with any of the provided keys and all asset
+ checks that target them.
+
+ Deprecated: use AssetSelection.assets instead.
+
+ Examples:
+
+ ```python
+ AssetSelection.keys(AssetKey(["a"]))
+
+ AssetSelection.keys("a")
+
+ AssetSelection.keys(AssetKey(["a"]), AssetKey(["b"]))
+
+ AssetSelection.keys("a", "b")
+
+ asset_key_list = [AssetKey(["a"]), AssetKey(["b"])]
+ AssetSelection.keys(*asset_key_list)
+ ```
+
+
+
+
+
+
materializable
+
+ Given an asset selection, returns a new asset selection that contains all of the assets
+ that are materializable. Removes any assets which are not materializable.
+
+
+
+
+
required_multi_asset_neighbors
+
+ Given an asset selection in which some assets are output from a multi-asset compute op
+ which cannot be subset, returns a new asset selection that contains all of the assets
+ required to execute the original asset selection. Includes the asset checks targeting the returned assets.
+
+
+
+
+
roots
+
+
+ Given an asset selection, returns a new asset selection that contains all of the root
+ assets within the original asset selection. Includes the asset checks targeting the returned assets.
+
+ A root asset is an asset that has no upstream dependencies within the asset selection.
+ The root asset can have downstream dependencies outside of the asset selection.
+
+ Because mixed selections of source and materializable assets are currently not supported,
+ keys corresponding to SourceAssets will not be included as roots. To select source assets,
+ use the upstream_source_assets method.
+
+
+
+
+
+
+
sinks
+
+
+ Given an asset selection, returns a new asset selection that contains all of the sink
+ assets within the original asset selection. Includes the asset checks targeting the returned assets.
+
+ A sink asset is an asset that has no downstream dependencies within the asset selection.
+ The sink asset can have downstream dependencies outside of the asset selection.
+
+
+
+
+
+
+
sources
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use AssetSelection.roots instead..
+
+ :::
+
+ Given an asset selection, returns a new asset selection that contains all of the root
+ assets within the original asset selection. Includes the asset checks targeting the returned assets.
+
+ A root asset is a materializable asset that has no upstream dependencies within the asset
+ selection. The root asset can have downstream dependencies outside of the asset selection.
+
+ Because mixed selections of source and materializable assets are currently not supported,
+ keys corresponding to SourceAssets will not be included as roots. To select source assets,
+ use the upstream_source_assets method.
+
+
+
+
+
+
+
static tag
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a selection that includes materializable assets that have the provided tag, and
+ all the asset checks that target them.
+
+ Parameters: include_sources (bool) – If True, then include source assets matching the group in the
+ selection.
+
+
+
+
+
+
upstream
+
+
+ Returns a selection that includes all materializable assets that are upstream of any of
+ the assets in this selection, selecting the assets in this selection by default. Includes
+ the asset checks targeting the returned assets. Iterates through each asset in this
+ selection and returns the union of all upstream assets.
+
+ Because mixed selections of source and materializable assets are currently not supported,
+ keys corresponding to SourceAssets will not be included as upstream of regular assets.
+
+ Parameters:
+ - depth (Optional[int]) – If provided, then only include assets to the given depth. A depth
+ - include_self (bool) – If True, then include the assets in this selection in the result.
+
+
+
+
+
+
+
+
upstream_source_assets
+
+ Given an asset selection, returns a new asset selection that contains all of the source
+ assets that are parents of assets in the original selection. Includes the asset checks
+ targeting the returned assets.
+
+
+
+
+
without_checks
+
+ Removes all asset checks in the selection.
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Code locations
+
+Loading assets and asset jobs into a [code location](https://docs.dagster.io/guides/deploy/code-locations/) makes them available to Dagster tools like the UI, CLI, and GraphQL API.
+
+
+
dagster.load_assets_from_modules
+
+
+ Constructs a list of assets and source assets from the given modules.
+
+ Parameters:
+ - modules (Iterable[ModuleType]) – The Python modules to look for assets inside.
+ - group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies
+ - freshness_policy (Optional[FreshnessPolicy]) – FreshnessPolicy to apply to all the loaded
+ - automation_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – AutomationCondition to apply
+ - backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
+ - source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned
+
+
+ Returns: A list containing assets and source assets defined in the given modules.Return type: Sequence[Union[[AssetsDefinition](#dagster.AssetsDefinition), [SourceAsset](#dagster.SourceAsset)]]
+
+
+
+
+
+
dagster.load_assets_from_current_module
+
+
+ Constructs a list of assets, source assets, and cacheable assets from the module where
+ this function is called.
+
+ Parameters:
+ - group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies
+ - freshness_policy (Optional[FreshnessPolicy]) – FreshnessPolicy to apply to all the loaded
+ - automation_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – AutomationCondition to apply
+ - backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
+ - source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned
+
+
+ Returns: A list containing assets, source assets, and cacheable assets defined in the module.Return type: Sequence[Union[[AssetsDefinition](#dagster.AssetsDefinition), [SourceAsset](#dagster.SourceAsset), CachableAssetsDefinition]]
+
+
+
+
+
+
dagster.load_assets_from_package_module
+
+
+ Constructs a list of assets and source assets that includes all asset
+ definitions, source assets, and cacheable assets in all sub-modules of the given package module.
+
+ A package module is the result of importing a package.
+
+ Parameters:
+ - package_module (ModuleType) – The package module to looks for assets inside.
+ - group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies
+ - freshness_policy (Optional[FreshnessPolicy]) – FreshnessPolicy to apply to all the loaded
+ - automation_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – AutomationCondition to apply
+ - backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
+ - source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned
+
+
+ Returns: A list containing assets, source assets, and cacheable assets defined in the module.Return type: Sequence[Union[[AssetsDefinition](#dagster.AssetsDefinition), [SourceAsset](#dagster.SourceAsset), CacheableAssetsDefinition]]
+
+
+
+
+
+
dagster.load_assets_from_package_name
+
+
+ Constructs a list of assets, source assets, and cacheable assets that includes all asset
+ definitions and source assets in all sub-modules of the given package.
+
+ Parameters:
+ - package_name (str) – The name of a Python package to look for assets inside.
+ - group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies
+ - freshness_policy (Optional[FreshnessPolicy]) – FreshnessPolicy to apply to all the loaded
+ - auto_materialize_policy (Optional[AutoMaterializePolicy]) – AutoMaterializePolicy to apply
+ - backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
+ - source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned
+
+
+ Returns: A list containing assets, source assets, and cacheable assets defined in the module.Return type: Sequence[Union[[AssetsDefinition](#dagster.AssetsDefinition), [SourceAsset](#dagster.SourceAsset), CacheableAssetsDefinition]]
+
+
+
+
+
+
+
+
+
+
+## Observations
+
+Refer to the [Asset observation](https://docs.dagster.io/guides/build/assets/metadata-and-tags/asset-observations) documentation for more information.
+
+
+
class dagster.AssetObservation
+
+
+ Event that captures metadata about an asset at a point in time.
+
+ Parameters:
+ - asset_key (Union[str, List[str], [*AssetKey*](#dagster.AssetKey)]) – A key to identify the asset.
+ - partition (Optional[str]) – The name of a partition of the asset that the metadata
+ - tags (Optional[Mapping[str, str]]) – A mapping containing tags for the observation.
+ - metadata (Optional[Dict[str, Union[str, float, int, [*MetadataValue*](metadata.mdx#dagster.MetadataValue)]]]) – Arbitrary metadata about the asset. Keys are displayed string labels, and values are
+
+
+
+
+
+
+
+
+
+
+
+
+## Declarative Automation
+
+Refer to the [Declarative Automation](https://docs.dagster.io/guides/automate/declarative-automation/) documentation for more information.
+
+
+
class dagster.AutomationCondition
+
+
+ An AutomationCondition represents a condition of an asset that impacts whether it should be
+ automatically executed. For example, you can have a condition which becomes true whenever the
+ code version of the asset is changed, or whenever an upstream dependency is updated.
+
+ ```python
+ from dagster import AutomationCondition, asset
+
+ @asset(automation_condition=AutomationCondition.on_cron("0 0 * * *"))
+ def my_asset(): ...
+ ```
+ AutomationConditions may be combined together into expressions using a variety of operators.
+
+ ```python
+ from dagster import AssetSelection, AutomationCondition, asset
+
+ # any dependencies from the "important" group are missing
+ any_important_deps_missing = AutomationCondition.any_deps_match(
+ AutomationCondition.missing(),
+ ).allow(AssetSelection.groups("important"))
+
+ # there is a new code version for this asset since the last time it was requested
+ new_code_version = AutomationCondition.code_version_changed().since(
+ AutomationCondition.newly_requested()
+ )
+
+ # there is a new code version and no important dependencies are missing
+ my_condition = new_code_version & ~any_important_deps_missing
+
+ @asset(automation_condition=my_condition)
+ def my_asset(): ...
+ ```
+
+
static all_checks_match
+
+
+ Returns an AutomationCondition that is true for an asset partition if all of its checks
+ evaluate to True for the given condition.
+
+ Parameters:
+ - condition ([*AutomationCondition*](#dagster.AutomationCondition)) – The AutomationCondition that will be evaluated against
+ - blocking_only (bool) – Determines if this condition will only be evaluated against blocking
+
+
+
+
+
+
+
+
static all_deps_blocking_checks_passed
+
+
+ Returns an AutomationCondition that is true for any partition where all upstream
+ blocking checks have passed, or will be requested on this tick.
+
+ In-tick requests are allowed to enable creating runs that target both a parent with
+ blocking checks and a child. Even though the checks have not currently passed, if
+ they fail within the run, the run machinery will prevent the child from being
+ materialized.
+
+
+
+
+
+
+
static all_deps_match
+
+
+ Returns an AutomationCondition that is true for a if at least one partition
+ of the all of the target’s dependencies evaluate to True for the given condition.
+
+ Parameters: condition ([*AutomationCondition*](#dagster.AutomationCondition)) – The AutomationCondition that will be evaluated against
+ this target’s dependencies.
+
+
+
+
+
+
static all_deps_updated_since_cron
+
+ Returns an AutomatonCondition that is true if all of the target’s dependencies have
+ updated since the latest tick of the provided cron schedule.
+
+
+
+
+
static any_checks_match
+
+
+ Returns an AutomationCondition that is true for if at least one of the target’s
+ checks evaluate to True for the given condition.
+
+ Parameters:
+ - condition ([*AutomationCondition*](#dagster.AutomationCondition)) – The AutomationCondition that will be evaluated against
+ - blocking_only (bool) – Determines if this condition will only be evaluated against blocking
+
+
+
+
+
+
+
+
static any_deps_in_progress
+
+ Returns an AutomationCondition that is true if the target has at least one dependency
+ that is in progress.
+
+
+
+
+
static any_deps_match
+
+
+ Returns an AutomationCondition that is true for a if at least one partition
+ of the any of the target’s dependencies evaluate to True for the given condition.
+
+ Parameters: condition ([*AutomationCondition*](#dagster.AutomationCondition)) – The AutomationCondition that will be evaluated against
+ this target’s dependencies.
+
+
+
+
+
+
static any_deps_missing
+
+ Returns an AutomationCondition that is true if the target has at least one dependency
+ that is missing, and will not be requested on this tick.
+
+
+
+
+
static any_deps_updated
+
+
+ Returns an AutomationCondition that is true if the target has at least one dependency
+ that has updated since the previous tick, or will be requested on this tick.
+
+ Will ignore parent updates if the run that updated the parent also plans to update
+ the asset or check that this condition is applied to.
+
+
+
+
+
+
+
static any_downstream_conditions
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns an AutomationCondition which represents the union of all distinct downstream conditions.
+
+
+
+
+
+
+
static asset_matches
+
+ Returns an AutomationCondition that is true if this condition is true for the given key.
+
+
+
+
+
static backfill_in_progress
+
+ Returns an AutomationCondition that is true if the target is part of an in-progress backfill.
+
+
+
+
+
static check_failed
+
+ Returns an AutomationCondition that is true for an asset check if it has evaluated against
+ the latest materialization of an asset and failed.
+
+
+
+
+
static check_passed
+
+ Returns an AutomationCondition that is true for an asset check if it has evaluated against
+ the latest materialization of an asset and passed.
+
+
+
+
+
static code_version_changed
+
+ Returns an AutomationCondition that is true if the target’s code version has been changed
+ since the previous tick.
+
+
+
+
+
static cron_tick_passed
+
+ Returns an AutomationCondition that is whenever a cron tick of the provided schedule is passed.
+
+
+
+
+
static eager
+
+
+ Returns an AutomationCondition which will cause a target to be executed if any of
+ its dependencies update, and will execute missing partitions if they become missing
+ after this condition is applied to the target.
+
+ This will not execute targets that have any missing or in progress dependencies, or
+ are currently in progress.
+
+ For time partitioned assets, only the latest time partition will be considered.
+
+
+
+
+
+
+
static execution_failed
+
+ Returns an AutomationCondition that is true if the latest execution of the target failed.
+
+
+
+
+
static in_latest_time_window
+
+
+ Returns an AutomationCondition that is true when the target it is within the latest
+ time window.
+
+ Parameters: lookback_delta (Optional, datetime.timedelta) – If provided, the condition will
+ return all partitions within the provided delta of the end of the latest time window.
+ For example, if this is used on a daily-partitioned asset with a lookback_delta of
+ 48 hours, this will return the latest two partitions.
+
+
+
+
+
+
static in_progress
+
+ Returns an AutomationCondition that is true for an asset partition if it is part of an
+ in-progress run or backfill.
+
+
+
+
+
static initial_evaluation
+
+ Returns an AutomationCondition that is true on the first evaluation of the expression.
+
+
+
+
+
static missing
+
+ Returns an AutomationCondition that is true if the target has not been executed.
+
+
+
+
+
static newly_missing
+
+ Returns an AutomationCondition that is true on the tick that the target becomes missing.
+
+
+
+
+
static newly_requested
+
+ Returns an AutomationCondition that is true if the target was requested on the previous tick.
+
+
+
+
+
static newly_updated
+
+ Returns an AutomationCondition that is true if the target has been updated since the previous tick.
+
+
+
+
+
static on_cron
+
+
+ Returns an AutomationCondition which will cause a target to be executed on a given
+ cron schedule, after all of its dependencies have been updated since the latest
+ tick of that cron schedule.
+
+ For time partitioned assets, only the latest time partition will be considered.
+
+
+
+
+
+
+
static on_missing
+
+
+ Returns an AutomationCondition which will execute partitions of the target that
+ are added after this condition is applied to the asset.
+
+ This will not execute targets that have any missing dependencies.
+
+ For time partitioned assets, only the latest time partition will be considered.
+
+
+
+
+
+
+
replace
+
+
+ Replaces all instances of `old` across any sub-conditions with `new`.
+
+ If `old` is a string, then conditions with a label matching
+ that string will be replaced.
+
+ Parameters:
+ - old (Union[[*AutomationCondition*](#dagster.AutomationCondition), str]) – The condition to replace.
+ - new ([*AutomationCondition*](#dagster.AutomationCondition)) – The condition to replace with.
+
+
+
+
+
+
+
+
static run_in_progress
+
+ Returns an AutomationCondition that is true if the target is part of an in-progress run.
+
+
+
+
+
static will_be_requested
+
+ Returns an AutomationCondition that is true if the target will be requested this tick.
+
+
+
+
+
+
+
+
+
class dagster.AutomationConditionSensorDefinition
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Targets a set of assets and repeatedly evaluates all the AutomationConditions on all of
+ those assets to determine which to request runs for.
+
+ Parameters:
+ - name – The name of the sensor.
+ - target (Union[str, Sequence[str], Sequence[[*AssetKey*](#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](#dagster.AssetsDefinition), [*SourceAsset*](#dagster.SourceAsset)]], [*AssetSelection*](#dagster.AssetSelection)]) – A selection of assets to evaluate AutomationConditions of and request runs for.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - run_tags (Optional[Mapping[str, Any]]) – Tags that will be automatically attached to runs launched by this sensor.
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - minimum_interval_seconds (Optional[int]) – The frequency at which to try to evaluate the
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - emit_backfills (bool) – If set to True, will emit a backfill on any tick where more than one partition
+ - use_user_code_server (bool) – (experimental) If set to True, this sensor will be evaluated in the user
+ - default_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – (experimental) If provided, this condition will
+
+
+ Examples:
+
+ ```python
+ import dagster as dg
+
+ # automation condition sensor that defaults to running
+ defs1 = dg.Definitions(
+ assets=...,
+ sensors=[
+ dg.AutomationConditionSensorDefinition(
+ name="automation_condition_sensor",
+ target=dg.AssetSelection.all(),
+ default_status=dg.DefaultSensorStatus.RUNNING,
+ ),
+ ]
+ )
+
+ # one automation condition sensor per group
+ defs2 = dg.Definitions(
+ assets=...,
+ sensors=[
+ dg.AutomationConditionSensorDefinition(
+ name="raw_data_automation_condition_sensor",
+ target=dg.AssetSelection.groups("raw_data"),
+ ),
+ dg.AutomationConditionSensorDefinition(
+ name="ml_automation_condition_sensor",
+ target=dg.AssetSelection.groups("machine_learning"),
+ ),
+ ]
+ )
+ ```
+
+
+
+
+
+
+
+
+
+## Asset values
+
+
+
class dagster.AssetValueLoader
+
+
+ Caches resource definitions that are used to load asset values across multiple load
+ invocations.
+
+ Should not be instantiated directly. Instead, use
+ [`get_asset_value_loader()`](repositories.mdx#dagster.RepositoryDefinition.get_asset_value_loader).
+
+
+
load_asset_value
+
+
+ Loads the contents of an asset as a Python object.
+
+ Invokes load_input on the [`IOManager`](io-managers.mdx#dagster.IOManager) associated with the asset.
+
+ Parameters:
+ - asset_key (Union[[*AssetKey*](#dagster.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.
+ - input_definition_metadata (Optional[Dict[str, Any]]) – Input metadata to pass to the [`IOManager`](io-managers.mdx#dagster.IOManager)
+ - resource_config (Optional[Any]) – A dictionary of resource configurations to be passed
+
+
+ Returns: The contents of an asset as a Python object.
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/cli.mdx b/docs/docs-beta/docs/api/python-api/cli.mdx
new file mode 100644
index 0000000000000..e90997e4ac7f0
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/cli.mdx
@@ -0,0 +1,1871 @@
+---
+title: 'dagster cli'
+title_meta: 'dagster cli API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dagster cli Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Dagster CLI
+
+
+
+
+## dagster asset
+
+Commands for working with Dagster assets.
+
+ ```shell
+ dagster asset [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
list
+
+ List assets
+
+
+
+
+
materialize
+
+ Execute a run to materialize a selection…
+
+
+
+
+
wipe
+
+ Eliminate asset key indexes from event logs.
+
+
+
+
+
wipe-partitions-status-cache
+
+ Clears the asset partitions status cache,…
+
+
+
+
+
+
+
+
+
+## dagster debug
+
+Commands for helping debug Dagster issues by dumping or loading artifacts from specific runs.
+
+This can be used to send a file to someone like the Dagster team who doesn’t have direct access
+to your instance to allow them to view the events and details of a specific run.
+
+Debug files can be viewed using dagster-webserver-debug cli.
+Debug files can also be downloaded from the Dagster UI.
+
+ ```shell
+ dagster debug [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
export
+
+ Export the relevant artifacts for a job…
+
+
+
+
+
import
+
+ Import the relevant artifacts from debug…
+
+
+
+
+
+
+
+## dagster definitions validate
+
+The dagster definitions validate command loads and validate your Dagster definitions using a Dagster instance.
+
+This command indicates which code locations contain errors, and which ones can be successfully loaded.
+Code locations containing errors are considered invalid, otherwise valid.
+
+When running, this command sets the environment variable DAGSTER_IS_DEFS_VALIDATION_CLI=1.
+This environment variable can be used to control the behavior of your code in validation mode.
+
+This command returns an exit code 1 when errors are found, otherwise an exit code 0.
+
+This command should be run in a Python environment where the dagster package is installed.
+
+ ```shell
+ dagster definitions validate [OPTIONS]
+ ```
+Options:
+
+
+
--log-format \
+
+
+ Format of the logs for dagster services
+
+ Default: `'colored'`Options: colored | json | rich
+
+
+
+
+
+
--log-level \
+
+
+ Set the log level for dagster services.
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
-w, --workspace \
+
+ Path to workspace file. Argument can be provided multiple times.
+
+
+
+
+
-f, --python-file \
+
+ Specify python file or files (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each file as a code location in the current python environment.
+
+
+
+
+
-m, --module-name \
+
+ Specify module or modules (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each module as a code location in the current python environment.
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-definitions-validate-f)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-definitions-validate-m)
+
+
+
+
+
+
+
+
+
+
+
+## dagster dev
+
+Start a local deployment of Dagster, including dagster-webserver running on localhost and the dagster-daemon running in the background
+
+ ```shell
+ dagster dev [OPTIONS]
+ ```
+Options:
+
+
+
-d, --working-directory \
+
+ Specify working directory to use when loading the repository or job
+
+
+
+
+
-m, --module-name \
+
+ Specify module or modules (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each module as a code location in the current python environment.
+
+
+
+
+
-f, --python-file \
+
+ Specify python file or files (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each file as a code location in the current python environment.
+
+
+
+
+
-w, --workspace \
+
+ Path to workspace file. Argument can be provided multiple times.
+
+
+
+
+
--code-server-log-level \
+
+
+ Set the log level for code servers spun up by dagster services.
+
+ Default: `'warning'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-level \
+
+
+ Set the log level for dagster services.
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-format \
+
+
+ Format of the logs for dagster services
+
+ Default: `'colored'`Options: colored | json | rich
+
+
+
+
+
+
-p, --port, --dagit-port \
+
+ Port to use for the Dagster webserver.
+
+
+
+
+
-h, --host, --dagit-host \
+
+ Host to use for the Dagster webserver.
+
+
+
+
+
--live-data-poll-rate \
+
+
+ Rate at which the dagster UI polls for updated asset data (in milliseconds)
+
+ Default: `'2000'`
+
+
+
+
+
+
--use-legacy-code-server-behavior
+
+ Use the legacy behavior of the daemon and webserver each starting up their own code server
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--working-directory`](#cmdoption-dagster-dev-d)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-dev-m)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-dev-f)
+
+
+
+
+
+
+
+
+
+
+
+
+
+## dagster instance
+
+Commands for working with the current Dagster instance.
+
+ ```shell
+ dagster instance [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
concurrency
+
+ Commands for working with the…
+
+
+
+
+
info
+
+ List the information about the current…
+
+
+
+
+
migrate
+
+ Automatically migrate an out of date…
+
+
+
+
+
reindex
+
+ Rebuild index over historical runs for…
+
+
+
+
+
+
+
+
+
+## dagster job
+
+Commands for working with Dagster jobs.
+
+ ```shell
+ dagster job [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
backfill
+
+ Backfill a partitioned job.
+
+
+
+
+
execute
+
+ Execute a job.
+
+
+
+
+
launch
+
+ Launch a job using the run launcher…
+
+
+
+
+
list
+
+ List the jobs in a repository.
+
+
+
+
+
print
+
+ Print a job.
+
+
+
+
+
scaffold_config
+
+ Scaffold the config for a job.
+
+
+
+
+
+
+
+
+
+## dagster run
+
+Commands for working with Dagster job runs.
+
+ ```shell
+ dagster run [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
delete
+
+ Delete a run by id and its associated…
+
+
+
+
+
list
+
+ List the runs in the current Dagster…
+
+
+
+
+
migrate-repository
+
+ Migrate the run history for a job from a…
+
+
+
+
+
wipe
+
+ Eliminate all run history and event logs.
+
+
+
+
+
+
+
+
+
+## dagster schedule
+
+Commands for working with Dagster schedules.
+
+ ```shell
+ dagster schedule [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
debug
+
+ Debug information about the scheduler.
+
+
+
+
+
list
+
+ List all schedules that correspond to a…
+
+
+
+
+
logs
+
+ Get logs for a schedule.
+
+
+
+
+
preview
+
+ Preview changes that will be performed by…
+
+
+
+
+
restart
+
+ Restart a running schedule.
+
+
+
+
+
start
+
+ Start an existing schedule.
+
+
+
+
+
stop
+
+ Stop an existing schedule.
+
+
+
+
+
wipe
+
+ Delete the schedule history and turn off…
+
+
+
+
+
+
+
+
+
+## dagster sensor
+
+Commands for working with Dagster sensors.
+
+ ```shell
+ dagster sensor [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
cursor
+
+ Set the cursor value for an existing sensor.
+
+
+
+
+
list
+
+ List all sensors that correspond to a…
+
+
+
+
+
preview
+
+ Preview an existing sensor execution.
+
+
+
+
+
start
+
+ Start an existing sensor.
+
+
+
+
+
stop
+
+ Stop an existing sensor.
+
+
+
+
+
+
+
+
+
+## dagster project
+
+Commands for bootstrapping new Dagster projects and code locations.
+
+ ```shell
+ dagster project [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
from-example
+
+ Download one of the official Dagster examples to the current directory. This CLI enables you to quickly bootstrap your project with an officially maintained example.
+
+
+
+
+
list-examples
+
+ List the examples that available to bootstrap with.
+
+
+
+
+
scaffold
+
+ Create a folder structure with a single Dagster code location and other files such as pyproject.toml. This CLI enables you to quickly start building a new Dagster project with everything set up.
+
+
+
+
+
scaffold-code-location
+
+ (DEPRECATED; Use dagster project scaffold –excludes README.md instead) Create a folder structure with a single Dagster code location, in the current directory. This CLI helps you to scaffold a new Dagster code location within a folder structure that includes multiple Dagster code locations.
+
+
+
+
+
scaffold-repository
+
+ (DEPRECATED; Use dagster project scaffold –excludes README.md instead) Create a folder structure with a single Dagster repository, in the current directory. This CLI helps you to scaffold a new Dagster repository within a folder structure that includes multiple Dagster repositories
+
+
+
+
+
+
+
+## dagster-graphql
+
+Run a GraphQL query against the dagster interface to a specified repository or pipeline/job.
+
+Can only use ONE of –workspace/-w, –python-file/-f, –module-name/-m, –grpc-port, –grpc-socket.
+
+Examples:
+
+1. dagster-graphql
+2. dagster-graphql -y path/to/workspace.yaml
+3. dagster-graphql -f path/to/file.py -a define_repo
+4. dagster-graphql -m some_module -a define_repo
+5. dagster-graphql -f path/to/file.py -a define_pipeline
+6. dagster-graphql -m some_module -a define_pipeline
+ ```shell
+ dagster-graphql [OPTIONS]
+ ```
+Options:
+
+
+
--version
+
+ Show the version and exit.
+
+
+
+
+
-t, --text \
+
+ GraphQL document to execute passed as a string
+
+
+
+
+
-f, --file \
+
+ GraphQL document to execute passed as a file
+
+
+
+
+
-p, --predefined \
+
+
+ GraphQL document to execute, from a predefined set provided by dagster-graphql.
+
+ Options: launchPipelineExecution
+
+
+
+
+
+
-v, --variables \
+
+ A JSON encoded string containing the variables for GraphQL execution.
+
+
+
+
+
-r, --remote \
+
+ A URL for a remote instance running dagster-webserver to send the GraphQL request to.
+
+
+
+
+
-o, --output \
+
+ A file path to store the GraphQL response to. This flag is useful when making pipeline/job execution queries, since pipeline/job execution causes logs to print to stdout and stderr.
+
+
+
+
+
--ephemeral-instance
+
+ Use an ephemeral DagsterInstance instead of resolving via DAGSTER_HOME
+
+
+
+
+
--empty-workspace
+
+ Allow an empty workspace
+
+
+
+
+
-w, --workspace \
+
+ Path to workspace file. Argument can be provided multiple times.
+
+
+
+
+
-d, --working-directory \
+
+ Specify working directory to use when loading the repository or job
+
+
+
+
+
-f, --python-file \
+
+ Specify python file or files (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each file as a code location in the current python environment.
+
+
+
+
+
-m, --module-name \
+
+ Specify module or modules (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each module as a code location in the current python environment.
+
+
+
+
+
--package-name \
+
+ Specify Python package where repository or job function lives
+
+
+
+
+
-a, --attribute \
+
+ Attribute that is either a 1) repository or job or 2) a function that returns a repository or job
+
+
+
+
+
--grpc-port \
+
+ Port to use to connect to gRPC server
+
+
+
+
+
--grpc-socket \
+
+ Named socket to use to connect to gRPC server
+
+
+
+
+
--grpc-host \
+
+ Host to use to connect to gRPC server, defaults to localhost
+
+
+
+
+
--use-ssl
+
+ Use a secure channel when connecting to the gRPC server
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--working-directory`](#cmdoption-dagster-graphql-d)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-graphql-0)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-graphql-m)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PACKAGE_NAME
+
+ >
+
+ Provide a default for [`--package-name`](#cmdoption-dagster-graphql-package-name)
+
+
+
+
+
+
+
+
+
+
DAGSTER_ATTRIBUTE
+
+ >
+
+ Provide a default for [`--attribute`](#cmdoption-dagster-graphql-a)
+
+
+
+
+
+
+
+
+
+
+
+## dagster-webserver
+
+Run dagster-webserver. Loads a code location.
+
+Can only use ONE of –workspace/-w, –python-file/-f, –module-name/-m, –grpc-port, –grpc-socket.
+
+Examples:
+
+1. dagster-webserver (works if ./workspace.yaml exists)
+2. dagster-webserver -w path/to/workspace.yaml
+3. dagster-webserver -f path/to/file.py
+4. dagster-webserver -f path/to/file.py -d path/to/working_directory
+5. dagster-webserver -m some_module
+6. dagster-webserver -f path/to/file.py -a define_repo
+7. dagster-webserver -m some_module -a define_repo
+8. dagster-webserver -p 3333
+Options can also provide arguments via environment variables prefixed with DAGSTER_WEBSERVER.
+
+For example, DAGSTER_WEBSERVER_PORT=3333 dagster-webserver
+
+ ```shell
+ dagster-webserver [OPTIONS]
+ ```
+Options:
+
+
+
--use-ssl
+
+ Use a secure channel when connecting to the gRPC server
+
+
+
+
+
--grpc-host \
+
+ Host to use to connect to gRPC server, defaults to localhost
+
+
+
+
+
--grpc-socket \
+
+ Named socket to use to connect to gRPC server
+
+
+
+
+
--grpc-port \
+
+ Port to use to connect to gRPC server
+
+
+
+
+
-a, --attribute \
+
+ Attribute that is either a 1) repository or job or 2) a function that returns a repository or job
+
+
+
+
+
--package-name \
+
+ Specify Python package where repository or job function lives
+
+
+
+
+
-m, --module-name \
+
+ Specify module or modules (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each module as a code location in the current python environment.
+
+
+
+
+
-f, --python-file \
+
+ Specify python file or files (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each file as a code location in the current python environment.
+
+
+
+
+
-d, --working-directory \
+
+ Specify working directory to use when loading the repository or job
+
+
+
+
+
-w, --workspace \
+
+ Path to workspace file. Argument can be provided multiple times.
+
+
+
+
+
--empty-workspace
+
+ Allow an empty workspace
+
+
+
+
+
-h, --host \
+
+
+ Host to run server on
+
+ Default: `'127.0.0.1'`
+
+
+
+
+
+
-p, --port \
+
+ Port to run server on - defaults to 3000
+
+
+
+
+
-l, --path-prefix \
+
+
+ The path prefix where server will be hosted (eg: /dagster-webserver)
+
+ Default: `''`
+
+
+
+
+
+
--db-statement-timeout \
+
+
+ The timeout in milliseconds to set on database statements sent to the DagsterInstance. Not respected in all configurations.
+
+ Default: `15000`
+
+
+
+
+
+
--db-pool-recycle \
+
+
+ The maximum age of a connection to use from the sqlalchemy pool without connection recycling. Set to -1 to disable. Not respected in all configurations.
+
+ Default: `3600`
+
+
+
+
+
+
--read-only
+
+ Start server in read-only mode, where all mutations such as launching runs and turning schedules on/off are turned off.
+
+
+
+
+
--suppress-warnings
+
+ Filter all warnings when hosting server.
+
+
+
+
+
--uvicorn-log-level, --log-level \
+
+
+ Set the log level for the uvicorn web server.
+
+ Default: `'warning'`Options: critical | error | warning | info | debug | trace
+
+
+
+
+
+
--dagster-log-level \
+
+
+ Set the log level for dagster log events.
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-format \
+
+
+ Format of the log output from the webserver
+
+ Default: `'colored'`Options: colored | json | rich
+
+
+
+
+
+
--code-server-log-level \
+
+
+ Set the log level for any code servers spun up by the webserver.
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--live-data-poll-rate \
+
+
+ Rate at which the dagster UI polls for updated asset data (in milliseconds)
+
+ Default: `2000`
+
+
+
+
+
+
--version
+
+ Show the version and exit.
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_ATTRIBUTE
+
+ >
+
+ Provide a default for [`--attribute`](#cmdoption-dagster-webserver-a)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PACKAGE_NAME
+
+ >
+
+ Provide a default for [`--package-name`](#cmdoption-dagster-webserver-package-name)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-webserver-m)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-webserver-f)
+
+
+
+
+
+
+
+
+
+
DAGSTER_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--working-directory`](#cmdoption-dagster-webserver-d)
+
+
+
+
+
+
+
+
+
+
DAGSTER_WEBSERVER_LOG_LEVEL
+
+ >
+
+ Provide a default for [`--dagster-log-level`](#cmdoption-dagster-webserver-dagster-log-level)
+
+
+
+
+
+
+
+
+
+
+
+## dagster-daemon run
+
+Run any daemons configured on the DagsterInstance.
+
+ ```shell
+ dagster-daemon run [OPTIONS]
+ ```
+Options:
+
+
+
--code-server-log-level \
+
+
+ Set the log level for any code servers spun up by the daemon.
+
+ Default: `'warning'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-level \
+
+
+ Set the log level for any code servers spun up by the daemon.
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-format \
+
+
+ Format of the log output from the webserver
+
+ Default: `'colored'`Options: colored | json | rich
+
+
+
+
+
+
--use-ssl
+
+ Use a secure channel when connecting to the gRPC server
+
+
+
+
+
--grpc-host \
+
+ Host to use to connect to gRPC server, defaults to localhost
+
+
+
+
+
--grpc-socket \
+
+ Named socket to use to connect to gRPC server
+
+
+
+
+
--grpc-port \
+
+ Port to use to connect to gRPC server
+
+
+
+
+
-a, --attribute \
+
+ Attribute that is either a 1) repository or job or 2) a function that returns a repository or job
+
+
+
+
+
--package-name \
+
+ Specify Python package where repository or job function lives
+
+
+
+
+
-m, --module-name \
+
+ Specify module or modules (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each module as a code location in the current python environment.
+
+
+
+
+
-f, --python-file \
+
+ Specify python file or files (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each file as a code location in the current python environment.
+
+
+
+
+
-d, --working-directory \
+
+ Specify working directory to use when loading the repository or job
+
+
+
+
+
-w, --workspace \
+
+ Path to workspace file. Argument can be provided multiple times.
+
+
+
+
+
--empty-workspace
+
+ Allow an empty workspace
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_DAEMON_LOG_LEVEL
+
+ >
+
+ Provide a default for [`--log-level`](#cmdoption-dagster-daemon-run-log-level)
+
+
+
+
+
+
+
+
+
+
DAGSTER_ATTRIBUTE
+
+ >
+
+ Provide a default for [`--attribute`](#cmdoption-dagster-daemon-run-a)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PACKAGE_NAME
+
+ >
+
+ Provide a default for [`--package-name`](#cmdoption-dagster-daemon-run-package-name)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-daemon-run-m)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-daemon-run-f)
+
+
+
+
+
+
+
+
+
+
DAGSTER_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--working-directory`](#cmdoption-dagster-daemon-run-d)
+
+
+
+
+
+## dagster api grpc
+
+Serve the Dagster inter-process API over GRPC
+
+ ```shell
+ dagster api grpc [OPTIONS]
+ ```
+Options:
+
+
+
-p, --port \
+
+ Port over which to serve. You must pass one and only one of –port/-p or –socket/-s.
+
+
+
+
+
-s, --socket \
+
+ Serve over a UDS socket. You must pass one and only one of –port/-p or –socket/-s.
+
+
+
+
+
-h, --host \
+
+ Hostname at which to serve. Default is localhost.
+
+
+
+
+
-n, --max-workers, --max_workers \
+
+ Maximum number of (threaded) workers to use in the GRPC server
+
+
+
+
+
--heartbeat
+
+ If set, the GRPC server will shut itself down when it fails to receive a heartbeat after a timeout configurable with –heartbeat-timeout.
+
+
+
+
+
--heartbeat-timeout \
+
+ Timeout after which to shutdown if –heartbeat is set and a heartbeat is not received
+
+
+
+
+
--lazy-load-user-code
+
+ Wait until the first LoadRepositories call to actually load the repositories, instead of waiting to load them when the server is launched. Useful for surfacing errors when the server is managed directly from the Dagster UI.
+
+
+
+
+
-a, --attribute \
+
+ Attribute that is either a 1) repository or job or 2) a function that returns a repository or job
+
+
+
+
+
--package-name \
+
+ Specify Python package where repository or job function lives
+
+
+
+
+
-m, --module-name \
+
+ Specify module where dagster definitions reside as top-level symbols/variables and load the module as a code location in the current python environment.
+
+
+
+
+
-f, --python-file \
+
+ Specify python file where dagster definitions reside as top-level symbols/variables and load the file as a code location in the current python environment.
+
+
+
+
+
-d, --working-directory \
+
+ Specify working directory to use when loading the repository or job
+
+
+
+
+
--use-python-environment-entry-point
+
+ If this flag is set, the server will signal to clients that they should launch dagster commands using \ -m dagster, instead of the default dagster entry point. This is useful when there are multiple Python environments running in the same machine, so a single dagster entry point is not enough to uniquely determine the environment.
+
+
+
+
+
--empty-working-directory
+
+ Indicates that the working directory should be empty and should not set to the current directory as a default
+
+
+
+
+
--fixed-server-id \
+
+ [INTERNAL] This option should generally not be used by users. Internal param used by dagster to spawn a gRPC server with the specified server id.
+
+
+
+
+
--log-level \
+
+
+ Level at which to log output from the code server process
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-format \
+
+
+ Format of the log output from the code server process
+
+ Default: `'colored'`Options: colored | json | rich
+
+
+
+
+
+
--container-image \
+
+ Container image to use to run code from this server.
+
+
+
+
+
--container-context \
+
+ Serialized JSON with configuration for any containers created to run the code from this server.
+
+
+
+
+
--inject-env-vars-from-instance
+
+ Whether to load env vars from the instance and inject them into the environment.
+
+
+
+
+
--location-name \
+
+ Name of the code location this server corresponds to.
+
+
+
+
+
--instance-ref \
+
+ [INTERNAL] Serialized InstanceRef to use for accessing the instance
+
+
+
+
+
--enable-metrics
+
+ [INTERNAL] Retrieves current utilization metrics from GRPC server.
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_GRPC_PORT
+
+ >
+
+ Provide a default for [`--port`](#cmdoption-dagster-api-grpc-p)
+
+
+
+
+
+
+
+
+
+
DAGSTER_GRPC_SOCKET
+
+ >
+
+ Provide a default for [`--socket`](#cmdoption-dagster-api-grpc-s)
+
+
+
+
+
+
+
+
+
+
DAGSTER_GRPC_HOST
+
+ >
+
+ Provide a default for [`--host`](#cmdoption-dagster-api-grpc-h)
+
+
+
+
+
+
+
+
+
+
DAGSTER_GRPC_MAX_WORKERS
+
+ >
+
+ Provide a default for [`--max-workers`](#cmdoption-dagster-api-grpc-n)
+
+
+
+
+
+
+
+
+
+
DAGSTER_LAZY_LOAD_USER_CODE
+
+ >
+
+ Provide a default for [`--lazy-load-user-code`](#cmdoption-dagster-api-grpc-lazy-load-user-code)
+
+
+
+
+
+
+
+
+
+
DAGSTER_ATTRIBUTE
+
+ >
+
+ Provide a default for [`--attribute`](#cmdoption-dagster-api-grpc-a)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PACKAGE_NAME
+
+ >
+
+ Provide a default for [`--package-name`](#cmdoption-dagster-api-grpc-package-name)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-api-grpc-m)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-api-grpc-f)
+
+
+
+
+
+
+
+
+
+
DAGSTER_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--working-directory`](#cmdoption-dagster-api-grpc-d)
+
+
+
+
+
+
+
+
+
+
DAGSTER_USE_PYTHON_ENVIRONMENT_ENTRY_POINT
+
+ >
+
+ Provide a default for [`--use-python-environment-entry-point`](#cmdoption-dagster-api-grpc-use-python-environment-entry-point)
+
+
+
+
+
+
+
+
+
+
DAGSTER_EMPTY_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--empty-working-directory`](#cmdoption-dagster-api-grpc-empty-working-directory)
+
+
+
+
+
+
+
+
+
+
DAGSTER_CONTAINER_IMAGE
+
+ >
+
+ Provide a default for [`--container-image`](#cmdoption-dagster-api-grpc-container-image)
+
+
+
+
+
+
+
+
+
+
DAGSTER_CONTAINER_CONTEXT
+
+ >
+
+ Provide a default for [`--container-context`](#cmdoption-dagster-api-grpc-container-context)
+
+
+
+
+
+
+
+
+
+
DAGSTER_INJECT_ENV_VARS_FROM_INSTANCE
+
+ >
+
+ Provide a default for [`--inject-env-vars-from-instance`](#cmdoption-dagster-api-grpc-inject-env-vars-from-instance)
+
+
+
+
+
+
+
+
+
+
DAGSTER_LOCATION_NAME
+
+ >
+
+ Provide a default for [`--location-name`](#cmdoption-dagster-api-grpc-location-name)
+
+
+
+
+
+
+
+
+
+
DAGSTER_INSTANCE_REF
+
+ >
+
+ Provide a default for [`--instance-ref`](#cmdoption-dagster-api-grpc-instance-ref)
+
+
+
+
+
+
+
+
+
+
DAGSTER_ENABLE_SERVER_METRICS
+
+ >
+
+ Provide a default for [`--enable-metrics`](#cmdoption-dagster-api-grpc-enable-metrics)
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/config.mdx b/docs/docs-beta/docs/api/python-api/config.mdx
new file mode 100644
index 0000000000000..a826f016526e1
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/config.mdx
@@ -0,0 +1,764 @@
+---
+title: 'config'
+title_meta: 'config API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'config Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Config
+
+
+
+
+## Pythonic config system
+
+The following classes are used as part of the new [Pythonic config system](https://docs.dagster.io/guides/operate/configuration/advanced-config-types). They are used in conjunction with builtin types.
+
+
+
class dagster.Config
+
+
+ Base class for Dagster configuration models, used to specify config schema for
+ ops and assets. Subclasses `pydantic.BaseModel`.
+
+ Example definition:
+
+ ```python
+ from pydantic import Field
+
+ class MyAssetConfig(Config):
+ my_str: str = "my_default_string"
+ my_int_list: List[int]
+ my_bool_with_metadata: bool = Field(default=False, description="A bool field")
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def asset_with_config(config: MyAssetConfig):
+ assert config.my_str == "my_default_string"
+ assert config.my_int_list == [1, 2, 3]
+ assert config.my_bool_with_metadata == False
+
+ asset_with_config(MyAssetConfig(my_int_list=[1, 2, 3], my_bool_with_metadata=True))
+ ```
+
+
+
+
+
+
class dagster.PermissiveConfig
+
+
+ Subclass of [`Config`](#dagster.Config) that allows arbitrary extra fields. This is useful for
+ config classes which may have open-ended inputs.
+
+ Example definition:
+
+ ```python
+ class MyPermissiveOpConfig(PermissiveConfig):
+ my_explicit_parameter: bool
+ my_other_explicit_parameter: str
+ ```
+ Example usage:
+
+ ```python
+ @op
+ def op_with_config(config: MyPermissiveOpConfig):
+ assert config.my_explicit_parameter == True
+ assert config.my_other_explicit_parameter == "foo"
+ assert config.dict().get("my_implicit_parameter") == "bar"
+
+ op_with_config(
+ MyPermissiveOpConfig(
+ my_explicit_parameter=True,
+ my_other_explicit_parameter="foo",
+ my_implicit_parameter="bar"
+ )
+ )
+ ```
+
+
+
+
+
+
class dagster.RunConfig
+
+
+ Container for all the configuration that can be passed to a run. Accepts Pythonic definitions
+ for op and asset config and resources and converts them under the hood to the appropriate config dictionaries.
+
+ Example usage:
+
+ ```python
+ class MyAssetConfig(Config):
+ a_str: str
+
+ @asset
+ def my_asset(config: MyAssetConfig):
+ assert config.a_str == "foo"
+
+ materialize(
+ [my_asset],
+ run_config=RunConfig(
+ ops={"my_asset": MyAssetConfig(a_str="foo")}
+ )
+ )
+ ```
+
+
to_config_dict
+
+
+ Converts the RunConfig to a dictionary representation.
+
+ Returns: The dictionary representation of the RunConfig.Return type: Dict[str, Any]
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Legacy Dagster config types
+
+The following types are used as part of the legacy [Dagster config system](https://docs.dagster.io/guides/operate/configuration/). They are used in conjunction with builtin types.
+
+
+
class dagster.ConfigSchema
+
+
+ Placeholder type for config schemas.
+
+ Any time that it appears in documentation, it means that any of the following types are
+ acceptable:
+
+ 1. A Python scalar type that resolves to a Dagster config type
+ (`python:int`, `python:float`, `python:bool`,
+ or `python:str`). For example:
+
+ - `@op(config_schema=int)`
+ - `@op(config_schema=str)`
+
+
+ 2. A built-in python collection (`python:list`, or `python:dict`).
+ `python:list` is exactly equivalent to [`Array`](#dagster.Array) [
+ `Any` ] and `python:dict` is equivalent to
+ [`Permissive`](#dagster.Permissive). For example:
+
+ - `@op(config_schema=list)`
+ - `@op(config_schema=dict)`
+
+
+ 3. A Dagster config type:
+
+ - `Any`
+ - [`Array`](#dagster.Array)
+ - `Bool`
+ - [`Enum`](#dagster.Enum)
+ - `Float`
+ - `Int`
+ - [`IntSource`](#dagster.IntSource)
+ - [`Noneable`](#dagster.Noneable)
+ - [`Permissive`](#dagster.Permissive)
+ - [`Map`](#dagster.Map)
+ - [`ScalarUnion`](#dagster.ScalarUnion)
+ - [`Selector`](#dagster.Selector)
+ - [`Shape`](#dagster.Shape)
+ - `String`
+ - [`StringSource`](#dagster.StringSource)
+
+
+ 4. A bare python dictionary, which will be automatically wrapped in
+ [`Shape`](#dagster.Shape). Values of the dictionary are resolved recursively
+ according to the same rules. For example:
+
+ - `\{'some_config': str}` is equivalent to `Shape(\{'some_config: str})`.
+ - `\{'some_config1': \{'some_config2': str}}` is equivalent to
+
+
+ 5. A bare python list of length one, whose single element will be wrapped in a
+ [`Array`](#dagster.Array) is resolved recursively according to the same
+ rules. For example:
+
+ - `[str]` is equivalent to `Array[str]`.
+ - `[[str]]` is equivalent to `Array[Array[str]]`.
+ - `[\{'some_config': str}]` is equivalent to `Array(Shape(\{'some_config: str}))`.
+
+
+ 6. An instance of [`Field`](#dagster.Field).
+
+
+
+
+
+
class dagster.Field
+
+
+ Defines the schema for a configuration field.
+
+ Fields are used in config schema instead of bare types when one wants to add a description,
+ a default value, or to mark it as not required.
+
+ Config fields are parsed according to their schemas in order to yield values available at
+ job execution time through the config system. Config fields can be set on ops, on
+ loaders for custom, and on other pluggable components of the system, such as resources, loggers,
+ and executors.
+
+ Parameters:
+ - config (Any) –
+
+ The schema for the config. This value can be any of:
+
+ 1. A Python primitive type that resolves to a Dagster config type
+ (`python:int`, `python:float`, `python:bool`,
+ `python:str`, or `python:list`).
+ 2. A Dagster config type:
+
+ - `Any`
+ - [`Array`](#dagster.Array)
+ - `Bool`
+ - [`Enum`](#dagster.Enum)
+ - `Float`
+ - `Int`
+ - [`IntSource`](#dagster.IntSource)
+ - [`Noneable`](#dagster.Noneable)
+ - [`Permissive`](#dagster.Permissive)
+ - [`ScalarUnion`](#dagster.ScalarUnion)
+ - [`Selector`](#dagster.Selector)
+ - [`Shape`](#dagster.Shape)
+ - `String`
+ - [`StringSource`](#dagster.StringSource)
+
+
+ 3. A bare python dictionary, which will be automatically wrapped in
+ [`Shape`](#dagster.Shape). Values of the dictionary are resolved recursively
+ according to the same rules.
+ 4. A bare python list of length one which itself is config type.
+ - default_value (Any) –
+
+ A default value for this field, conformant to the schema set by the `dagster_type`
+ argument. If a default value is provided, `is_required` should be `False`.
+
+ - is_required (bool) – Whether the presence of this field is required. Defaults to true. If `is_required`
+ - description (str) – A human-readable description of this config field.
+
+
+ Examples:
+
+ ```python
+ @op(
+ config_schema={
+ 'word': Field(str, description='I am a word.'),
+ 'repeats': Field(Int, default_value=1, is_required=False),
+ }
+ )
+ def repeat_word(context):
+ return context.op_config['word'] * context.op_config['repeats']
+ ```
+
+
property default_provided
+
+
+ Was a default value provided.
+
+ Returns: Yes or noReturn type: bool
+
+
+
+
+
+
property default_value
+
+
+ The default value for the field.
+
+ Raises an exception if no default value was provided.
+
+
+
+
+
+
+
property description
+
+ A human-readable description of this config field, if provided.
+
+
+
+
+
property is_required
+
+
+ Whether a value for this field must be provided at runtime.
+
+ Cannot be True if a default value is provided.
+
+
+
+
+
+
+
+
+
+
+
class dagster.Selector
+
+
+ Define a config field requiring the user to select one option.
+
+ Selectors are used when you want to be able to present several different options in config but
+ allow only one to be selected. For example, a single input might be read in from either a csv
+ file or a parquet file, but not both at once.
+
+ Note that in some other type systems this might be called an ‘input union’.
+
+ Functionally, a selector is like a `Dict`, except that only one key from the dict can
+ be specified in valid config.
+
+ Parameters: fields (Dict[str, [*Field*](#dagster.Field)]) – The fields from which the user must select.
+ Examples:
+
+ ```python
+ @op(
+ config_schema=Field(
+ Selector(
+ {
+ 'haw': {'whom': Field(String, default_value='honua', is_required=False)},
+ 'cn': {'whom': Field(String, default_value='世界', is_required=False)},
+ 'en': {'whom': Field(String, default_value='world', is_required=False)},
+ }
+ ),
+ is_required=False,
+ default_value={'en': {'whom': 'world'}},
+ )
+ )
+ def hello_world_with_default(context):
+ if 'haw' in context.op_config:
+ return 'Aloha {whom}!'.format(whom=context.op_config['haw']['whom'])
+ if 'cn' in context.op_config:
+ return '你好, {whom}!'.format(whom=context.op_config['cn']['whom'])
+ if 'en' in context.op_config:
+ return 'Hello, {whom}!'.format(whom=context.op_config['en']['whom'])
+ ```
+
+
+
+
+
+
class dagster.Permissive
+
+
+ Defines a config dict with a partially specified schema.
+
+ A permissive dict allows partial specification of the config schema. Any fields with a
+ specified schema will be type checked. Other fields will be allowed, but will be ignored by
+ the type checker.
+
+ Parameters: fields (Dict[str, [*Field*](#dagster.Field)]) – The partial specification of the config dict.
+ Examples:
+
+ ```python
+ @op(config_schema=Field(Permissive({'required': Field(String)})))
+ def map_config_op(context) -> List:
+ return sorted(list(context.op_config.items()))
+ ```
+
+
+
+
+
+
class dagster.Shape
+
+
+ Schema for configuration data with string keys and typed values via [`Field`](#dagster.Field).
+
+ Unlike [`Permissive`](#dagster.Permissive), unspecified fields are not allowed and will throw a
+ [`DagsterInvalidConfigError`](errors.mdx#dagster.DagsterInvalidConfigError).
+
+ Parameters:
+ - fields (Dict[str, [*Field*](#dagster.Field)]) – The specification of the config dict.
+ - field_aliases (Dict[str, str]) – Maps a string key to an alias that can be used instead of the original key. For example,
+
+
+
+
+
+
+
+
class dagster.Map
+
+
+ Defines a config dict with arbitrary scalar keys and typed values.
+
+ A map can contrain arbitrary keys of the specified scalar type, each of which has
+ type checked values. Unlike [`Shape`](#dagster.Shape) and [`Permissive`](#dagster.Permissive), scalar
+ keys other than strings can be used, and unlike [`Permissive`](#dagster.Permissive), all
+ values are type checked.
+
+ Parameters:
+ - key_type (type) – The type of keys this map can contain. Must be a scalar type.
+ - inner_type (type) – The type of the values that this map type can contain.
+ - key_label_name (string) – Optional name which describes the role of keys in the map.
+
+
+ Examples:
+
+ ```python
+ @op(config_schema=Field(Map({str: int})))
+ def partially_specified_config(context) -> List:
+ return sorted(list(context.op_config.items()))
+ ```
+
+
property key_label_name
+
+ Name which describes the role of keys in the map, if provided.
+
+
+
+
+
+
+
+
+
class dagster.Array
+
+
+ Defines an array (list) configuration type that contains values of type `inner_type`.
+
+ Parameters: inner_type (type) – The type of the values that this configuration type can contain.
+
+
property description
+
+ A human-readable description of this Array type.
+
+
+
+
+
+
+
+
+
class dagster.Noneable
+
+
+ Defines a configuration type that is the union of `NoneType` and the type `inner_type`.
+
+ Parameters: inner_type (type) – The type of the values that this configuration type can contain.
+ Examples:
+
+ ```python
+ config_schema={"name": Noneable(str)}
+
+ config={"name": "Hello"} # Ok
+ config={"name": None} # Ok
+ config={} # Error
+ ```
+
+
+
+
+
+
class dagster.Enum
+
+
+ Defines a enum configuration type that allows one of a defined set of possible values.
+
+ Parameters:
+ - name (str) – The name of the enum configuration type.
+ - enum_values (List[[*EnumValue*](#dagster.EnumValue)]) – The set of possible values for the enum configuration type.
+
+
+ Examples:
+
+ ```python
+ @op(
+ config_schema=Field(
+ Enum(
+ 'CowboyType',
+ [
+ EnumValue('good'),
+ EnumValue('bad'),
+ EnumValue('ugly'),
+ ]
+ )
+ )
+ )
+ def resolve_standoff(context):
+ # ...
+ ```
+
+
+
+
+
+
class dagster.EnumValue
+
+
+ Define an entry in a [`Enum`](#dagster.Enum).
+
+ Parameters:
+ - config_value (str) – The string representation of the config to accept when passed.
+ - python_value (Optional[Any]) – The python value to convert the enum entry in to. Defaults to the `config_value`.
+ - description (Optional[str]) – A human-readable description of the enum entry.
+
+
+
+
+
+
+
+
class dagster.ScalarUnion
+
+
+ Defines a configuration type that accepts a scalar value OR a non-scalar value like a
+ `List`, `Dict`, or [`Selector`](#dagster.Selector).
+
+ This allows runtime scalars to be configured without a dictionary with the key `value` and
+ instead just use the scalar value directly. However this still leaves the option to
+ load scalars from a json or pickle file.
+
+ Parameters:
+ - scalar_type (type) – The scalar type of values that this configuration type can hold. For example,
+ - non_scalar_schema ([*ConfigSchema*](#dagster.ConfigSchema)) – The schema of a non-scalar Dagster configuration type. For example, `List`,
+ - key (Optional[str]) – The configuation type’s unique key. If not set, then the key will be set to
+
+
+ Examples:
+
+ ```yaml
+ graph:
+ transform_word:
+ inputs:
+ word:
+ value: foobar
+ ```
+ becomes, optionally,
+
+ ```yaml
+ graph:
+ transform_word:
+ inputs:
+ word: foobar
+ ```
+
+
+
+
+
+
dagster.StringSource
+
+
+ Use this type when you want to read a string config value from an environment variable. The value
+ passed to a config field of this type may either be a string literal, or a selector describing
+ how to look up the value from the executing process’s environment variables.
+
+ Examples:
+
+ ```python
+ from dagster import job, op, StringSource
+
+ @op(config_schema=StringSource)
+ def secret_op(context) -> str:
+ return context.op_config
+
+ @job
+ def secret_job():
+ secret_op()
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_op': {'config': 'test_value'}}
+ }
+ )
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_op': {'config': {'env': 'VERY_SECRET_ENV_VARIABLE'}}}
+ }
+ )
+ ```
+
+
+
+
+
+
dagster.IntSource
+
+
+ Use this type when you want to read an integer config value from an environment variable. The
+ value passed to a config field of this type may either be a integer literal, or a selector
+ describing how to look up the value from the executing process’s environment variables.
+
+ Examples:
+
+ ```python
+ from dagster import job, op, IntSource
+
+ @op(config_schema=IntSource)
+ def secret_int_op(context) -> int:
+ return context.op_config
+
+ @job
+ def secret_job():
+ secret_int_op()
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_int_op': {'config': 1234}}
+ }
+ )
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_int_op': {'config': {'env': 'VERY_SECRET_ENV_VARIABLE_INT'}}}
+ }
+ )
+ ```
+
+
+
+
+
+
dagster.BoolSource
+
+
+ Use this type when you want to read an boolean config value from an environment variable. The
+ value passed to a config field of this type may either be a boolean literal, or a selector
+ describing how to look up the value from the executing process’s environment variables. Set the
+ value of the corresponding environment variable to `""` to indicate `False`.
+
+ Examples:
+
+ ```python
+ from dagster import job, op, BoolSource
+
+ @op(config_schema=BoolSource)
+ def secret_bool_op(context) -> bool:
+ return context.op_config
+
+ @job
+ def secret_job():
+ secret_bool_op()
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_bool_op': {'config': False}}
+ }
+ )
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_bool_op': {'config': {'env': 'VERY_SECRET_ENV_VARIABLE_BOOL'}}}
+ }
+ )
+ ```
+
+
+
+
+
+
+
+
+
+## Config Utilities
+
+
+
class dagster.ConfigMapping
+
+
+ Defines a config mapping for a graph (or job).
+
+ By specifying a config mapping function, you can override the configuration for the child
+ ops and graphs contained within a graph.
+
+ Config mappings require the configuration schema to be specified as `config_schema`, which will
+ be exposed as the configuration schema for the graph, as well as a configuration mapping
+ function, `config_fn`, which maps the config provided to the graph to the config
+ that will be provided to the child nodes.
+
+ Parameters:
+ - config_fn (Callable[[dict], dict]) – The function that will be called
+ - config_schema ([*ConfigSchema*](#dagster.ConfigSchema)) – The schema of the graph config.
+ - receive_processed_config_values (Optional[bool]) – If true, config values provided to the config_fn
+
+
+
+
+
+
+
+
@dagster.configured
+
+
+ A decorator that makes it easy to create a function-configured version of an object.
+
+ The following definition types can be configured using this function:
+
+ - [`GraphDefinition`](graphs.mdx#dagster.GraphDefinition)
+ - [`ExecutorDefinition`](internals.mdx#dagster.ExecutorDefinition)
+ - [`LoggerDefinition`](loggers.mdx#dagster.LoggerDefinition)
+ - [`ResourceDefinition`](resources.mdx#dagster.ResourceDefinition)
+ - [`OpDefinition`](ops.mdx#dagster.OpDefinition)
+
+
+ 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.
+
+ If the config that will be supplied to the object is constant, you may alternatively invoke this
+ and call the result with a dict of config values to be curried. Examples of both strategies
+ below.
+
+ Parameters:
+ - configurable (ConfigurableDefinition) – An object that can be configured.
+ - config_schema ([*ConfigSchema*](#dagster.ConfigSchema)) – The config schema that the inputs to the decorated function
+ - **kwargs – Arbitrary keyword arguments that will be passed to the initializer of the returned
+
+
+ Returns: (Callable[[Union[Any, Callable[[Any], Any]]], ConfigurableDefinition])
+ Examples:
+
+ ```python
+ class GreetingConfig(Config):
+ message: str
+
+ @op
+ def greeting_op(config: GreetingConfig):
+ print(config.message)
+
+ class HelloConfig(Config):
+ name: str
+
+ @configured(greeting_op)
+ def hello_op(config: HelloConfig):
+ return GreetingConfig(message=f"Hello, {config.name}!")
+ ```
+ ```python
+ dev_s3 = configured(S3Resource, name="dev_s3")({'bucket': 'dev'})
+
+ @configured(S3Resource)
+ def dev_s3(_):
+ return {'bucket': 'dev'}
+
+ @configured(S3Resource, {'bucket_prefix', str})
+ def dev_s3(config):
+ return {'bucket': config['bucket_prefix'] + 'dev'}
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/definitions.mdx b/docs/docs-beta/docs/api/python-api/definitions.mdx
new file mode 100644
index 0000000000000..dacdde1256dd9
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/definitions.mdx
@@ -0,0 +1,350 @@
+---
+title: 'definitions'
+title_meta: 'definitions API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'definitions Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Definitions
+
+
+
class dagster.Definitions
+
+
+ A set of definitions explicitly available and loadable by Dagster tools.
+
+ Parameters:
+ - assets (Optional[Iterable[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset), CacheableAssetsDefinition]]]) – A list of assets. Assets can be created by annotating
+ - asset_checks (Optional[Iterable[[*AssetChecksDefinition*](asset-checks.mdx#dagster.AssetChecksDefinition)]]) – A list of asset checks.
+ - schedules (Optional[Iterable[Union[[*ScheduleDefinition*](schedules-sensors.mdx#dagster.ScheduleDefinition), UnresolvedPartitionedAssetScheduleDefinition]]]) – List of schedules.
+ - sensors (Optional[Iterable[[*SensorDefinition*](schedules-sensors.mdx#dagster.SensorDefinition)]]) – List of sensors, typically created with [`@sensor`](schedules-sensors.mdx#dagster.sensor).
+ - jobs (Optional[Iterable[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]]) – List of jobs. Typically created with [`define_asset_job`](assets.mdx#dagster.define_asset_job)
+ - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets.
+ - executor (Optional[Union[[*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition), [*Executor*](internals.mdx#dagster.Executor)]]) – Default executor for jobs. Individual jobs can override this and define their own executors
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.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:
+
+ ```python
+ 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`](#dagster.Definitions).
+
+
+
get_all_asset_specs
+
+
+ :::warning[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`](io-managers.mdx#dagster.IOManager) associated with the assets. Avoids
+ spinning up resources separately for each asset.
+
+ Usage:
+
+ ```python
+ 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 of [`define_asset_job()`](assets.mdx#dagster.define_asset_job)) it will be resolved to a [`JobDefinition`](jobs.mdx#dagster.JobDefinition) when returned
+ from this function, with all resource dependencies fully resolved.
+
+
+
+
+
get_schedule_def
+
+ Get a [`ScheduleDefinition`](schedules-sensors.mdx#dagster.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`](schedules-sensors.mdx#dagster.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`](io-managers.mdx#dagster.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()`](#dagster.Definitions.get_asset_value_loader), which avoids spinning up
+ resources separately for each asset.
+
+ Parameters:
+ - asset_key (Union[[*AssetKey*](assets.mdx#dagster.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`](io-managers.mdx#dagster.IOManager)
+
+
+ Returns: The contents of an asset as a Python object.
+
+
+
+
+
+
static merge
+
+
+ :::warning[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:
+
+ ```python
+ import submodule1
+ import submodule2
+
+ defs = Definitions.merge(submodule1.defs, submodule2.defs)
+ ```
+ Returns: The merged definitions.Return type: [Definitions](#dagster.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
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Create a named repository using the same arguments as [`Definitions`](#dagster.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`](#dagster.Definitions) API and 2) but still
+ want multiple logical groups of assets in the same code location, we have introduced this function.
+
+ Example usage:
+
+ ```python
+ 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
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs the [`dagster.Definitions`](#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`](#dagster.Definitions).
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – Default loggers for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+ - executor (Optional[Union[[*Executor*](internals.mdx#dagster.Executor), [*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]]) – Default executor for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+
+
+ Returns: The [`dagster.Definitions`](#dagster.Definitions) defined in the current module.Return type: [Definitions](#dagster.Definitions)
+
+
+
+
+
+
dagster.load_definitions_from_module
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs the [`dagster.Definitions`](#dagster.Definitions) from the given module.
+
+ Parameters:
+ - module (ModuleType) – The Python module to look for [`dagster.Definitions`](#dagster.Definitions) inside.
+ - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded [`dagster.Definitions`](#dagster.Definitions).
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – Default loggers for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+ - executor (Optional[Union[[*Executor*](internals.mdx#dagster.Executor), [*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]]) – Default executor for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+
+
+ Returns: The [`dagster.Definitions`](#dagster.Definitions) defined in the given module.Return type: [Definitions](#dagster.Definitions)
+
+
+
+
+
+
dagster.load_definitions_from_modules
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs the [`dagster.Definitions`](#dagster.Definitions) from the given modules.
+
+ Parameters:
+ - modules (Iterable[ModuleType]) – The Python modules to look for [`dagster.Definitions`](#dagster.Definitions) inside.
+ - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded [`dagster.Definitions`](#dagster.Definitions).
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – Default loggers for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+ - executor (Optional[Union[[*Executor*](internals.mdx#dagster.Executor), [*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]]) – Default executor for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+
+
+ Returns: The [`dagster.Definitions`](#dagster.Definitions) defined in the given modules.Return type: [Definitions](#dagster.Definitions)
+
+
+
+
+
+
dagster.load_definitions_from_package_module
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs the [`dagster.Definitions`](#dagster.Definitions) from the given package module.
+
+ Parameters:
+ - package_module (ModuleType) – The package module to look for [`dagster.Definitions`](#dagster.Definitions) inside.
+ - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded [`dagster.Definitions`](#dagster.Definitions).
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – Default loggers for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+ - executor (Optional[Union[[*Executor*](internals.mdx#dagster.Executor), [*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]]) – Default executor for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+
+
+ Returns: The [`dagster.Definitions`](#dagster.Definitions) defined in the given package module.Return type: [Definitions](#dagster.Definitions)
+
+
+
+
+
+
dagster.load_definitions_from_package_name
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs the [`dagster.Definitions`](#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`](#dagster.Definitions) inside.
+ - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded [`dagster.Definitions`](#dagster.Definitions).
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – Default loggers for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+ - executor (Optional[Union[[*Executor*](internals.mdx#dagster.Executor), [*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]]) – Default executor for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+
+
+ Returns: The [`dagster.Definitions`](#dagster.Definitions) defined in the package module for the given package name.Return type: [Definitions](#dagster.Definitions)
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/dynamic.mdx b/docs/docs-beta/docs/api/python-api/dynamic.mdx
new file mode 100644
index 0000000000000..84769596f5df6
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/dynamic.mdx
@@ -0,0 +1,105 @@
+---
+title: 'dynamic mapping & collect'
+title_meta: 'dynamic mapping & collect API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dynamic mapping & collect Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+# Dynamic Mapping & Collect
+
+These APIs provide the means for a simple kind of dynamic orchestration — where the work to be orchestrated is determined not at job definition time but at runtime, dependent on data that’s observed as part of job execution.
+
+
+
+
class dagster.DynamicOut
+
+
+ Variant of [`Out`](ops.mdx#dagster.Out) for an output that will dynamically alter the graph at
+ runtime.
+
+ When using in a composition function such as [`@graph`](graphs.mdx#dagster.graph),
+ dynamic outputs must be used with either
+
+ - `map` - clone downstream ops for each separate [`DynamicOut`](#dagster.DynamicOut)
+ - `collect` - gather across all [`DynamicOut`](#dagster.DynamicOut) in to a list
+
+
+ Uses the same constructor as [`Out`](ops.mdx#dagster.Out)
+
+ >
+
+ ```python
+ @op(
+ config_schema={
+ "path": Field(str, default_value=file_relative_path(__file__, "sample"))
+ },
+ out=DynamicOut(str),
+ )
+ def files_in_directory(context):
+ path = context.op_config["path"]
+ dirname, _, filenames = next(os.walk(path))
+ for file in filenames:
+ yield DynamicOutput(os.path.join(dirname, file), mapping_key=_clean(file))
+
+ @job
+ def process_directory():
+ files = files_in_directory()
+
+ # use map to invoke an op on each dynamic output
+ file_results = files.map(process_file)
+
+ # use collect to gather the results in to a list
+ summarize_directory(file_results.collect())
+ ```
+
+
+
+
+
+
+
+
class dagster.DynamicOutput
+
+
+ Variant of [`Output`](ops.mdx#dagster.Output) used to support
+ dynamic mapping & collect. Each `DynamicOutput` produced by an op represents
+ one item in a set that can be processed individually with `map` or gathered
+ with `collect`.
+
+ Each `DynamicOutput` must have a unique `mapping_key` to distinguish it with it’s set.
+
+ Parameters:
+ - value (Any) – The value returned by the compute function.
+ - mapping_key (str) – The key that uniquely identifies this dynamic value relative to its peers.
+ - output_name (Optional[str]) – Name of the corresponding [`DynamicOut`](#dagster.DynamicOut) defined on the op.
+ - metadata (Optional[Dict[str, Union[str, float, int, [*MetadataValue*](metadata.mdx#dagster.MetadataValue)]]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
+
+
+
+
property mapping_key
+
+ The mapping_key that was set for this DynamicOutput at instantiation.
+
+
+
+
+
property output_name
+
+ Name of the [`DynamicOut`](#dagster.DynamicOut) defined on the op that this DynamicOut is associated with.
+
+
+
+
+
property value
+
+ The value that is returned by the compute function for this DynamicOut.
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/errors.mdx b/docs/docs-beta/docs/api/python-api/errors.mdx
new file mode 100644
index 0000000000000..89d15986d8643
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/errors.mdx
@@ -0,0 +1,234 @@
+---
+title: 'errors'
+title_meta: 'errors API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'errors Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Errors
+
+Core Dagster error classes.
+
+All errors thrown by the Dagster framework inherit from [`DagsterError`](#dagster.DagsterError). Users
+should not subclass this base class for their own exceptions.
+
+There is another exception base class, [`DagsterUserCodeExecutionError`](#dagster.DagsterUserCodeExecutionError), which is
+used by the framework in concert with the [`user_code_error_boundary()`](internals.mdx#dagster._core.errors.user_code_error_boundary).
+
+Dagster uses this construct to wrap user code into which it calls. User code can perform arbitrary
+computations and may itself throw exceptions. The error boundary catches these user code-generated
+exceptions, and then reraises them wrapped in a subclass of
+[`DagsterUserCodeExecutionError`](#dagster.DagsterUserCodeExecutionError).
+
+The wrapped exceptions include additional context for the original exceptions, injected by the
+Dagster runtime.
+
+
+
exception dagster.DagsterError
+
+
+ Base class for all errors thrown by the Dagster framework.
+
+ Users should not subclass this base class for their own exceptions.
+
+
+
property is_user_code_error
+
+ Returns true if this error is attributable to user code.
+
+ Indicates that an unexpected error occurred while executing the body of a config mapping
+ function defined in a [`JobDefinition`](jobs.mdx#dagster.JobDefinition) or ~dagster.GraphDefinition during
+ config parsing.
+
+
+
+
+
exception dagster.DagsterEventLogInvalidForRun
+
+ Raised when the event logs for a historical run are malformed or invalid.
+
+
+ Indicates that you have attempted to construct a config with an invalid value.
+
+ Acceptable values for config types are any of:
+ 1. A Python primitive type that resolves to a Dagster config type
+ (`python:int`, `python:float`, `python:bool`,
+ `python:str`, or `python:list`).
+
+ 2. A Dagster config type: `Int`, `Float`,
+ `Bool`, `String`,
+ [`StringSource`](config.mdx#dagster.StringSource), `Any`,
+ [`Array`](config.mdx#dagster.Array), [`Noneable`](config.mdx#dagster.Noneable), [`Enum`](config.mdx#dagster.Enum),
+ [`Selector`](config.mdx#dagster.Selector), [`Shape`](config.mdx#dagster.Shape), or
+ [`Permissive`](config.mdx#dagster.Permissive).
+
+ 3. A bare python dictionary, which will be automatically wrapped in
+ [`Shape`](config.mdx#dagster.Shape). Values of the dictionary are resolved recursively
+ according to the same rules.
+
+ 4. A bare python list of length one which itself is config type.
+ Becomes [`Array`](config.mdx#dagster.Array) with list element as an argument.
+
+ 5. An instance of [`Field`](config.mdx#dagster.Field).
+
+
+
+
+
+
exception dagster.DagsterInvalidDefinitionError
+
+ Indicates that the rules for a definition have been violated by the user.
+
+
+
+
+
exception dagster.DagsterInvalidSubsetError
+
+ Indicates that a subset of a pipeline is invalid because either:
+ - One or more ops in the specified subset do not exist on the job.’
+ - The subset produces an invalid job.
+
+
+
+
+
exception dagster.DagsterInvariantViolationError
+
+ Indicates the user has violated a well-defined invariant that can only be enforced
+ at runtime.
+
+
+
+
+
exception dagster.DagsterResourceFunctionError
+
+ Indicates an error occurred while executing the body of the `resource_fn` in a
+ [`ResourceDefinition`](resources.mdx#dagster.ResourceDefinition) during resource initialization.
+
+
+
+
+
exception dagster.DagsterRunNotFoundError
+
+ Thrown when a run cannot be found in run storage.
+
+
+
+
+
exception dagster.DagsterStepOutputNotFoundError
+
+ Indicates that previous step outputs required for an execution step to proceed are not
+ available.
+
+
+
+
+
exception dagster.DagsterSubprocessError
+
+ An exception has occurred in one or more of the child processes dagster manages.
+ This error forwards the message and stack trace for all of the collected errors.
+
+
+
+
+
exception dagster.DagsterTypeCheckDidNotPass
+
+
+ Indicates that a type check failed.
+
+ This is raised when `raise_on_error` is `True` in calls to the synchronous job and
+ graph execution APIs (e.g. graph.execute_in_process(), job.execute_in_process() – typically
+ within a test), and a [`DagsterType`](types.mdx#dagster.DagsterType)’s type check fails by returning either
+ `False` or an instance of [`TypeCheck`](ops.mdx#dagster.TypeCheck) whose `success` member is `False`.
+
+
+
+
+
+
+
exception dagster.DagsterTypeCheckError
+
+ Indicates an error in the op type system at runtime. E.g. a op receives an
+ unexpected input, or produces an output that does not match the type of the output definition.
+
+
+
+
+
exception dagster.DagsterUnknownResourceError
+
+ Indicates that an unknown resource was accessed in the body of an execution step. May often
+ happen by accessing a resource in the compute function of an op without first supplying the
+ op with the correct required_resource_keys argument.
+
+ Indicates the resolved executor is incompatible with the state of other systems
+ such as the [`DagsterInstance`](internals.mdx#dagster.DagsterInstance) or system storage configuration.
+
+
+
+
+
exception dagster.DagsterUserCodeExecutionError
+
+
+ This is the base class for any exception that is meant to wrap an
+ `python:Exception` thrown by user code. It wraps that existing user code.
+ The `original_exc_info` argument to the constructor is meant to be a tuple of the type
+ returned by `sys.exc_info` at the call site of the constructor.
+
+ Users should not subclass this base class for their own exceptions and should instead throw
+ freely from user code. User exceptions will be automatically wrapped and rethrown.
+
+
+
property is_user_code_error
+
+ Returns true if this error is attributable to user code.
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/execution.mdx b/docs/docs-beta/docs/api/python-api/execution.mdx
new file mode 100644
index 0000000000000..e871596744f60
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/execution.mdx
@@ -0,0 +1,3797 @@
+---
+title: 'execution'
+title_meta: 'execution API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'execution Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Execution
+
+
+
+
+## Materializing Assets
+
+
+
dagster.materialize
+
+
+ Executes a single-threaded, in-process run which materializes provided assets.
+
+ By default, will materialize assets to the local filesystem.
+
+ Parameters:
+ - assets (Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*AssetSpec*](assets.mdx#dagster.AssetSpec), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]]) –
+
+ The assets to materialize.
+
+ Unless you’re using deps or non_argument_deps, you must also include all assets that are
+ upstream of the assets that you want to materialize. This is because those upstream
+ asset definitions have information that is needed to load their contents while
+ materializing the downstream assets.
+
+ - resources (Optional[Mapping[str, object]]) – The resources needed for execution. Can provide resource instances
+ - run_config (Optional[Any]) – The run config to use for the run that materializes the assets.
+ - partition_key – (Optional[str])
+ - tags (Optional[Mapping[str, str]]) – Tags for the run.
+ - selection (Optional[Union[str, Sequence[str], Sequence[[*AssetKey*](assets.mdx#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]], [*AssetSelection*](assets.mdx#dagster.AssetSelection)]]) –
+
+ A sub-selection of assets to materialize.
+
+ If not provided, then all assets will be materialized.
+
+
+
+ Returns: The result of the execution.Return type: [ExecuteInProcessResult](#dagster.ExecuteInProcessResult)
+ Examples:
+
+ ```python
+ @asset
+ def asset1():
+ ...
+
+ @asset
+ def asset2(asset1):
+ ...
+
+ # executes a run that materializes asset1 and then asset2
+ materialize([asset1, asset2])
+
+ # executes a run that materializes just asset2, loading its input from asset1
+ materialize([asset1, asset2], selection=[asset2])
+ ```
+
+
+
+
+
+
dagster.materialize_to_memory
+
+
+ Executes a single-threaded, in-process run which materializes provided assets in memory.
+
+ Will explicitly use [`mem_io_manager()`](io-managers.mdx#dagster.mem_io_manager) for all required io manager
+ keys. If any io managers are directly provided using the resources
+ argument, a [`DagsterInvariantViolationError`](errors.mdx#dagster.DagsterInvariantViolationError) will be thrown.
+
+ Parameters:
+ - assets (Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*AssetSpec*](assets.mdx#dagster.AssetSpec), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]]) – The assets to materialize. Can also provide [`SourceAsset`](assets.mdx#dagster.SourceAsset) objects to fill dependencies for asset defs.
+ - run_config (Optional[Any]) – The run config to use for the run that materializes the assets.
+ - resources (Optional[Mapping[str, object]]) – The resources needed for execution. Can provide resource instances
+ - partition_key – (Optional[str])
+ - tags (Optional[Mapping[str, str]]) – Tags for the run.
+ - selection (Optional[Union[str, Sequence[str], Sequence[[*AssetKey*](assets.mdx#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]], [*AssetSelection*](assets.mdx#dagster.AssetSelection)]]) –
+
+ A sub-selection of assets to materialize.
+
+ If not provided, then all assets will be materialized.
+
+
+
+ Returns: The result of the execution.Return type: [ExecuteInProcessResult](#dagster.ExecuteInProcessResult)
+ Examples:
+
+ ```python
+ @asset
+ def asset1():
+ ...
+
+ @asset
+ def asset2(asset1):
+ ...
+
+ # executes a run that materializes asset1 and then asset2
+ materialize([asset1, asset2])
+
+ # executes a run that materializes just asset1
+ materialize([asset1, asset2], selection=[asset1])
+ ```
+
+
+
+
+
+
+
+
+
+
+## Executing Jobs
+
+
+
class dagster.JobDefinition
+
+
+ Defines a Dagster job.
+
+
+
execute_in_process
+
+
+ Execute the Job in-process, gathering results in-memory.
+
+ The executor_def on the Job will be ignored, and replaced with the in-process executor.
+ If using the default io_manager, it will switch from filesystem to in-memory.
+
+ Parameters:
+ - (Optional[Mapping[str (run_config) – The configuration for the run
+ - Any]] – The configuration for the run
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The instance to execute against, an ephemeral one will be used if none provided.
+ - partition_key – (Optional[str])
+ - raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur.
+ - op_selection (Optional[Sequence[str]]) – A list of op selection queries (including single op
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of the job. Input
+ - resources (Optional[Mapping[str, Any]]) – The resources needed if any are required. Can provide resource instances directly,
+
+
+ Returns: [`ExecuteInProcessResult`](#dagster.ExecuteInProcessResult)
+
+
+
+
+
+
run_request_for_partition
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.0.
+ Directly instantiate `RunRequest(partition_key=...)` instead..
+
+ :::
+
+ Creates a RunRequest object for a run that processes the given partition.
+
+ Parameters:
+ - partition_key – The key of the partition to request a run for.
+ - run_key (Optional[str]) – A string key to identify this launched run. For sensors, ensures that
+ - tags (Optional[Dict[str, str]]) – A dictionary of tags (string key-value pairs) to attach
+ - (Optional[Mapping[str (run_config) – Configuration for the run. If the job has
+ - Any]] – Configuration for the run. If the job has
+ - current_time (Optional[datetime]) – Used to determine which time-partitions exist.
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: an object that requests a run to process the given partition.Return type: [RunRequest](schedules-sensors.mdx#dagster.RunRequest)
+
+
+
+
+
+
with_hooks
+
+ Apply a set of hooks to all op instances within the job.
+
+
+
+
+
with_top_level_resources
+
+ Apply a set of resources to all op instances within the job.
+
+
+
+
+
property config_mapping
+
+
+ The config mapping for the job, if it has one.
+
+ A config mapping defines a way to map a top-level config schema to run config for the job.
+
+
+
+
+
+
+
property executor_def
+
+
+ Returns the default [`ExecutorDefinition`](internals.mdx#dagster.ExecutorDefinition) for the job.
+
+ If the user has not specified an executor definition, then this will default to the
+ [`multi_or_in_process_executor()`](#dagster.multi_or_in_process_executor). If a default is specified on the
+ [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to, then that will be used instead.
+
+
+
+
+
+
+
property has_specified_executor
+
+ Returns True if this job has explicitly specified an executor, and False if the executor
+ was inherited through defaults or the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to.
+
+
+
+
+
property has_specified_loggers
+
+ Returns true if the job explicitly set loggers, and False if loggers were inherited
+ through defaults or the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to.
+
+
+
+
+
property loggers
+
+
+ Returns the set of LoggerDefinition objects specified on the job.
+
+ If the user has not specified a mapping of [`LoggerDefinition`](loggers.mdx#dagster.LoggerDefinition) objects, then this
+ will default to the `colored_console_logger()` under the key console. If a default
+ is specified on the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to, then that will
+ be used instead.
+
+
+
+
+
+
+
property partitioned_config
+
+
+ The partitioned config for the job, if it has one.
+
+ A partitioned config defines a way to map partition keys to run config for the job.
+
+
+
+
+
+
+
property partitions_def
+
+
+ Returns the [`PartitionsDefinition`](partitions.mdx#dagster.PartitionsDefinition) for the job, if it has one.
+
+ A partitions definition defines the set of partition keys the job operates on.
+
+
+
+
+
+
+
property resource_defs
+
+
+ Returns the set of ResourceDefinition objects specified on the job.
+
+ This may not be the complete set of resources required by the job, since those can also be
+ provided on the [`Definitions`](definitions.mdx#dagster.Definitions) object the job may be provided to.
+
+
+
+
+
+
+
+
+
+
+
dagster.execute_job
+
+
+ Execute a job synchronously.
+
+ This API represents dagster’s python entrypoint for out-of-process
+ execution. For most testing purposes, `
+ execute_in_process()` will be more suitable, but when wanting to run
+ execution using an out-of-process executor (such as `dagster.
+ multiprocess_executor`), then execute_job is suitable.
+
+ execute_job expects a persistent [`DagsterInstance`](internals.mdx#dagster.DagsterInstance) for
+ execution, meaning the $DAGSTER_HOME environment variable must be set.
+ It also expects a reconstructable pointer to a [`JobDefinition`](jobs.mdx#dagster.JobDefinition) so
+ that it can be reconstructed in separate processes. This can be done by
+ wrapping the `JobDefinition` in a call to `dagster.
+ reconstructable()`.
+
+ ```python
+ from dagster import DagsterInstance, execute_job, job, reconstructable
+
+ @job
+ def the_job():
+ ...
+
+ instance = DagsterInstance.get()
+ result = execute_job(reconstructable(the_job), instance=instance)
+ assert result.success
+ ```
+ If using the [`to_job()`](graphs.mdx#dagster.GraphDefinition.to_job) method to
+ construct the `JobDefinition`, then the invocation must be wrapped in a
+ module-scope function, which can be passed to `reconstructable`.
+
+ ```python
+ from dagster import graph, reconstructable
+
+ @graph
+ def the_graph():
+ ...
+
+ def define_job():
+ return the_graph.to_job(...)
+
+ result = execute_job(reconstructable(define_job), ...)
+ ```
+ Since execute_job is potentially executing outside of the current
+ process, output objects need to be retrieved by use of the provided job’s
+ io managers. Output objects can be retrieved by opening the result of
+ execute_job as a context manager.
+
+ ```python
+ from dagster import execute_job
+
+ with execute_job(...) as result:
+ output_obj = result.output_for_node("some_op")
+ ```
+ `execute_job` can also be used to reexecute a run, by providing a [`ReexecutionOptions`](#dagster.ReexecutionOptions) object.
+
+ ```python
+ from dagster import ReexecutionOptions, execute_job
+
+ instance = DagsterInstance.get()
+
+ options = ReexecutionOptions.from_failure(run_id=failed_run_id, instance)
+ execute_job(reconstructable(job), instance, reexecution_options=options)
+ ```
+ Parameters:
+ - job (ReconstructableJob) – A reconstructable pointer to a [`JobDefinition`](jobs.mdx#dagster.JobDefinition).
+ - instance ([*DagsterInstance*](internals.mdx#dagster.DagsterInstance)) – The instance to execute against.
+ - run_config (Optional[dict]) – The configuration that parametrizes this run, as a dict.
+ - tags (Optional[Dict[str, Any]]) – Arbitrary key-value pairs that will be added to run logs.
+ - raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur.
+ - op_selection (Optional[List[str]]) –
+
+ A list of op selection queries (including single
+ op names) to execute. For example:
+
+ - `['some_op']`: selects `some_op` itself.
+ - `['*some_op']`: select `some_op` and all its ancestors (upstream dependencies).
+ - `['*some_op+++']`: select `some_op`, all its ancestors, and its descendants
+ - `['*some_op', 'other_op_a', 'other_op_b+']`: select `some_op` and all its
+ - reexecution_options (Optional[[*ReexecutionOptions*](#dagster.ReexecutionOptions)]) – Reexecution options to provide to the run, if this run is
+
+
+ Returns: The result of job execution.Return type: [`JobExecutionResult`](#dagster.JobExecutionResult)
+
+
+
+
+
+
class dagster.ReexecutionOptions
+
+
+ Reexecution options for python-based execution in Dagster.
+
+ Parameters:
+ - parent_run_id (str) – The run_id of the run to reexecute.
+ - step_selection (Sequence[str]) –
+
+ The list of step selections to reexecute. Must be a subset or match of the
+ set of steps executed in the original run. For example:
+
+ - `['some_op']`: selects `some_op` itself.
+ - `['*some_op']`: select `some_op` and all its ancestors (upstream dependencies).
+ - `['*some_op+++']`: select `some_op`, all its ancestors, and its descendants
+ - `['*some_op', 'other_op_a', 'other_op_b+']`: select `some_op` and all its
+
+
+
+
+
+
+
+
dagster.instance_for_test
+
+
+ Creates a persistent [`DagsterInstance`](internals.mdx#dagster.DagsterInstance) available within a context manager.
+
+ When a context manager is opened, if no temp_dir parameter is set, a new
+ temporary directory will be created for the duration of the context
+ manager’s opening. If the set_dagster_home parameter is set to True
+ (True by default), the $DAGSTER_HOME environment variable will be
+ overridden to be this directory (or the directory passed in by temp_dir)
+ for the duration of the context manager being open.
+
+ Parameters:
+ - overrides (Optional[Mapping[str, Any]]) – Config to provide to instance (config format follows that typically found in an instance.yaml file).
+ - set_dagster_home (Optional[bool]) – If set to True, the $DAGSTER_HOME environment variable will be
+ - temp_dir (Optional[str]) – The directory to use for storing local artifacts produced by the
+
+
+
+
+
+
+
+
+
+
+
+
+## Executing Graphs
+
+
+
class dagster.GraphDefinition
+
+
+ Defines a Dagster op graph.
+
+ An op graph is made up of
+
+ - Nodes, which can either be an op (the functional unit of computation), or another graph.
+ - Dependencies, which determine how the values produced by nodes as outputs flow from
+
+
+ End users should prefer the [`@graph`](graphs.mdx#dagster.graph) decorator. GraphDefinition is generally
+ intended to be used by framework authors or for programatically generated graphs.
+
+ Parameters:
+ - name (str) – The name of the graph. Must be unique within any [`GraphDefinition`](graphs.mdx#dagster.GraphDefinition)
+ - description (Optional[str]) – A human-readable description of the job.
+ - node_defs (Optional[Sequence[NodeDefinition]]) – The set of ops / graphs used in this graph.
+ - dependencies (Optional[Dict[Union[str, [*NodeInvocation*](graphs.mdx#dagster.NodeInvocation)], Dict[str, [*DependencyDefinition*](graphs.mdx#dagster.DependencyDefinition)]]]) – A structure that declares the dependencies of each op’s inputs on the outputs of other
+ - input_mappings (Optional[Sequence[[*InputMapping*](graphs.mdx#dagster.InputMapping)]]) – Defines the inputs to the nested graph, and
+ - output_mappings (Optional[Sequence[[*OutputMapping*](graphs.mdx#dagster.OutputMapping)]]) – Defines the outputs of the nested graph,
+ - config (Optional[[*ConfigMapping*](config.mdx#dagster.ConfigMapping)]) – Defines the config of the graph, and how its schema maps
+ - tags (Optional[Dict[str, Any]]) – Arbitrary metadata for any execution of the graph.
+ - composition_fn (Optional[Callable]) – The function that defines this graph. Used to generate
+
+
+ Examples:
+
+ ```python
+ @op
+ def return_one():
+ return 1
+
+ @op
+ def add_one(num):
+ return num + 1
+
+ graph_def = GraphDefinition(
+ name='basic',
+ node_defs=[return_one, add_one],
+ dependencies={'add_one': {'num': DependencyDefinition('return_one')}},
+ )
+ ```
+
+
alias
+
+
+ Aliases the graph with a new name.
+
+ Can only be used in the context of a [`@graph`](graphs.mdx#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.alias("my_graph_alias")
+ ```
+
+
+
+
+
+
execute_in_process
+
+
+ Execute this graph in-process, collecting results in-memory.
+
+ Parameters:
+ - run_config (Optional[Mapping[str, Any]]) – Run config to provide to execution. The configuration for the underlying graph
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The instance to execute against, an ephemeral one will be used if none provided.
+ - resources (Optional[Mapping[str, Any]]) – The resources needed if any are required. Can provide resource instances directly,
+ - raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur.
+ - op_selection (Optional[List[str]]) – A list of op selection queries (including single op
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of the graph.
+
+
+ Returns: [`ExecuteInProcessResult`](#dagster.ExecuteInProcessResult)
+
+
+
+
+
+
tag
+
+
+ Attaches the provided tags to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](graphs.mdx#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.tag({"my_tag": "my_value"})
+ ```
+
+
+
+
+
+
to_job
+
+
+ Make this graph in to an executable Job by providing remaining components required for execution.
+
+ Parameters:
+ - name (Optional[str]) – The name for the Job. Defaults to the name of the this graph.
+ - resource_defs (Optional[Mapping [str, object]]) – Resources that are required by this graph for execution.
+ - config –
+
+ Describes how the job is parameterized at runtime.
+
+ If no value is provided, then the schema for the job’s run config is a standard
+ format based on its ops and resources.
+
+ If a dictionary is provided, then it must conform to the standard config schema, and
+ it will be used as the job’s run config for the job whenever the job is executed.
+ The values provided will be viewable and editable in the Dagster UI, so be
+ careful with secrets.
+
+ If a [`ConfigMapping`](config.mdx#dagster.ConfigMapping) object is provided, then the schema for the job’s run config is
+ determined by the config mapping, and the ConfigMapping, which should return
+ configuration in the standard format to configure the job.
+
+ - tags (Optional[Mapping[str, object]]) – A set of key-value tags that annotate the job and can
+ - run_tags (Optional[Mapping[str, object]]) – A set of key-value tags that will be automatically attached to runs launched by this
+ - metadata (Optional[Mapping[str, RawMetadataValue]]) – Arbitrary information that will be attached to the JobDefinition and be viewable in the Dagster UI.
+ - logger_defs (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – A dictionary of string logger identifiers to their implementations.
+ - executor_def (Optional[[*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]) – How this Job will be executed. Defaults to [`multi_or_in_process_executor`](#dagster.multi_or_in_process_executor),
+ - op_retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The default retry policy for all ops in this job.
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines a discrete set of partition
+ - asset_layer (Optional[AssetLayer]) – Top level information about the assets this job
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of a job.
+
+
+ Returns: JobDefinition
+
+
+
+
+
+
with_hooks
+
+
+ Attaches the provided hooks to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](graphs.mdx#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.with_hooks({my_hook})
+ ```
+
+
+
+
+
+
with_retry_policy
+
+
+ Attaches the provided retry policy to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](graphs.mdx#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.with_retry_policy(RetryPolicy(max_retries=5))
+ ```
+
+
+
+
+
+
property config_mapping
+
+
+ The config mapping for the graph, if present.
+
+ By specifying a config mapping function, you can override the configuration for the child nodes contained within a graph.
+
+
+
+
+
+
+
property input_mappings
+
+
+ Input mappings for the graph.
+
+ An input mapping is a mapping from an input of the graph to an input of a child node.
+
+
+
+
+
+
+
property name
+
+ The name of the graph.
+
+
+
+
+
property output_mappings
+
+
+ Output mappings for the graph.
+
+ An output mapping is a mapping from an output of the graph to an output of a child node.
+
+
+
+
+
+
+
property tags
+
+ The tags associated with the graph.
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Execution results
+
+
+
class dagster.ExecuteInProcessResult
+
+
+ Result object returned by in-process testing APIs.
+
+ Users should not instantiate this object directly. Used for retrieving run success, events, and outputs from execution methods that return this object.
+
+ This object is returned by:
+ - [`dagster.GraphDefinition.execute_in_process()`](graphs.mdx#dagster.GraphDefinition.execute_in_process)
+ - [`dagster.JobDefinition.execute_in_process()`](jobs.mdx#dagster.JobDefinition.execute_in_process)
+ - [`dagster.materialize_to_memory()`](#dagster.materialize_to_memory)
+ - [`dagster.materialize()`](#dagster.materialize)
+
+
+
asset_value
+
+
+ Retrieves the value of an asset that was materialized during the execution of the job.
+
+ Parameters: asset_key (CoercibleToAssetKey) – The key of the asset to retrieve.Returns: The value of the retrieved asset.Return type: Any
+
+
+
+
+
+
output_for_node
+
+
+ Retrieves output value with a particular name from the in-process run of the job.
+
+ Parameters:
+ - node_str (str) – Name of the op/graph whose output should be retrieved. If the intended
+ - output_name (Optional[str]) – Name of the output on the op/graph to retrieve. Defaults to
+
+
+ Returns: The value of the retrieved output.Return type: Any
+
+
+
+
+
+
output_value
+
+
+ Retrieves output of top-level job, if an output is returned.
+
+ Parameters: output_name (Optional[str]) – The name of the output to retrieve. Defaults to result,
+ the default output name in dagster.Returns: The value of the retrieved output.Return type: Any
+
+
+
+
+
+
property all_events
+
+
+ All dagster events emitted during execution.
+
+ Type: List[[DagsterEvent](#dagster.DagsterEvent)]
+
+
+
+
+
+
property dagster_run
+
+
+ The Dagster run that was executed.
+
+ Type: [DagsterRun](internals.mdx#dagster.DagsterRun)
+
+
+
+
+
+
property job_def
+
+
+ The job definition that was executed.
+
+ Type: [JobDefinition](jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
property run_id
+
+
+ The run ID of the executed [`DagsterRun`](internals.mdx#dagster.DagsterRun).
+
+ Type: str
+
+
+
+
+
+
+
+
+
+
class dagster.JobExecutionResult
+
+
+ Result object returned by [`dagster.execute_job()`](#dagster.execute_job).
+
+ Used for retrieving run success, events, and outputs from execute_job.
+ Users should not directly instantiate this class.
+
+ Events and run information can be retrieved off of the object directly. In
+ order to access outputs, the ExecuteJobResult object needs to be opened
+ as a context manager, which will re-initialize the resources from
+ execution.
+
+
+
output_for_node
+
+
+ Retrieves output value with a particular name from the run of the job.
+
+ In order to use this method, the ExecuteJobResult object must be opened as a context manager. If this method is used without opening the context manager, it will result in a [`DagsterInvariantViolationError`](errors.mdx#dagster.DagsterInvariantViolationError).
+
+ Parameters:
+ - node_str (str) – Name of the op/graph whose output should be retrieved. If the intended
+ - output_name (Optional[str]) – Name of the output on the op/graph to retrieve. Defaults to
+
+
+ Returns: The value of the retrieved output.Return type: Any
+
+
+
+
+
+
output_value
+
+
+ Retrieves output of top-level job, if an output is returned.
+
+ In order to use this method, the ExecuteJobResult object must be opened as a context manager. If this method is used without opening the context manager, it will result in a [`DagsterInvariantViolationError`](errors.mdx#dagster.DagsterInvariantViolationError). If the top-level job has no output, calling this method will also result in a [`DagsterInvariantViolationError`](errors.mdx#dagster.DagsterInvariantViolationError).
+
+ Parameters: output_name (Optional[str]) – The name of the output to retrieve. Defaults to result,
+ the default output name in dagster.Returns: The value of the retrieved output.Return type: Any
+
+
+
+
+
+
property all_events
+
+
+ List of all events yielded by the job execution.
+
+ Type: Sequence[[DagsterEvent](#dagster.DagsterEvent)]
+
+
+
+
+
+
property dagster_run
+
+
+ The Dagster run that was executed.
+
+ Type: [DagsterRun](internals.mdx#dagster.DagsterRun)
+
+
+
+
+
+
property job_def
+
+
+ The job definition that was executed.
+
+ Type: [JobDefinition](jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
property run_id
+
+
+ The id of the Dagster run that was executed.
+
+ Type: str
+
+
+
+
+
+
+
+
+
+
class dagster.DagsterEvent
+
+
+ Events yielded by op and job execution.
+
+ Users should not instantiate this class.
+
+
+
event_type_value
+
+
+ Value for a DagsterEventType.
+
+ Type: str
+
+
+
+
+
+
job_name
+
+
+ Type: str
+
+
+
+
+
+
node_handle
+
+
+ Type: NodeHandle
+
+
+
+
+
+
step_kind_value
+
+
+ Value for a StepKind.
+
+ Type: str
+
+
+
+
+
+
logging_tags
+
+
+ Type: Dict[str, str]
+
+
+
+
+
+
event_specific_data
+
+
+ Type must correspond to event_type_value.
+
+ Type: Any
+
+
+
+
+
+
message
+
+
+ Type: str
+
+
+
+
+
+
pid
+
+
+ Type: int
+
+
+
+
+
+
step_key
+
+
+ DEPRECATED
+
+ Type: Optional[str]
+
+
+
+
+
+
property asset_key
+
+
+ For events that correspond to a specific asset_key / partition
+ (ASSET_MATERIALIZTION, ASSET_OBSERVATION, ASSET_MATERIALIZATION_PLANNED), returns that
+ asset key. Otherwise, returns None.
+
+ Type: Optional[[AssetKey](assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
property event_type
+
+
+ The type of this event.
+
+ Type: [DagsterEventType](#dagster.DagsterEventType)
+
+
+
+
+
+
property is_asset_materialization_planned
+
+
+ If this event is of type ASSET_MATERIALIZATION_PLANNED.
+
+ Type: bool
+
+
+
+
+
+
property is_asset_observation
+
+
+ If this event is of type ASSET_OBSERVATION.
+
+ Type: bool
+
+
+
+
+
+
property is_engine_event
+
+
+ If this event is of type ENGINE_EVENT.
+
+ Type: bool
+
+
+
+
+
+
property is_expectation_result
+
+
+ If this event is of type STEP_EXPECTATION_RESULT.
+
+ Type: bool
+
+
+
+
+
+
property is_failure
+
+
+ If this event represents the failure of a run or step.
+
+ Type: bool
+
+
+
+
+
+
property is_handled_output
+
+
+ If this event is of type HANDLED_OUTPUT.
+
+ Type: bool
+
+
+
+
+
+
property is_hook_event
+
+
+ If this event relates to the execution of a hook.
+
+ Type: bool
+
+
+
+
+
+
property is_loaded_input
+
+
+ If this event is of type LOADED_INPUT.
+
+ Type: bool
+
+
+
+
+
+
property is_resource_init_failure
+
+
+ If this event is of type RESOURCE_INIT_FAILURE.
+
+ Type: bool
+
+
+
+
+
+
property is_step_event
+
+
+ If this event relates to a specific step.
+
+ Type: bool
+
+
+
+
+
+
property is_step_failure
+
+
+ If this event is of type STEP_FAILURE.
+
+ Type: bool
+
+
+
+
+
+
property is_step_materialization
+
+
+ If this event is of type ASSET_MATERIALIZATION.
+
+ Type: bool
+
+
+
+
+
+
property is_step_restarted
+
+
+ If this event is of type STEP_RESTARTED.
+
+ Type: bool
+
+
+
+
+
+
property is_step_skipped
+
+
+ If this event is of type STEP_SKIPPED.
+
+ Type: bool
+
+
+
+
+
+
property is_step_start
+
+
+ If this event is of type STEP_START.
+
+ Type: bool
+
+
+
+
+
+
property is_step_success
+
+
+ If this event is of type STEP_SUCCESS.
+
+ Type: bool
+
+
+
+
+
+
property is_step_up_for_retry
+
+
+ If this event is of type STEP_UP_FOR_RETRY.
+
+ Type: bool
+
+
+
+
+
+
property is_successful_output
+
+
+ If this event is of type STEP_OUTPUT.
+
+ Type: bool
+
+
+
+
+
+
property partition
+
+
+ For events that correspond to a specific asset_key / partition
+ (ASSET_MATERIALIZTION, ASSET_OBSERVATION, ASSET_MATERIALIZATION_PLANNED), returns that
+ partition. Otherwise, returns None.
+
+ Type: Optional[[AssetKey](assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
+
+
+
+
class dagster.DagsterEventType
+
+ The types of events that may be yielded by op and job execution.
+
+
+
+
+
+
+
+
+
+## Reconstructable jobs
+
+
+
class dagster.reconstructable
+
+
+ Create a `ReconstructableJob` from a
+ function that returns a [`JobDefinition`](jobs.mdx#dagster.JobDefinition)/[`JobDefinition`](jobs.mdx#dagster.JobDefinition),
+ or a function decorated with [`@job`](jobs.mdx#dagster.job).
+
+ When your job must cross process boundaries, e.g., for execution on multiple nodes or
+ in different systems (like `dagstermill`), Dagster must know how to reconstruct the job
+ on the other side of the process boundary.
+
+ Passing a job created with `~dagster.GraphDefinition.to_job` to `reconstructable()`,
+ requires you to wrap that job’s definition in a module-scoped function, and pass that function
+ instead:
+
+ ```python
+ from dagster import graph, reconstructable
+
+ @graph
+ def my_graph():
+ ...
+
+ def define_my_job():
+ return my_graph.to_job()
+
+ reconstructable(define_my_job)
+ ```
+ This function implements a very conservative strategy for reconstruction, so that its behavior
+ is easy to predict, but as a consequence it is not able to reconstruct certain kinds of jobs
+ or jobs, such as those defined by lambdas, in nested scopes (e.g., dynamically within a method
+ call), or in interactive environments such as the Python REPL or Jupyter notebooks.
+
+ If you need to reconstruct objects constructed in these ways, you should use
+ `build_reconstructable_job()` instead, which allows you to
+ specify your own reconstruction strategy.
+
+ Examples:
+
+ ```python
+ from dagster import job, reconstructable
+
+ @job
+ def foo_job():
+ ...
+
+ reconstructable_foo_job = reconstructable(foo_job)
+
+
+ @graph
+ def foo():
+ ...
+
+ def make_bar_job():
+ return foo.to_job()
+
+ reconstructable_bar_job = reconstructable(make_bar_job)
+ ```
+
+
+
+ The default executor for a job.
+
+ This is the executor available by default on a [`JobDefinition`](jobs.mdx#dagster.JobDefinition)
+ that does not provide custom executors. This executor has a multiprocessing-enabled mode, and a
+ single-process mode. By default, multiprocessing mode is enabled. Switching between multiprocess
+ mode and in-process mode can be achieved via config.
+
+ ```yaml
+ execution:
+ config:
+ multiprocess:
+
+
+ execution:
+ config:
+ in_process:
+ ```
+ When using the multiprocess mode, `max_concurrent` and `retries` can also be configured.
+
+ ```yaml
+ execution:
+ config:
+ multiprocess:
+ max_concurrent: 4
+ retries:
+ enabled:
+ ```
+ The `max_concurrent` arg is optional and tells the execution engine how many processes may run
+ concurrently. By default, or if you set `max_concurrent` to be 0, this is the return value of
+ `python:multiprocessing.cpu_count()`.
+
+ When using the in_process mode, then only retries can be configured.
+
+ Execution priority can be configured using the `dagster/priority` tag via op metadata,
+ where the higher the number the higher the priority. 0 is the default and both positive
+ and negative numbers can be used.
+
+
+
+
+
+
+
dagster.in_process_executor ExecutorDefinition
+
+
+ The in-process executor executes all steps in a single process.
+
+ To select it, include the following top-level fragment in config:
+
+ ```yaml
+ execution:
+ in_process:
+ ```
+ Execution priority can be configured using the `dagster/priority` tag via op metadata,
+ where the higher the number the higher the priority. 0 is the default and both positive
+ and negative numbers can be used.
+
+
+
+
+
+
+
dagster.multiprocess_executor ExecutorDefinition
+
+
+ The multiprocess executor executes each step in an individual process.
+
+ Any job that does not specify custom executors will use the multiprocess_executor by default.
+ To configure the multiprocess executor, include a fragment such as the following in your run
+ config:
+
+ ```yaml
+ execution:
+ config:
+ multiprocess:
+ max_concurrent: 4
+ ```
+ The `max_concurrent` arg is optional and tells the execution engine how many processes may run
+ concurrently. By default, or if you set `max_concurrent` to be None or 0, this is the return value of
+ `python:multiprocessing.cpu_count()`.
+
+ Execution priority can be configured using the `dagster/priority` tag via op metadata,
+ where the higher the number the higher the priority. 0 is the default and both positive
+ and negative numbers can be used.
+
+
+
+
+
+
+
+
+
+
+
+## Contexts
+
+
+
class dagster.AssetExecutionContext
+
+
+
+
add_asset_metadata
+
+
+ Add metadata to an asset materialization event. This metadata will be
+ available in the Dagster UI.
+
+ Parameters:
+ - metadata (Mapping[str, Any]) – The metadata to add to the asset
+ - asset_key (Optional[CoercibleToAssetKey]) – The asset key to add metadata to.
+ - partition_key (Optional[str]) – The partition key to add metadata to, if
+
+
+ Examples:
+
+ Adding metadata to the asset materialization event for a single asset:
+
+ ```python
+ import dagster as dg
+
+ @dg.asset
+ def my_asset(context):
+ # Add metadata
+ context.add_asset_metadata({"key": "value"})
+ ```
+ Adding metadata to the asset materialization event for a particular partition of a partitioned asset:
+
+ ```python
+ import dagster as dg
+
+ @dg.asset(partitions_def=dg.StaticPartitionsDefinition(["a", "b"]))
+ def my_asset(context):
+ # Adds metadata to all partitions currently being materialized, since no
+ # partition is specified.
+ context.add_asset_metadata({"key": "value"})
+
+ for partition_key in context.partition_keys:
+ # Add metadata only to the event for partition "a"
+ if partition_key == "a":
+ context.add_asset_metadata({"key": "value"}, partition_key=partition_key)
+ ```
+ Adding metadata to the asset materialization event for a particular asset in a multi-asset.
+
+ ```python
+ import dagster as dg
+
+ @dg.multi_asset(specs=[dg.AssetSpec("asset1"), dg.AssetSpec("asset2")])
+ def my_multi_asset(context):
+ # Add metadata to the materialization event for "asset1"
+ context.add_asset_metadata({"key": "value"}, asset_key="asset1")
+
+ # THIS line will fail since asset key is not specified:
+ context.add_asset_metadata({"key": "value"})
+ ```
+
+
+
+
+
+
add_output_metadata
+
+
+ Add metadata to one of the outputs of an op.
+
+ This can be invoked multiple times per output in the body of an op. If the same key is
+ passed multiple times, the value associated with the last call will be used.
+
+ Parameters:
+ - metadata (Mapping[str, Any]) – The metadata to attach to the output
+ - output_name (Optional[str]) – The name of the output to attach metadata to. If there is only one output on the op, then this argument does not need to be provided. The metadata will automatically be attached to the only output.
+ - mapping_key (Optional[str]) – The mapping key of the output to attach metadata to. If the
+
+
+ Examples:
+
+ ```python
+ from dagster import Out, op
+ from typing import Tuple
+
+ @op
+ def add_metadata(context):
+ context.add_output_metadata({"foo", "bar"})
+ return 5 # Since the default output is called "result", metadata will be attached to the output "result".
+
+ @op(out={"a": Out(), "b": Out()})
+ def add_metadata_two_outputs(context) -> Tuple[str, int]:
+ context.add_output_metadata({"foo": "bar"}, output_name="b")
+ context.add_output_metadata({"baz": "bat"}, output_name="a")
+
+ return ("dog", 5)
+ ```
+
+
+
+
+
+
asset_key_for_input
+
+ Return the AssetKey for the corresponding input.
+
+
+
+
+
asset_key_for_output
+
+ Return the AssetKey for the corresponding output.
+
+
+
+
+
asset_partition_key_for_input
+
+
+ Returns the partition key of the upstream asset corresponding to the given input.
+
+ Parameters: input_name (str) – The name of the input to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_for_input("upstream_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_for_input("self_dependent_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-20"
+ ```
+
+
+
+
+
+
asset_partition_key_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method asset_partition_key_for_output on AssetExecutionContext. Use context.partition_key instead..
+
+ :::
+
+ Returns the asset partition key for the given output.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_for_output())
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_for_output("first_asset"))
+ context.log.info(context.asset_partition_key_for_output("second_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ # "2023-08-21"
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ ```
+
+
+
+
+
+
asset_partition_key_range_for_input
+
+
+ Return the PartitionKeyRange for the corresponding input. Errors if the asset depends on a
+ non-contiguous chunk of the input.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_key_range_for_input` to get the range of partitions keys of the input that
+ are relevant to that backfill.
+
+ Parameters: input_name (str) – The name of the input to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_range_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_range_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-20", end="2023-08-24")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_range_for_input("self_dependent_asset"))
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-20", end="2023-08-24")
+ ```
+
+
+
+
+
+
asset_partition_key_range_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method asset_partition_key_range_for_output on AssetExecutionContext. Use context.partition_key_range instead..
+
+ :::
+
+ Return the PartitionKeyRange for the corresponding output. Errors if the run is not partitioned.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_key_range_for_output` to get all of the partitions being materialized
+ by the backfill.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition key range for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_range_for_output())
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_range_for_output("first_asset"))
+ context.log.info(context.asset_partition_key_range_for_output("second_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_range_for_output())
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ ```
+
+
+
+
+
+
asset_partition_keys_for_input
+
+
+ Returns a list of the partition keys of the upstream asset corresponding to the
+ given input.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_keys_for_input` to get all of the partition keys of the input that
+ are relevant to that backfill.
+
+ Parameters: input_name (str) – The name of the input to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_keys_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_keys_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-20", "2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24"]
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_keys_for_input("self_dependent_asset"))
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-20", "2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24"]
+ ```
+
+
+
+
+
+
asset_partition_keys_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method asset_partition_keys_for_output on AssetExecutionContext. Use context.partition_keys instead..
+
+ :::
+
+ Returns a list of the partition keys for the given output.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_keys_for_output` to get all of the partitions being materialized
+ by the backfill.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition keys for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_keys_for_output())
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_keys_for_output("first_asset"))
+ context.log.info(context.asset_partition_keys_for_output("second_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_keys_for_output())
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ ```
+
+
+
+
+
+
asset_partitions_def_for_input
+
+
+ The PartitionsDefinition on the upstream asset corresponding to this input.
+
+ Parameters: input_name (str) – The name of the input to get the PartitionsDefinition for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_def_for_input("upstream_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+ ```
+
+
+
+
+
+
asset_partitions_def_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method asset_partitions_def_for_output on AssetExecutionContext. Use context.assets_def.partitions_def instead..
+
+ :::
+
+ The PartitionsDefinition on the asset corresponding to this output.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the PartitionsDefinition for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_def_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_def_for_output("first_asset"))
+ context.log.info(context.asset_partitions_def_for_output("second_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+ # DailyPartitionsDefinition("2023-08-20")
+ ```
+
+
+
+
+
+
asset_partitions_time_window_for_input
+
+
+ The time window for the partitions of the input asset.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partitions_time_window_for_input` to get the time window of the input that
+ are relevant to that backfill.
+
+ Raises an error if either of the following are true:
+ - The input asset has no partitioning.
+ - The input asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+ Parameters: input_name (str) – The name of the input to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("upstream_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("upstream_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-21")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("self_dependent_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-21")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-25")
+ ```
+
+
+
+
+
+
asset_partitions_time_window_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method asset_partitions_time_window_for_output on AssetExecutionContext. Use context.partition_time_window instead..
+
+ :::
+
+ The time window for the partitions of the output asset.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partitions_time_window_for_output` to get the TimeWindow of all of the partitions
+ being materialized by the backfill.
+
+ Raises an error if either of the following are true:
+ - The output asset has no partitioning.
+ - The output asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_time_window_for_output())
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_time_window_for_output("first_asset"))
+ context.log.info(context.asset_partitions_time_window_for_output("second_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partitions_time_window_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+ ```
+
+
+
+
+
+
get_asset_provenance
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Return the provenance information for the most recent materialization of an asset.
+
+ Parameters: asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – Key of the asset for which to retrieve provenance.Returns:
+ Provenance information for the most recent
+ materialization of the asset. Returns None if the asset was never materialized or
+ the materialization record is too old to contain provenance information.
+
+ Return type: Optional[DataProvenance]
+
+
+
+
+
+
get_mapping_key
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method get_mapping_key on AssetExecutionContext. Use context.op_execution_context.get_mapping_key instead..
+
+ :::
+
+ Which mapping_key this execution is for if downstream of a DynamicOutput, otherwise None.
+
+
+
+
+
+
+
get_tag
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method get_tag on AssetExecutionContext. Use context.run.tags.get(key) instead..
+
+ :::
+
+ Get a logging tag.
+
+ Parameters: key (tag) – The tag to get.Returns: The value of the tag, if present.Return type: Optional[str]
+
+
+
+
+
+
has_tag
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method has_tag on AssetExecutionContext. Use key in context.run.tags instead..
+
+ :::
+
+ Check if a logging tag is set.
+
+ Parameters: key (str) – The tag to check.Returns: Whether the tag is set.Return type: bool
+
+
+
+
+
+
log_event
+
+
+ Log an AssetMaterialization, AssetObservation, or ExpectationResult from within the body of an op.
+
+ Events logged with this method will appear in the list of DagsterEvents, as well as the event log.
+
+ Parameters: event (Union[[*AssetMaterialization*](ops.mdx#dagster.AssetMaterialization), [*AssetObservation*](assets.mdx#dagster.AssetObservation), [*ExpectationResult*](ops.mdx#dagster.ExpectationResult)]) – The event to log.
+ Examples:
+
+ ```python
+ from dagster import op, AssetMaterialization
+
+ @op
+ def log_materialization(context):
+ context.log_event(AssetMaterialization("foo"))
+ ```
+
+
+
+
+
+
output_for_asset_key
+
+ Return the output name for the corresponding asset key.
+
+
+
+
+
property asset_key
+
+ The AssetKey for the current asset. In a multi_asset, use asset_key_for_output instead.
+
+
+
+
+
property asset_partition_key_range
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_key_range` instead..
+
+ :::
+
+ The range of partition keys for the current run.
+
+ If run is for a single partition key, return a PartitionKeyRange with the same start and
+ end. Raises an error if the current run is not a partitioned run.
+
+
+
+
+
+
+
property assets_def
+
+ The backing AssetsDefinition for what is currently executing, errors if not available.
+
+
+
+
+
property has_assets_def
+
+ If there is a backing AssetsDefinition for what is currently executing.
+
+
+
+
+
property has_partition_key
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property has_partition_key_range
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property instance
+
+
+ The current Dagster instance.
+
+ Type: [DagsterInstance](internals.mdx#dagster.DagsterInstance)
+
+
+
+
+
+
property job_def
+
+ The definition for the currently executing job. Information like the job name, and job tags
+ can be found on the JobDefinition.
+ Returns: JobDefinition.
+
+
+
+
+
property job_name
+
+
+ The name of the currently executing pipeline.
+
+ Type: str
+
+
+
+
+
+
property log
+
+
+ The log manager available in the execution context. Logs will be viewable in the Dagster UI.
+ Returns: DagsterLogManager.
+
+ Example:
+
+ ```python
+ @asset
+ def logger(context):
+ context.log.info("Info level message")
+ ```
+
+
+
+
+
+
property op_config
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method op_config on AssetExecutionContext. Use context.op_execution_context.op_config instead..
+
+ :::
+
+ The parsed config specific to this op.
+
+ Type: Any
+
+
+
+
+
+
property op_def
+
+
+ The current op definition.
+
+ Type: [OpDefinition](ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
property partition_key
+
+
+ The partition key for the current run.
+
+ Raises an error if the current run is not a partitioned run. Or if the current run is operating
+ over a range of partitions (ie. a backfill of several partitions executed in a single run).
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_key)
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ ```
+
+
+
+
+
+
property partition_key_range
+
+
+ The range of partition keys for the current run.
+
+ If run is for a single partition key, returns a PartitionKeyRange with the same start and
+ end. Raises an error if the current run is not a partitioned run.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_key_range)
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ ```
+
+
+
+
+
+
property partition_keys
+
+
+ Returns a list of the partition keys for the current run.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `partition_keys` to get all of the partitions being materialized
+ by the backfill.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(partitions_def=partitions_def)
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_keys)
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ ```
+
+
+
+
+
+
property partition_time_window
+
+
+ The partition time window for the current run.
+
+ Raises an error if the current run is not a partitioned run, or if the job’s partition
+ definition is not a TimeWindowPartitionsDefinition.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_time_window)
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+ ```
+
+
+
+
+
+
property pdb
+
+
+ Gives access to pdb debugging from within the asset. Materializing the asset via the
+ Dagster UI or CLI will enter the pdb debugging context in the process used to launch the UI or
+ run the CLI.
+
+ Returns: dagster.utils.forked_pdb.ForkedPdb
+
+ Example:
+
+ ```python
+ @asset
+ def debug(context):
+ context.pdb.set_trace()
+ ```
+
+
+
+
+
+
property resources
+
+
+ The currently available resources.
+
+ Type: Resources
+
+
+
+
+
+
property selected_asset_check_keys
+
+ Get the asset check keys that correspond to the current selection of assets this execution is expected to materialize.
+
+
+
+
+
property selected_asset_keys
+
+ Get the set of AssetKeys this execution is expected to materialize.
+
+
+
+
+
property selected_output_names
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method selected_output_names on AssetExecutionContext. Use context.op_execution_context.selected_output_names instead..
+
+ :::
+
+ Get the output names that correspond to the current selection of assets this execution is expected to materialize.
+
+
+
+
+
+
+
+
+
+
+
class dagster.OpExecutionContext
+
+
+ The `context` object that can be made available as the first argument to the function
+ used for computing an op or asset.
+
+ This context object provides system information such as resources, config, and logging.
+
+ To construct an execution context for testing purposes, use [`dagster.build_op_context()`](#dagster.build_op_context).
+
+ Example:
+
+ ```python
+ from dagster import op, OpExecutionContext
+
+ @op
+ def hello_world(context: OpExecutionContext):
+ context.log.info("Hello, world!")
+ ```
+
+
add_output_metadata
+
+
+ Add metadata to one of the outputs of an op.
+
+ This can be invoked multiple times per output in the body of an op. If the same key is
+ passed multiple times, the value associated with the last call will be used.
+
+ Parameters:
+ - metadata (Mapping[str, Any]) – The metadata to attach to the output
+ - output_name (Optional[str]) – The name of the output to attach metadata to. If there is only one output on the op, then this argument does not need to be provided. The metadata will automatically be attached to the only output.
+ - mapping_key (Optional[str]) – The mapping key of the output to attach metadata to. If the
+
+
+ Examples:
+
+ ```python
+ from dagster import Out, op
+ from typing import Tuple
+
+ @op
+ def add_metadata(context):
+ context.add_output_metadata({"foo", "bar"})
+ return 5 # Since the default output is called "result", metadata will be attached to the output "result".
+
+ @op(out={"a": Out(), "b": Out()})
+ def add_metadata_two_outputs(context) -> Tuple[str, int]:
+ context.add_output_metadata({"foo": "bar"}, output_name="b")
+ context.add_output_metadata({"baz": "bat"}, output_name="a")
+
+ return ("dog", 5)
+ ```
+
+
+
+
+
+
asset_key_for_input
+
+ Return the AssetKey for the corresponding input.
+
+
+
+
+
asset_key_for_output
+
+ Return the AssetKey for the corresponding output.
+
+
+
+
+
asset_partition_key_for_input
+
+
+ Returns the partition key of the upstream asset corresponding to the given input.
+
+ Parameters: input_name (str) – The name of the input to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_for_input("upstream_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_for_input("self_dependent_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-20"
+ ```
+
+
+
+
+
+
asset_partition_key_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_key` instead..
+
+ :::
+
+ Returns the asset partition key for the given output.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_for_output())
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_for_output("first_asset"))
+ context.log.info(context.asset_partition_key_for_output("second_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ # "2023-08-21"
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ ```
+
+
+
+
+
+
asset_partition_key_range_for_input
+
+
+ Return the PartitionKeyRange for the corresponding input. Errors if the asset depends on a
+ non-contiguous chunk of the input.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_key_range_for_input` to get the range of partitions keys of the input that
+ are relevant to that backfill.
+
+ Parameters: input_name (str) – The name of the input to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_range_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_range_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-20", end="2023-08-24")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_range_for_input("self_dependent_asset"))
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-20", end="2023-08-24")
+ ```
+
+
+
+
+
+
asset_partition_key_range_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_key_range` instead..
+
+ :::
+
+ Return the PartitionKeyRange for the corresponding output. Errors if the run is not partitioned.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_key_range_for_output` to get all of the partitions being materialized
+ by the backfill.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition key range for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_range_for_output())
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_range_for_output("first_asset"))
+ context.log.info(context.asset_partition_key_range_for_output("second_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_range_for_output())
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ ```
+
+
+
+
+
+
asset_partition_keys_for_input
+
+
+ Returns a list of the partition keys of the upstream asset corresponding to the
+ given input.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_keys_for_input` to get all of the partition keys of the input that
+ are relevant to that backfill.
+
+ Parameters: input_name (str) – The name of the input to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_keys_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_keys_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-20", "2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24"]
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_keys_for_input("self_dependent_asset"))
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-20", "2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24"]
+ ```
+
+
+
+
+
+
asset_partition_keys_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_keys` instead..
+
+ :::
+
+ Returns a list of the partition keys for the given output.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_keys_for_output` to get all of the partitions being materialized
+ by the backfill.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition keys for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_keys_for_output())
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_keys_for_output("first_asset"))
+ context.log.info(context.asset_partition_keys_for_output("second_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_keys_for_output())
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ ```
+
+
+
+
+
+
asset_partitions_def_for_input
+
+
+ The PartitionsDefinition on the upstream asset corresponding to this input.
+
+ Parameters: input_name (str) – The name of the input to get the PartitionsDefinition for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_def_for_input("upstream_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+ ```
+
+
+
+
+
+
asset_partitions_def_for_output
+
+
+ The PartitionsDefinition on the asset corresponding to this output.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the PartitionsDefinition for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_def_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_def_for_output("first_asset"))
+ context.log.info(context.asset_partitions_def_for_output("second_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+ # DailyPartitionsDefinition("2023-08-20")
+ ```
+
+
+
+
+
+
asset_partitions_time_window_for_input
+
+
+ The time window for the partitions of the input asset.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partitions_time_window_for_input` to get the time window of the input that
+ are relevant to that backfill.
+
+ Raises an error if either of the following are true:
+ - The input asset has no partitioning.
+ - The input asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+ Parameters: input_name (str) – The name of the input to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("upstream_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("upstream_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-21")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("self_dependent_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-21")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-25")
+ ```
+
+
+
+
+
+
asset_partitions_time_window_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_time_window` instead..
+
+ :::
+
+ The time window for the partitions of the output asset.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partitions_time_window_for_output` to get the TimeWindow of all of the partitions
+ being materialized by the backfill.
+
+ Raises an error if either of the following are true:
+ - The output asset has no partitioning.
+ - The output asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_time_window_for_output())
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_time_window_for_output("first_asset"))
+ context.log.info(context.asset_partitions_time_window_for_output("second_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partitions_time_window_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+ ```
+
+
+
+
+
+
get_asset_provenance
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Return the provenance information for the most recent materialization of an asset.
+
+ Parameters: asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – Key of the asset for which to retrieve provenance.Returns:
+ Provenance information for the most recent
+ materialization of the asset. Returns None if the asset was never materialized or
+ the materialization record is too old to contain provenance information.
+
+ Return type: Optional[DataProvenance]
+
+
+
+
+
+
get_mapping_key
+
+ Which mapping_key this execution is for if downstream of a DynamicOutput, otherwise None.
+
+
+
+
+
get_tag
+
+
+ Get a logging tag.
+
+ Parameters: key (tag) – The tag to get.Returns: The value of the tag, if present.Return type: Optional[str]
+
+
+
+
+
+
has_tag
+
+
+ Check if a logging tag is set.
+
+ Parameters: key (str) – The tag to check.Returns: Whether the tag is set.Return type: bool
+
+
+
+
+
+
log_event
+
+
+ Log an AssetMaterialization, AssetObservation, or ExpectationResult from within the body of an op.
+
+ Events logged with this method will appear in the list of DagsterEvents, as well as the event log.
+
+ Parameters: event (Union[[*AssetMaterialization*](ops.mdx#dagster.AssetMaterialization), [*AssetObservation*](assets.mdx#dagster.AssetObservation), [*ExpectationResult*](ops.mdx#dagster.ExpectationResult)]) – The event to log.
+ Examples:
+
+ ```python
+ from dagster import op, AssetMaterialization
+
+ @op
+ def log_materialization(context):
+ context.log_event(AssetMaterialization("foo"))
+ ```
+
+
+
+
+
+
output_for_asset_key
+
+ Return the output name for the corresponding asset key.
+
+
+
+
+
property asset_key
+
+ The AssetKey for the current asset. In a multi_asset, use asset_key_for_output instead.
+
+
+
+
+
property asset_partition_key_range
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_key_range` instead..
+
+ :::
+
+ The range of partition keys for the current run.
+
+ If run is for a single partition key, return a PartitionKeyRange with the same start and
+ end. Raises an error if the current run is not a partitioned run.
+
+
+
+
+
+
+
property assets_def
+
+ The backing AssetsDefinition for what is currently executing, errors if not available.
+
+
+
+
+
property has_assets_def
+
+ If there is a backing AssetsDefinition for what is currently executing.
+
+
+
+
+
property has_partition_key
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property has_partition_key_range
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property instance
+
+
+ The current Dagster instance.
+
+ Type: [DagsterInstance](internals.mdx#dagster.DagsterInstance)
+
+
+
+
+
+
property job_def
+
+
+ The currently executing job.
+
+ Type: [JobDefinition](jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
property job_name
+
+
+ The name of the currently executing pipeline.
+
+ Type: str
+
+
+
+
+
+
property log
+
+
+ The log manager available in the execution context.
+
+ Type: [DagsterLogManager](loggers.mdx#dagster.DagsterLogManager)
+
+
+
+
+
+
property op_config
+
+
+ The parsed config specific to this op.
+
+ Type: Any
+
+
+
+
+
+
property op_def
+
+
+ The current op definition.
+
+ Type: [OpDefinition](ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
property partition_key
+
+
+ The partition key for the current run.
+
+ Raises an error if the current run is not a partitioned run. Or if the current run is operating
+ over a range of partitions (ie. a backfill of several partitions executed in a single run).
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_key)
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ ```
+
+
+
+
+
+
property partition_key_range
+
+
+ The range of partition keys for the current run.
+
+ If run is for a single partition key, returns a PartitionKeyRange with the same start and
+ end. Raises an error if the current run is not a partitioned run.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_key_range)
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ ```
+
+
+
+
+
+
property partition_keys
+
+
+ Returns a list of the partition keys for the current run.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `partition_keys` to get all of the partitions being materialized
+ by the backfill.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(partitions_def=partitions_def)
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_keys)
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ ```
+
+
+
+
+
+
property partition_time_window
+
+
+ The partition time window for the current run.
+
+ Raises an error if the current run is not a partitioned run, or if the job’s partition
+ definition is not a TimeWindowPartitionsDefinition.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_time_window)
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+ ```
+
+
+
+
+
+
property pdb
+
+
+ Gives access to pdb debugging from within the op.
+
+ Example:
+
+ ```python
+ @op
+ def debug(context):
+ context.pdb.set_trace()
+ ```
+ Type: dagster.utils.forked_pdb.ForkedPdb
+
+
+
+
+
+
property resources
+
+
+ The currently available resources.
+
+ Type: Resources
+
+
+
+
+
+
property retry_number
+
+ Which retry attempt is currently executing i.e. 0 for initial attempt, 1 for first retry, etc.
+
+
+
+
+
property run
+
+
+ The current run.
+
+ Type: [DagsterRun](internals.mdx#dagster.DagsterRun)
+
+
+
+
+
+
property run_config
+
+
+ The run config for the current execution.
+
+ Type: dict
+
+
+
+
+
+
property run_id
+
+
+ The id of the current execution’s run.
+
+ Type: str
+
+
+
+
+
+
property selected_asset_check_keys
+
+ Get the asset check keys that correspond to the current selection of assets this execution is expected to materialize.
+
+
+
+
+
property selected_asset_keys
+
+ Get the set of AssetKeys this execution is expected to materialize.
+
+
+
+
+
property selected_output_names
+
+ Get the output names that correspond to the current selection of assets this execution is expected to materialize.
+
+
+
+
+
+
+
+
+
dagster.build_op_context
+
+
+ Builds op execution context from provided parameters.
+
+ `build_op_context` can be used as either a function or context manager. If there is a
+ provided resource that is a context manager, then `build_op_context` must be used as a
+ context manager. This function can be used to provide the context argument when directly
+ invoking a op.
+
+ Parameters:
+ - resources (Optional[Dict[str, Any]]) – The resources to provide to the context. These can be
+ - op_config (Optional[Mapping[str, Any]]) – The config to provide to the op.
+ - resources_config (Optional[Mapping[str, Any]]) – The config to provide to the resources.
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The dagster instance configured for the context.
+ - mapping_key (Optional[str]) – A key representing the mapping key from an upstream dynamic
+ - partition_key (Optional[str]) – String value representing partition key to execute with.
+ - partition_key_range (Optional[[*PartitionKeyRange*](partitions.mdx#dagster.PartitionKeyRange)]) – Partition key range to execute with.
+ - run_tags – Optional[Mapping[str, str]]: The tags for the executing run.
+
+
+ Examples:
+
+ ```python
+ context = build_op_context()
+ op_to_invoke(context)
+
+ with build_op_context(resources={"foo": context_manager_resource}) as context:
+ op_to_invoke(context)
+ ```
+
+
+
+
+
+
dagster.build_asset_context
+
+
+ Builds asset execution context from provided parameters.
+
+ `build_asset_context` can be used as either a function or context manager. If there is a
+ provided resource that is a context manager, then `build_asset_context` must be used as a
+ context manager. This function can be used to provide the context argument when directly
+ invoking an asset.
+
+ Parameters:
+ - resources (Optional[Dict[str, Any]]) – The resources to provide to the context. These can be
+ - resources_config (Optional[Mapping[str, Any]]) – The config to provide to the resources.
+ - asset_config (Optional[Mapping[str, Any]]) – The config to provide to the asset.
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The dagster instance configured for the context.
+ - partition_key (Optional[str]) – String value representing partition key to execute with.
+ - partition_key_range (Optional[[*PartitionKeyRange*](partitions.mdx#dagster.PartitionKeyRange)]) – Partition key range to execute with.
+ - run_tags – Optional[Mapping[str, str]]: The tags for the executing run.
+
+
+ Examples:
+
+ ```python
+ context = build_asset_context()
+ asset_to_invoke(context)
+
+ with build_asset_context(resources={"foo": context_manager_resource}) as context:
+ asset_to_invoke(context)
+ ```
+
+
+
+
+
+
class dagster.TypeCheckContext
+
+
+ The `context` object available to a type check function on a DagsterType.
+
+
+
property log
+
+ Centralized log dispatch from user code.
+
+
+
+
+
property resources
+
+ An object whose attributes contain the resources available to this op.
+
+
+
+
+
property run_id
+
+ The id of this job run.
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Job configuration
+
+
+
dagster.validate_run_config
+
+
+ Function to validate a provided run config blob against a given job.
+
+ If validation is successful, this function will return a dictionary representation of the
+ validated config actually used during execution.
+
+ Parameters:
+ - job_def ([*JobDefinition*](jobs.mdx#dagster.JobDefinition)) – The job definition to validate run
+ - run_config (Optional[Dict[str, Any]]) – The run config to validate
+
+
+ Returns: A dictionary representation of the validated config.Return type: Dict[str, Any]
+
+
+
+
+
+
+
+### Run Config Schema
+
+>
+
+The `run_config` used for jobs has the following schema:
+
+ ```default
+ {
+ # configuration for execution, required if executors require config
+ execution: {
+ # the name of one, and only one available executor, typically 'in_process' or 'multiprocess'
+ __executor_name__: {
+ # executor-specific config, if required or permitted
+ config: {
+ ...
+ }
+ }
+ },
+
+ # configuration for loggers, required if loggers require config
+ loggers: {
+ # the name of an available logger
+ __logger_name__: {
+ # logger-specific config, if required or permitted
+ config: {
+ ...
+ }
+ },
+ ...
+ },
+
+ # configuration for resources, required if resources require config
+ resources: {
+ # the name of a resource
+ __resource_name__: {
+ # resource-specific config, if required or permitted
+ config: {
+ ...
+ }
+ },
+ ...
+ },
+
+ # configuration for underlying ops, required if ops require config
+ ops: {
+
+ # these keys align with the names of the ops, or their alias in this job
+ __op_name__: {
+
+ # pass any data that was defined via config_field
+ config: ...,
+
+ # configurably specify input values, keyed by input name
+ inputs: {
+ __input_name__: {
+ # if an dagster_type_loader is specified, that schema must be satisfied here;
+ # scalar, built-in types will generally allow their values to be specified directly:
+ value: ...
+ }
+ },
+
+ }
+ },
+
+ }
+ ```
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/external-assets.mdx b/docs/docs-beta/docs/api/python-api/external-assets.mdx
new file mode 100644
index 0000000000000..a13569848c064
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/external-assets.mdx
@@ -0,0 +1,54 @@
+---
+title: 'external assets (experimental)'
+title_meta: 'external assets (experimental) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'external assets (experimental) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# External assets (Experimental)
+
+As Dagster doesn’t control scheduling or materializing [external assets](https://docs.dagster.io/guides/build/assets/external-assets), it’s up to you to keep their metadata updated. The APIs in this reference can be used to keep external assets updated in Dagster.
+
+
+
+
+## Instance API
+
+External asset events can be recorded using `DagsterInstance.report_runless_asset_event()` on `DagsterInstance`.
+
+Example: Reporting an asset materialization:
+
+ ```python
+ from dagster import DagsterInstance, AssetMaterialization, AssetKey
+
+ instance = DagsterInstance.get()
+ instance.report_runless_asset_event(AssetMaterialization(AssetKey("example_asset")))
+ ```
+Example: Reporting an asset check evaluation:
+
+ ```python
+ from dagster import DagsterInstance, AssetCheckEvaluation, AssetCheckKey
+
+ instance = DagsterInstance.get()
+ instance.report_runless_asset_event(
+ AssetCheckEvaluation(
+ asset_key=AssetKey("example_asset"),
+ check_name="example_check",
+ passed=True
+ )
+ )
+ ```
+
+
+
+
+
+## REST API
+
+Refer to the [External assets REST API reference](https://docs.dagster.io/apidocs/external-assets-rest) for information and examples on the available APIs.
+
+
diff --git a/docs/docs-beta/docs/api/python-api/graphs.mdx b/docs/docs-beta/docs/api/python-api/graphs.mdx
new file mode 100644
index 0000000000000..12fe20021870a
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/graphs.mdx
@@ -0,0 +1,545 @@
+---
+title: 'graphs'
+title_meta: 'graphs API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'graphs Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Graphs
+
+The core of a job is a _graph_ of ops - connected via data dependencies.
+
+
+
@dagster.graph
+
+
+ Create an op graph with the specified parameters from the decorated composition function.
+
+ Using this decorator allows you to build up a dependency graph by writing a
+ function that invokes ops (or other graphs) and passes the output to subsequent invocations.
+
+ Parameters:
+ - name (Optional[str]) – The name of the op graph. Must be unique within any [`RepositoryDefinition`](repositories.mdx#dagster.RepositoryDefinition) containing the graph.
+ - description (Optional[str]) – A human-readable description of the graph.
+ - input_defs (Optional[List[InputDefinition]]) –
+
+ Information about the inputs that this graph maps. Information provided here
+ will be combined with what can be inferred from the function signature, with these
+ explicit InputDefinitions taking precedence.
+
+ - output_defs (Optional[List[OutputDefinition]]) –
+
+ Output definitions for the graph. If not provided explicitly, these will be inferred from typehints.
+
+ Uses of these outputs in the body of the decorated composition function, as well as the
+ return value of the decorated function, will be used to infer the appropriate set of
+ [`OutputMappings`](#dagster.OutputMapping) for the underlying
+ [`GraphDefinition`](#dagster.GraphDefinition).
+
+ - ins (Optional[Dict[str, [*GraphIn*](#dagster.GraphIn)]]) – Information about the inputs that this graph maps. Information provided here
+ - out –
+
+ Information about the outputs that this graph maps. Information provided here will be
+ combined with what can be inferred from the return type signature if the function does
+ not use yield.
+
+
+
+
+
+
+
+
+
class dagster.GraphDefinition
+
+
+ Defines a Dagster op graph.
+
+ An op graph is made up of
+
+ - Nodes, which can either be an op (the functional unit of computation), or another graph.
+ - Dependencies, which determine how the values produced by nodes as outputs flow from
+
+
+ End users should prefer the [`@graph`](#dagster.graph) decorator. GraphDefinition is generally
+ intended to be used by framework authors or for programatically generated graphs.
+
+ Parameters:
+ - name (str) – The name of the graph. Must be unique within any [`GraphDefinition`](#dagster.GraphDefinition)
+ - description (Optional[str]) – A human-readable description of the job.
+ - node_defs (Optional[Sequence[NodeDefinition]]) – The set of ops / graphs used in this graph.
+ - dependencies (Optional[Dict[Union[str, [*NodeInvocation*](#dagster.NodeInvocation)], Dict[str, [*DependencyDefinition*](#dagster.DependencyDefinition)]]]) – A structure that declares the dependencies of each op’s inputs on the outputs of other
+ - input_mappings (Optional[Sequence[[*InputMapping*](#dagster.InputMapping)]]) – Defines the inputs to the nested graph, and
+ - output_mappings (Optional[Sequence[[*OutputMapping*](#dagster.OutputMapping)]]) – Defines the outputs of the nested graph,
+ - config (Optional[[*ConfigMapping*](config.mdx#dagster.ConfigMapping)]) – Defines the config of the graph, and how its schema maps
+ - tags (Optional[Dict[str, Any]]) – Arbitrary metadata for any execution of the graph.
+ - composition_fn (Optional[Callable]) – The function that defines this graph. Used to generate
+
+
+ Examples:
+
+ ```python
+ @op
+ def return_one():
+ return 1
+
+ @op
+ def add_one(num):
+ return num + 1
+
+ graph_def = GraphDefinition(
+ name='basic',
+ node_defs=[return_one, add_one],
+ dependencies={'add_one': {'num': DependencyDefinition('return_one')}},
+ )
+ ```
+
+
alias
+
+
+ Aliases the graph with a new name.
+
+ Can only be used in the context of a [`@graph`](#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.alias("my_graph_alias")
+ ```
+
+
+
+
+
+
execute_in_process
+
+
+ Execute this graph in-process, collecting results in-memory.
+
+ Parameters:
+ - run_config (Optional[Mapping[str, Any]]) – Run config to provide to execution. The configuration for the underlying graph
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The instance to execute against, an ephemeral one will be used if none provided.
+ - resources (Optional[Mapping[str, Any]]) – The resources needed if any are required. Can provide resource instances directly,
+ - raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur.
+ - op_selection (Optional[List[str]]) – A list of op selection queries (including single op
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of the graph.
+
+
+ Returns: [`ExecuteInProcessResult`](execution.mdx#dagster.ExecuteInProcessResult)
+
+
+
+
+
+
tag
+
+
+ Attaches the provided tags to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.tag({"my_tag": "my_value"})
+ ```
+
+
+
+
+
+
to_job
+
+
+ Make this graph in to an executable Job by providing remaining components required for execution.
+
+ Parameters:
+ - name (Optional[str]) – The name for the Job. Defaults to the name of the this graph.
+ - resource_defs (Optional[Mapping [str, object]]) – Resources that are required by this graph for execution.
+ - config –
+
+ Describes how the job is parameterized at runtime.
+
+ If no value is provided, then the schema for the job’s run config is a standard
+ format based on its ops and resources.
+
+ If a dictionary is provided, then it must conform to the standard config schema, and
+ it will be used as the job’s run config for the job whenever the job is executed.
+ The values provided will be viewable and editable in the Dagster UI, so be
+ careful with secrets.
+
+ If a [`ConfigMapping`](config.mdx#dagster.ConfigMapping) object is provided, then the schema for the job’s run config is
+ determined by the config mapping, and the ConfigMapping, which should return
+ configuration in the standard format to configure the job.
+
+ - tags (Optional[Mapping[str, object]]) – A set of key-value tags that annotate the job and can
+ - run_tags (Optional[Mapping[str, object]]) – A set of key-value tags that will be automatically attached to runs launched by this
+ - metadata (Optional[Mapping[str, RawMetadataValue]]) – Arbitrary information that will be attached to the JobDefinition and be viewable in the Dagster UI.
+ - logger_defs (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – A dictionary of string logger identifiers to their implementations.
+ - executor_def (Optional[[*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]) – How this Job will be executed. Defaults to [`multi_or_in_process_executor`](execution.mdx#dagster.multi_or_in_process_executor),
+ - op_retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The default retry policy for all ops in this job.
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines a discrete set of partition
+ - asset_layer (Optional[AssetLayer]) – Top level information about the assets this job
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of a job.
+
+
+ Returns: JobDefinition
+
+
+
+
+
+
with_hooks
+
+
+ Attaches the provided hooks to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.with_hooks({my_hook})
+ ```
+
+
+
+
+
+
with_retry_policy
+
+
+ Attaches the provided retry policy to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.with_retry_policy(RetryPolicy(max_retries=5))
+ ```
+
+
+
+
+
+
property config_mapping
+
+
+ The config mapping for the graph, if present.
+
+ By specifying a config mapping function, you can override the configuration for the child nodes contained within a graph.
+
+
+
+
+
+
+
property input_mappings
+
+
+ Input mappings for the graph.
+
+ An input mapping is a mapping from an input of the graph to an input of a child node.
+
+
+
+
+
+
+
property name
+
+ The name of the graph.
+
+
+
+
+
property output_mappings
+
+
+ Output mappings for the graph.
+
+ An output mapping is a mapping from an output of the graph to an output of a child node.
+
+
+
+
+
+
+
property tags
+
+ The tags associated with the graph.
+
+
+
+
+
+
+
+
+
class dagster.GraphIn
+
+
+ Represents information about an input that a graph maps.
+
+ Parameters: description (Optional[str]) – Human-readable description of the input.
+
+
+
+
+
+
class dagster.GraphOut
+
+
+ Represents information about the outputs that a graph maps.
+
+ Parameters: description (Optional[str]) – Human-readable description of the output.
+
+
+
+
+
+
+## Explicit dependencies
+
+
+
class dagster.DependencyDefinition
+
+
+ Represents an edge in the DAG of nodes (ops or graphs) forming a job.
+
+ This object is used at the leaves of a dictionary structure that represents the complete
+ dependency structure of a job whose keys represent the dependent node and dependent
+ input, so this object only contains information about the dependee.
+
+ Concretely, if the input named ‘input’ of op_b depends on the output named ‘result’ of
+ op_a, and the output named ‘other_result’ of graph_a, the structure will look as follows:
+
+ ```python
+ dependency_structure = {
+ 'my_downstream_op': {
+ 'input': DependencyDefinition('my_upstream_op', 'result')
+ }
+ 'my_downstream_op': {
+ 'input': DependencyDefinition('my_upstream_graph', 'result')
+ }
+ }
+ ```
+ In general, users should prefer not to construct this class directly or use the
+ [`JobDefinition`](jobs.mdx#dagster.JobDefinition) API that requires instances of this class. Instead, use the
+ [`@job`](jobs.mdx#dagster.job) API:
+
+ ```python
+ @job
+ def the_job():
+ node_b(node_a())
+ ```
+ Parameters:
+ - node (str) – The name of the node (op or graph) that is depended on, that is, from which the value
+ - output (Optional[str]) – The name of the output that is depended on. (default: “result”)
+ - description (Optional[str]) – Human-readable description of this dependency.
+
+
+
+
is_fan_in
+
+ Return True if the dependency is fan-in (always False for DependencyDefinition).
+
+
+
+
+
+
+
+
+
class dagster.MultiDependencyDefinition
+
+
+ Represents a fan-in edge in the DAG of op instances forming a job.
+
+ This object is used only when an input of type `List[T]` is assembled by fanning-in multiple
+ upstream outputs of type `T`.
+
+ This object is used at the leaves of a dictionary structure that represents the complete
+ dependency structure of a job whose keys represent the dependent ops or graphs and dependent
+ input, so this object only contains information about the dependee.
+
+ Concretely, if the input named ‘input’ of op_c depends on the outputs named ‘result’ of
+ op_a and op_b, this structure will look as follows:
+
+ ```python
+ dependency_structure = {
+ 'op_c': {
+ 'input': MultiDependencyDefinition(
+ [
+ DependencyDefinition('op_a', 'result'),
+ DependencyDefinition('op_b', 'result')
+ ]
+ )
+ }
+ }
+ ```
+ In general, users should prefer not to construct this class directly or use the
+ [`JobDefinition`](jobs.mdx#dagster.JobDefinition) API that requires instances of this class. Instead, use the
+ [`@job`](jobs.mdx#dagster.job) API:
+
+ ```python
+ @job
+ def the_job():
+ op_c(op_a(), op_b())
+ ```
+ Parameters: dependencies (List[Union[[*DependencyDefinition*](#dagster.DependencyDefinition), Type[MappedInputPlaceHolder]]]) – List of
+ upstream dependencies fanned in to this input.
+
+
get_dependencies_and_mappings
+
+ Return the combined list of dependencies contained by this object, inculding of [`DependencyDefinition`](#dagster.DependencyDefinition) and `MappedInputPlaceholder` objects.
+
+
+
+
+
get_node_dependencies
+
+ Return the list of [`DependencyDefinition`](#dagster.DependencyDefinition) contained by this object.
+
+
+
+
+
is_fan_in
+
+ Return True if the dependency is fan-in (always True for MultiDependencyDefinition).
+
+
+
+
+
+
+
+
+
class dagster.NodeInvocation
+
+
+ Identifies an instance of a node in a graph dependency structure.
+
+ Parameters:
+ - name (str) – Name of the node of which this is an instance.
+ - alias (Optional[str]) – Name specific to this instance of the node. Necessary when there are
+ - tags (Optional[Dict[str, Any]]) – Optional tags values to extend or override those
+ - hook_defs (Optional[AbstractSet[[*HookDefinition*](hooks.mdx#dagster.HookDefinition)]]) – A set of hook definitions applied to the
+
+
+ Examples:
+ In general, users should prefer not to construct this class directly or use the
+ [`JobDefinition`](jobs.mdx#dagster.JobDefinition) API that requires instances of this class. Instead, use the
+ [`@job`](jobs.mdx#dagster.job) API:
+
+ ```python
+ from dagster import job
+
+ @job
+ def my_job():
+ other_name = some_op.alias('other_name')
+ some_graph(other_name(some_op))
+ ```
+
+
+
+
+
+
class dagster.OutputMapping
+
+
+ Defines an output mapping for a graph.
+
+ Parameters:
+ - graph_output_name (str) – Name of the output in the graph being mapped to.
+ - mapped_node_name (str) – Named of the node (op/graph) that the output is being mapped from.
+ - mapped_node_output_name (str) – Name of the output in the node (op/graph) that is being mapped from.
+ - graph_output_description (Optional[str]) – A description of the output in the graph being mapped from.
+ - from_dynamic_mapping (bool) – Set to true if the node being mapped to is a mapped dynamic node.
+ - dagster_type (Optional[[*DagsterType*](types.mdx#dagster.DagsterType)]) – deprecateddagster_type should come from the underlying op Output.) The dagster type of the graph’s output being mapped to.
+
+
+ Examples:
+
+ ```python
+ from dagster import OutputMapping, GraphDefinition, op, graph, GraphOut
+
+ @op
+ def emit_five(x):
+ return 5
+
+ # The following two graph definitions are equivalent
+ GraphDefinition(
+ name="the_graph",
+ node_defs=[emit_five],
+ output_mappings=[
+ OutputMapping(
+ graph_output_name="result", # Default output name
+ mapped_node_name="emit_five",
+ mapped_node_output_name="result"
+ )
+ ]
+ )
+
+ @graph(out=GraphOut())
+ def the_graph:
+ return emit_five()
+ ```
+
+
+
+
+
+
class dagster.InputMapping
+
+
+ Defines an input mapping for a graph.
+
+ Parameters:
+ - graph_input_name (str) – Name of the input in the graph being mapped from.
+ - mapped_node_name (str) – Named of the node (op/graph) that the input is being mapped to.
+ - mapped_node_input_name (str) – Name of the input in the node (op/graph) that is being mapped to.
+ - fan_in_index (Optional[int]) – The index in to a fanned input, otherwise None.
+ - graph_input_description (Optional[str]) – A description of the input in the graph being mapped from.
+ - dagster_type (Optional[[*DagsterType*](types.mdx#dagster.DagsterType)]) – deprecateddagster_type should come from the upstream op Output.) The dagster type of the graph’s input
+
+
+ Examples:
+
+ ```python
+ from dagster import InputMapping, GraphDefinition, op, graph
+
+ @op
+ def needs_input(x):
+ return x + 1
+
+ # The following two graph definitions are equivalent
+ GraphDefinition(
+ name="the_graph",
+ node_defs=[needs_input],
+ input_mappings=[
+ InputMapping(
+ graph_input_name="maps_x", mapped_node_name="needs_input",
+ mapped_node_input_name="x"
+ )
+ ]
+ )
+
+ @graph
+ def the_graph(maps_x):
+ needs_input(maps_x)
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/hooks.mdx b/docs/docs-beta/docs/api/python-api/hooks.mdx
new file mode 100644
index 0000000000000..2f81ec791a68d
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/hooks.mdx
@@ -0,0 +1,227 @@
+---
+title: 'hooks'
+title_meta: 'hooks API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'hooks Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Hooks
+
+
+
@dagster.success_hook
+
+
+ Create a hook on step success events with the specified parameters from the decorated function.
+
+ Parameters:
+ - name (Optional[str]) – The name of this hook.
+ - required_resource_keys (Optional[AbstractSet[str]]) – Keys for the resources required by the
+
+
+ Examples:
+
+ ```python
+ @success_hook(required_resource_keys={'slack'})
+ def slack_message_on_success(context):
+ message = 'op {} succeeded'.format(context.op.name)
+ context.resources.slack.send_message(message)
+
+ @success_hook
+ def do_something_on_success(context):
+ do_something()
+ ```
+
+
+
+
+
+
@dagster.failure_hook
+
+
+ Create a hook on step failure events with the specified parameters from the decorated function.
+
+ Parameters:
+ - name (Optional[str]) – The name of this hook.
+ - required_resource_keys (Optional[AbstractSet[str]]) – Keys for the resources required by the
+
+
+ Examples:
+
+ ```python
+ @failure_hook(required_resource_keys={'slack'})
+ def slack_message_on_failure(context):
+ message = 'op {} failed'.format(context.op.name)
+ context.resources.slack.send_message(message)
+
+ @failure_hook
+ def do_something_on_failure(context):
+ do_something()
+ ```
+
+
+
+
+
+
class dagster.HookDefinition
+
+
+ Define a hook which can be triggered during a op execution (e.g. a callback on the step
+ execution failure event during a op execution).
+
+ Parameters:
+ - name (str) – The name of this hook.
+ - hook_fn (Callable) – The callback function that will be triggered.
+ - required_resource_keys (Optional[AbstractSet[str]]) – Keys for the resources required by the
+
+
+
+
+
+
+
+
class dagster.HookContext
+
+
+ The `context` object available to a hook function on an DagsterEvent.
+
+
+
property hook_def
+
+ The hook that the context object belongs to.
+
+
+
+
+
property instance
+
+ The instance configured to run the current job.
+
+
+
+
+
property job_name
+
+ The name of the job where this hook is being triggered.
+
+
+
+
+
property log
+
+ Centralized log dispatch from user code.
+
+
+
+
+
property op_config
+
+ The parsed config specific to this op.
+
+
+
+
+
property op_exception
+
+ The thrown exception in a failed op.
+
+
+
+
+
property op_output_metadata
+
+
+ The applied output metadata.
+
+ Returns a dictionary where keys are output names and the values are:
+ - the applied output metadata in the normal case
+ - a dictionary from mapping key to corresponding metadata in the mapped case
+
+
+
+
+
+
+
+
property op_output_values
+
+
+ The computed output values.
+
+ Returns a dictionary where keys are output names and the values are:
+ - the output values in the normal case
+ - a dictionary from mapping key to corresponding value in the mapped case
+
+
+
+
+
+
+
+
property required_resource_keys
+
+ Resources required by this hook.
+
+
+
+
+
property resources
+
+ Resources available in the hook context.
+
+
+
+
+
property run_id
+
+ The id of the run where this hook is being triggered.
+
+
+
+
+
property step_key
+
+ The key for the step where this hook is being triggered.
+
+
+
+
+
+
+
+
+
dagster.build_hook_context
+
+
+ Builds hook context from provided parameters.
+
+ `build_hook_context` can be used as either a function or a context manager. If there is a
+ provided resource to `build_hook_context` that is a context manager, then it must be used as a
+ context manager. This function can be used to provide the context argument to the invocation of
+ a hook definition.
+
+ Parameters:
+ - resources (Optional[Dict[str, Any]]) – The resources to provide to the context. These can
+ - op (Optional[[*OpDefinition*](ops.mdx#dagster.OpDefinition), PendingNodeInvocation]) – The op definition which the
+ - run_id (Optional[str]) – The id of the run in which the hook is invoked (provided for mocking purposes).
+ - job_name (Optional[str]) – The name of the job in which the hook is used (provided for mocking purposes).
+ - op_exception (Optional[Exception]) – The exception that caused the hook to be triggered.
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The Dagster instance configured to run the hook.
+
+
+ Examples:
+
+ ```python
+ context = build_hook_context()
+ hook_to_invoke(context)
+
+ with build_hook_context(resources={"foo": context_manager_resource}) as context:
+ hook_to_invoke(context)
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/internals.mdx b/docs/docs-beta/docs/api/python-api/internals.mdx
new file mode 100644
index 0000000000000..23856c4c4c97a
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/internals.mdx
@@ -0,0 +1,1568 @@
+---
+title: 'internals'
+title_meta: 'internals API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'internals Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# 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](https://join.slack.com/t/dagster/shared_invite/enQtNjEyNjkzNTA2OTkzLTI0MzdlNjU0ODVhZjQyOTMyMGM1ZDUwZDQ1YjJmYjI3YzExZGViMDI1ZDlkNTY5OThmYWVlOWM1MWVjN2I3NjU).
+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`](#dagster.InitExecutorContext) and return an instance
+ of [`Executor`](#dagster.Executor).
+
+ Parameters:
+ - name (Optional[str]) – The name of the executor.
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – 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[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data
+ - requirements (Optional[List[ExecutorRequirement]]) – Any requirements that must
+ - executor_creation_fn (Optional[Callable]) – Should accept an [`InitExecutorContext`](#dagster.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[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – 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`](#dagster.InitExecutorContext) and returns an instance of
+ [`Executor`](#dagster.Executor).
+
+
+
+
+
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](#dagster.ExecutorDefinition)
+
+
+
+
+
+
executor_config
+
+
+ The parsed config passed to the executor.
+
+ Type: dict
+
+
+
+
+
+
instance
+
+
+ The current instance.
+
+ Type: [DagsterInstance](#dagster.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()`](#dagster._core.storage.file_manager.FileManager.read),
+ [`read_data()`](#dagster._core.storage.file_manager.FileManager.read_data),
+ [`write()`](#dagster._core.storage.file_manager.FileManager.write), and
+ [`write_data()`](#dagster._core.storage.file_manager.FileManager.write_data) methods.
+
+ Parameters: file_handle ([*FileHandle*](#dagster.FileHandle)) – 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()`](#dagster._core.storage.file_manager.FileManager.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 ([*FileHandle*](#dagster.FileHandle)) – 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 ([*FileHandle*](#dagster.FileHandle)) – 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](#dagster.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.FileHandle)
+
+
+
+
+
+
+
+
+
+
dagster.local_file_manager ResourceDefinition
+
+
+ FileManager that provides abstract access to a local filesystem.
+
+ By default, files will be stored in \/storage/file_manager where
+ \ can be configured the `dagster.yaml` file in `$DAGSTER_HOME`.
+
+ Implements the [`FileManager`](#dagster._core.storage.file_manager.FileManager) API.
+
+ Examples:
+
+ ```python
+ 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:
+
+ ```python
+ @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 a `dagster.yaml` such as the
+ following:
+
+ dagster.yaml
+
+ ```YAML
+ storage:
+ postgres:
+ postgres_db:
+ username: my_username
+ password: my_password
+ hostname: my_hostname
+ db_name: my_database
+ port: 5432
+ ```
+ Parameters:
+ - instance_type (InstanceType) – Indicates whether the instance is ephemeral or persistent.
+ - local_artifact_storage ([*LocalArtifactStorage*](#dagster._core.storage.root.LocalArtifactStorage)) – The local artifact storage is used to
+ - run_storage ([*RunStorage*](#dagster._core.storage.runs.RunStorage)) – The run storage is used to store metadata about ongoing and past
+ - event_storage ([*EventLogStorage*](#dagster._core.storage.event_log.EventLogStorage)) – Used to store the structured event logs generated by
+ - compute_log_manager (Optional[[*ComputeLogManager*](#dagster._core.storage.compute_log_manager.ComputeLogManager)]) – The compute log manager handles stdout
+ - run_coordinator (Optional[RunCoordinator]) – A runs coordinator may be used to manage the execution
+ - run_launcher (Optional[[*RunLauncher*](#dagster._core.launcher.RunLauncher)]) – Optionally, a run launcher may be used to enable
+ - settings (Optional[Dict]) – Specifies certain per-instance settings,
+ - ref (Optional[[*InstanceRef*](#dagster._core.instance.InstanceRef)]) – Used by internal machinery to pass instances across process
+
+
+
+
add_dynamic_partitions
+
+
+ Add partitions to the specified [`DynamicPartitionsDefinition`](partitions.mdx#dagster.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`](partitions.mdx#dagster.DynamicPartitionsDefinition).
+ If the partition does not exist, exits silently.
+
+ Parameters:
+ - partitions_def_name (str) – The name of the DynamicPartitionsDefinition.
+ - partition_key (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](#dagster.DagsterInstance)
+
+
+
+
+
+
fetch_materializations
+
+
+ Return a list of materialization records stored in the event log storage.
+
+ Parameters:
+ - records_filter (Union[[*AssetKey*](assets.mdx#dagster.AssetKey), 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[[*AssetKey*](assets.mdx#dagster.AssetKey), 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[[*DagsterEventType*](execution.mdx#dagster.DagsterEventType), 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](#dagster.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](assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
get_asset_records
+
+
+ Return an AssetRecord for each of the given asset keys.
+
+ Parameters: asset_keys (Optional[Sequence[[*AssetKey*](assets.mdx#dagster.AssetKey)]]) – List of asset keys to retrieve records for.Returns: List of asset records.Return type: Sequence[[AssetRecord](#dagster._core.storage.event_log.AssetRecord)]
+
+
+
+
+
+
get_dynamic_partitions
+
+
+ Get the set of partition keys for the specified [`DynamicPartitionsDefinition`](partitions.mdx#dagster.DynamicPartitionsDefinition).
+
+ Parameters: partitions_def_name (str) – The name of the DynamicPartitionsDefinition.
+
+
+
+
+
+
get_latest_materialization_code_versions
+
+
+ Returns the code version used for the latest materialization of each of the provided
+ assets.
+
+ Parameters: asset_keys (Iterable[[*AssetKey*](assets.mdx#dagster.AssetKey)]) – 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](assets.mdx#dagster.AssetKey), Optional[str]]
+
+
+
+
+
+
get_latest_materialization_event
+
+
+ Fetch the latest materialization event for the given asset key.
+
+ Parameters: asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – 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](#dagster.EventLogEntry)]
+
+
+
+
+
+
get_run_by_id
+
+
+ Get a [`DagsterRun`](#dagster.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](#dagster.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](#dagster._core.storage.dagster_run.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[[*RunsFilter*](#dagster.RunsFilter)]) – 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](#dagster._core.storage.dagster_run.RunRecord)]
+
+
+
+
+
+
get_status_by_partition
+
+
+ Get the current status of provided partition_keys for the provided asset.
+
+ Parameters:
+ - asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – The asset to get per-partition status for.
+ - partition_keys (Sequence[str]) – The partitions to get status for.
+ - partitions_def ([*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)) – 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 ([*AssetKey*](assets.mdx#dagster.AssetKey)) – Asset key to check.
+
+
+
+
+
+
has_dynamic_partition
+
+
+ Check if a partition key exists for the [`DynamicPartitionsDefinition`](partitions.mdx#dagster.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
+
+
+ :::warning[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[[*AssetKey*](assets.mdx#dagster.AssetKey)]) – Asset keys to wipe.
+
+
+
+
+
+
+
+
+
+
class dagster._core.instance.InstanceRef
+
+
+ Serializable representation of a [`DagsterInstance`](#dagster.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:
+
+ ```yaml
+ 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 of `ConfigurableClassData`.
+
+ Pieces of the Dagster system which we wish to make pluggable in this way should consume a config
+ type such as:
+
+ ```python
+ {'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.
+
+
+
+
+
+
+
class dagster._core.storage.root.LocalArtifactStorage
+
+
+
+
+
+
+
+
+
+
+
+## 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` and `dagster-daemon` load, based on the values in the
+ `dagster.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`](#dagster._core.storage.runs.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[[*DagsterRunStatus*](#dagster.DagsterRunStatus)]]) – 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`](#dagster._core.storage.runs.SqlRunStorage).
+
+ Users should not directly instantiate concrete subclasses of this class; they are instantiated
+ by internal machinery when `dagster-webserver` and `dagster-graphql` load, based on the values in the
+ `dagster.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` and `dagster-graphql` load, based on the values in the `dagster.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`:
+
+ ```YAML
+ run_storage:
+ module: dagster._core.storage.runs
+ class: SqliteRunStorage
+ config:
+ base_dir: /path/to/dir
+ ```
+ The `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`](#dagster._core.storage.runs.RunStorage).
+
+ Users should not invoke this class directly.
+
+
+
+
+
+See also: [`dagster_postgres.PostgresRunStorage`](libraries/dagster-postgres.mdx#dagster_postgres.PostgresRunStorage) and [`dagster_mysql.MySQLRunStorage`](libraries/dagster-mysql.mdx#dagster_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[[*DagsterEvent*](execution.mdx#dagster.DagsterEvent)]) – 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](execution.mdx#dagster.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`](#dagster._core.storage.event_log.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 ([*DagsterEventType*](execution.mdx#dagster.DagsterEventType)) – Filter argument for dagster event type
+ - asset_key (Optional[[*AssetKey*](assets.mdx#dagster.AssetKey)]) – 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`](#dagster._core.storage.event_log.SqlEventLogStorage).
+
+ Users should not directly instantiate concrete subclasses of this class; they are instantiated
+ by internal machinery when `dagster-webserver` and `dagster-graphql` load, based on the values in the
+ `dagster.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` and `dagster-graphql` load, based on the values in the `dagster.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`:
+
+ ```YAML
+ event_log_storage:
+ module: dagster._core.storage.event_log
+ class: SqliteEventLogStorage
+ config:
+ base_dir: /path/to/dir
+ ```
+ The `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` and `dagster-graphql` load, based on the values in the `dagster.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`:
+
+ ```YAML
+ run_storage:
+ module: dagster._core.storage.event_log
+ class: ConsolidatedSqliteEventLogStorage
+ config:
+ base_dir: /path/to/dir
+ ```
+ The `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`](#dagster._core.storage.event_log.EventLogStorage).
+
+ Users should not invoke this class directly.
+
+
+
+
+
+See also: [`dagster_postgres.PostgresEventLogStorage`](libraries/dagster-postgres.mdx#dagster_postgres.PostgresEventLogStorage) and [`dagster_mysql.MySQLEventLogStorage`](libraries/dagster-mysql.mdx#dagster_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.RunLauncher
+
+
+
+
+
+
+
class dagster._core.launcher.DefaultRunLauncher
+
+ Launches runs against running GRPC servers.
+
+
+
+
+
+
+
+
+
+## Run coordinator
+
+
+
class dagster._core.run_coordinator.DefaultRunCoordinator
+
+ 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.PostgresScheduleStorage`](libraries/dagster-postgres.mdx#dagster_postgres.PostgresScheduleStorage) and [`dagster_mysql.MySQLScheduleStorage`](libraries/dagster-mysql.mdx#dagster_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](https://docs.dagster.io/guides/migrations/from-step-launchers-to-pipes).
+
+
+
class dagster.StepLauncher
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ While there is no plan to remove this functionality, for new projects, we recommend using Dagster Pipes. For more information, see https://docs.dagster.io/guides/build/external-pipelines/.
+
+ :::
+
+ 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.
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/io-managers.mdx b/docs/docs-beta/docs/api/python-api/io-managers.mdx
new file mode 100644
index 0000000000000..4343300c84b47
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/io-managers.mdx
@@ -0,0 +1,1194 @@
+---
+title: 'io managers'
+title_meta: 'io managers API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'io managers Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# IO Managers
+
+IO managers are user-provided objects that store op outputs and load them as inputs to downstream
+ops.
+
+
+
class dagster.ConfigurableIOManager
+
+
+ Base class for Dagster IO managers that utilize structured config.
+
+ This class is a subclass of both [`IOManagerDefinition`](#dagster.IOManagerDefinition), [`Config`](config.mdx#dagster.Config),
+ and [`IOManager`](#dagster.IOManager). Implementers must provide an implementation of the
+ `handle_output()` and `load_input()` methods.
+
+ Example definition:
+
+ ```python
+ class MyIOManager(ConfigurableIOManager):
+ path_prefix: List[str]
+
+ def _get_path(self, context) -> str:
+ return "/".join(context.asset_key.path)
+
+ def handle_output(self, context, obj):
+ write_csv(self._get_path(context), obj)
+
+ def load_input(self, context):
+ return read_csv(self._get_path(context))
+
+ defs = Definitions(
+ ...,
+ resources={
+ "io_manager": MyIOManager(path_prefix=["my", "prefix"])
+ }
+ )
+ ```
+
+
+
+
+
+
class dagster.ConfigurableIOManagerFactory
+
+
+ Base class for Dagster IO managers that utilize structured config. This base class
+ is useful for cases in which the returned IO manager is not the same as the class itself
+ (e.g. when it is a wrapper around the actual IO manager implementation).
+
+ This class is a subclass of both [`IOManagerDefinition`](#dagster.IOManagerDefinition) and [`Config`](config.mdx#dagster.Config).
+ Implementers should provide an implementation of the `resource_function()` method,
+ which should return an instance of [`IOManager`](#dagster.IOManager).
+
+ Example definition:
+
+ ```python
+ class ExternalIOManager(IOManager):
+
+ def __init__(self, connection):
+ self._connection = connection
+
+ def handle_output(self, context, obj):
+ ...
+
+ def load_input(self, context):
+ ...
+
+ class ConfigurableExternalIOManager(ConfigurableIOManagerFactory):
+ username: str
+ password: str
+
+ def create_io_manager(self, context) -> IOManager:
+ with database.connect(username, password) as connection:
+ return MyExternalIOManager(connection)
+
+ defs = Definitions(
+ ...,
+ resources={
+ "io_manager": ConfigurableExternalIOManager(
+ username="dagster",
+ password=EnvVar("DB_PASSWORD")
+ )
+ }
+ )
+ ```
+
+
+
+
+
+
class dagster.IOManager
+
+
+ Base class for user-provided IO managers.
+
+ IOManagers are used to store op outputs and load them as inputs to downstream ops.
+
+ Extend this class to handle how objects are loaded and stored. Users should implement
+ `handle_output` to store an object and `load_input` to retrieve an object.
+
+
+
abstract handle_output
+
+
+ User-defined method that stores an output of an op.
+
+ Parameters:
+ - context ([*OutputContext*](#dagster.OutputContext)) – The context of the step output that produces this object.
+ - obj (Any) – The object, returned by the op, to be stored.
+
+
+
+
+
+
+
+
abstract load_input
+
+
+ User-defined method that loads an input to an op.
+
+ Parameters: context ([*InputContext*](#dagster.InputContext)) – The input context, which describes the input that’s being loaded
+ and the upstream output that’s being loaded from.Returns: The data object.Return type: Any
+
+
+
+
+
+
+
+
+
+
class dagster.IOManagerDefinition
+
+
+ Definition of an IO manager resource.
+
+ IOManagers are used to store op outputs and load them as inputs to downstream ops.
+
+ An IOManagerDefinition is a [`ResourceDefinition`](resources.mdx#dagster.ResourceDefinition) whose resource_fn returns an
+ [`IOManager`](#dagster.IOManager).
+
+ The easiest way to create an IOManagerDefnition is with the [`@io_manager`](#dagster.io_manager)
+ decorator.
+
+
+
static hardcoded_io_manager
+
+
+ A helper function that creates an `IOManagerDefinition` with a hardcoded IOManager.
+
+ Parameters:
+ - value ([*IOManager*](#dagster.IOManager)) – A hardcoded IO Manager which helps mock the definition.
+ - description ([Optional[str]]) – The description of the IO Manager. Defaults to None.
+
+
+ Returns: A hardcoded resource.Return type: [[IOManagerDefinition](#dagster.IOManagerDefinition)]
+
+
+
+
+
+
+
+
+
+
@dagster.io_manager
+
+
+ Define an IO manager.
+
+ IOManagers are used to store op outputs and load them as inputs to downstream ops.
+
+ The decorated function should accept an [`InitResourceContext`](resources.mdx#dagster.InitResourceContext) and return an
+ [`IOManager`](#dagster.IOManager).
+
+ Parameters:
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the resource config. Configuration
+ - description (Optional[str]) – A human-readable description of the resource.
+ - output_config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for per-output config. If not set,
+ - input_config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for per-input config. If not set,
+ - required_resource_keys (Optional[Set[str]]) – Keys for the resources required by the object
+ - version (Optional[str]) – (Experimental) The version of a resource function. Two wrapped
+
+
+ Examples:
+
+ ```python
+ class MyIOManager(IOManager):
+ def handle_output(self, context, obj):
+ write_csv("some/path")
+
+ def load_input(self, context):
+ return read_csv("some/path")
+
+ @io_manager
+ def my_io_manager(init_context):
+ return MyIOManager()
+
+ @op(out=Out(io_manager_key="my_io_manager_key"))
+ def my_op(_):
+ return do_stuff()
+
+ @job(resource_defs={"my_io_manager_key": my_io_manager})
+ def my_job():
+ my_op()
+ ```
+
+
+
+
+
+
+
+## Input and Output Contexts
+
+
+
class dagster.InputContext
+
+
+ The `context` object available to the load_input method of [`InputManager`](#dagster.InputManager).
+
+ Users should not instantiate this object directly. In order to construct
+ an InputContext for testing an IO Manager’s load_input method, use
+ [`dagster.build_input_context()`](#dagster.build_input_context).
+
+ Example:
+
+ ```python
+ from dagster import IOManager, InputContext
+
+ class MyIOManager(IOManager):
+ def load_input(self, context: InputContext):
+ ...
+ ```
+
+
get_asset_identifier
+
+ The sequence of strings making up the AssetKey for the asset being loaded as an input.
+ If the asset is partitioned, the identifier contains the partition key as the final element in the
+ sequence. For example, for the asset key `AssetKey(["foo", "bar", "baz"])`, materialized with
+ partition key “2023-06-01”, `get_asset_identifier` will return `["foo", "bar", "baz", "2023-06-01"]`.
+
+
+
+
+
get_identifier
+
+
+ Utility method to get a collection of identifiers that as a whole represent a unique
+ step input.
+
+ If not using memoization, the unique identifier collection consists of
+
+ - `run_id`: the id of the run which generates the input.
+ - `step_key`: the key for a compute step.
+ - `name`: the name of the output. (default: ‘result’).
+
+
+ If using memoization, the `version` corresponding to the step output is used in place of
+ the `run_id`.
+
+ Returns: A list of identifiers, i.e. (run_id or version), step_key, and output_nameReturn type: List[str, …]
+
+
+
+
+
+
property asset_key
+
+ The `AssetKey` of the asset that is being loaded as an input.
+
+
+
+
+
property asset_partition_key
+
+
+ The partition key for input asset.
+
+ Raises an error if the input asset has no partitioning, or if the run covers a partition
+ range for the input asset.
+
+
+
+
+
+
+
property asset_partition_key_range
+
+
+ The partition key range for input asset.
+
+ Raises an error if the input asset has no partitioning.
+
+
+
+
+
+
+
property asset_partition_keys
+
+
+ The partition keys for input asset.
+
+ Raises an error if the input asset has no partitioning.
+
+
+
+
+
+
+
property asset_partitions_def
+
+ The PartitionsDefinition on the upstream asset corresponding to this input.
+
+
+
+
+
property asset_partitions_time_window
+
+
+ The time window for the partitions of the input asset.
+
+ Raises an error if either of the following are true:
+ - The input asset has no partitioning.
+ - The input asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+
+
+
+
+
+
property config
+
+ The config attached to the input that we’re loading.
+
+
+
+
+
property dagster_type
+
+ The type of this input.
+ Dagster types do not propagate from an upstream output to downstream inputs,
+ and this property only captures type information for the input that is either
+ passed in explicitly with [`AssetIn`](assets.mdx#dagster.AssetIn) or [`In`](ops.mdx#dagster.In), or can be
+ infered from type hints. For an asset input, the Dagster type from the upstream
+ asset definition is ignored.
+
+
+
+
+
property definition_metadata
+
+ A dict of metadata that is assigned to the InputDefinition that we’re loading.
+ This property only contains metadata passed in explicitly with [`AssetIn`](assets.mdx#dagster.AssetIn)
+ or [`In`](ops.mdx#dagster.In). To access metadata of an upstream asset or op definition,
+ use the definition_metadata in [`InputContext.upstream_output`](#dagster.InputContext.upstream_output).
+
+
+
+
+
property has_asset_key
+
+ Returns True if an asset is being loaded as input, otherwise returns False. A return value of False
+ indicates that an output from an op is being loaded as the input.
+
+
+
+
+
property has_asset_partitions
+
+ Returns True if the asset being loaded as input is partitioned.
+
+
+
+
+
property has_input_name
+
+ If we’re the InputContext is being used to load the result of a run from outside the run,
+ then it won’t have an input name.
+
+
+
+
+
property has_partition_key
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property log
+
+ The log manager to use for this input.
+
+
+
+
+
property metadata
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.0.
+ Use definition_metadata instead.
+
+ :::
+
+ Use definitiion_metadata instead.
+
+ Type: Deprecated
+
+
+
+
+
+
property name
+
+ The name of the input that we’re loading.
+
+
+
+
+
property op_def
+
+ The definition of the op that’s loading the input.
+
+
+
+
+
property partition_key
+
+
+ The partition key for the current run.
+
+ Raises an error if the current run is not a partitioned run.
+
+
+
+
+
+
+
property resource_config
+
+ The config associated with the resource that initializes the InputManager.
+
+
+
+
+
property resources
+
+ The resources required by the resource that initializes the
+ input manager. If using the `@input_manager()` decorator, these resources
+ correspond to those requested with the required_resource_keys parameter.
+
+
+
+
+
property upstream_output
+
+ Info about the output that produced the object we’re loading.
+
+
+
+
+
+
+
+
+
class dagster.OutputContext
+
+
+ The context object that is available to the handle_output method of an [`IOManager`](#dagster.IOManager).
+
+ Users should not instantiate this object directly. To construct an
+ OutputContext for testing an IO Manager’s handle_output method, use
+ [`dagster.build_output_context()`](#dagster.build_output_context).
+
+ Example:
+
+ ```python
+ from dagster import IOManager, OutputContext
+
+ class MyIOManager(IOManager):
+ def handle_output(self, context: OutputContext, obj):
+ ...
+ ```
+
+
add_output_metadata
+
+
+ Add a dictionary of metadata to the handled output.
+
+ Metadata entries added will show up in the HANDLED_OUTPUT and ASSET_MATERIALIZATION events for the run.
+
+ Parameters: metadata (Mapping[str, RawMetadataValue]) – A metadata dictionary to log
+ Examples:
+
+ ```python
+ from dagster import IOManager
+
+ class MyIOManager(IOManager):
+ def handle_output(self, context, obj):
+ context.add_output_metadata({"foo": "bar"})
+ ```
+
+
+
+
+
+
get_asset_identifier
+
+ The sequence of strings making up the AssetKey for the asset being stored as an output.
+ If the asset is partitioned, the identifier contains the partition key as the final element in the
+ sequence. For example, for the asset key `AssetKey(["foo", "bar", "baz"])` materialized with
+ partition key “2023-06-01”, `get_asset_identifier` will return `["foo", "bar", "baz", "2023-06-01"]`.
+
+
+
+
+
get_identifier
+
+
+ Utility method to get a collection of identifiers that as a whole represent a unique
+ step output.
+
+ If not using memoization, the unique identifier collection consists of
+
+ - `run_id`: the id of the run which generates the output.
+ - `step_key`: the key for a compute step.
+ - `name`: the name of the output. (default: ‘result’).
+
+
+ If using memoization, the `version` corresponding to the step output is used in place of
+ the `run_id`.
+
+ Returns: A list of identifiers, i.e. (run_id or version), step_key, and output_nameReturn type: Sequence[str, …]
+
+
+
+
+
+
log_event
+
+
+ Log an AssetMaterialization or AssetObservation from within the body of an io manager’s handle_output method.
+
+ Events logged with this method will appear in the event log.
+
+ Parameters: event (Union[[*AssetMaterialization*](ops.mdx#dagster.AssetMaterialization), [*AssetObservation*](assets.mdx#dagster.AssetObservation)]) – The event to log.
+ Examples:
+
+ ```python
+ from dagster import IOManager, AssetMaterialization
+
+ class MyIOManager(IOManager):
+ def handle_output(self, context, obj):
+ context.log_event(AssetMaterialization("foo"))
+ ```
+
+
+
+
+
+
property asset_key
+
+ The `AssetKey` of the asset that is being stored as an output.
+
+
+
+
+
property asset_partition_key
+
+
+ The partition key for output asset.
+
+ Raises an error if the output asset has no partitioning, or if the run covers a partition
+ range for the output asset.
+
+
+
+
+
+
+
property asset_partition_key_range
+
+
+ The partition key range for output asset.
+
+ Raises an error if the output asset has no partitioning.
+
+
+
+
+
+
+
property asset_partition_keys
+
+
+ The partition keys for the output asset.
+
+ Raises an error if the output asset has no partitioning.
+
+
+
+
+
+
+
property asset_partitions_def
+
+ The PartitionsDefinition on the asset corresponding to this output.
+
+
+
+
+
property asset_partitions_time_window
+
+
+ The time window for the partitions of the output asset.
+
+ Raises an error if either of the following are true:
+ - The output asset has no partitioning.
+ - The output asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+
+
+
+
+
+
property asset_spec
+
+ The `AssetSpec` that is being stored as an output.
+
+
+
+
+
property config
+
+ The configuration for the output.
+
+
+
+
+
property dagster_type
+
+ The type of this output.
+
+
+
+
+
property definition_metadata
+
+ A dict of the metadata that is assigned to the OutputDefinition that produced
+ the output. Metadata is assigned to an OutputDefinition either directly on the OutputDefinition
+ or in the @asset decorator.
+
+
+
+
+
property has_asset_key
+
+ Returns True if an asset is being stored, otherwise returns False. A return value of False
+ indicates that an output from an op is being stored.
+
+
+
+
+
property has_asset_partitions
+
+ Returns True if the asset being stored is partitioned.
+
+
+
+
+
property has_partition_key
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property log
+
+ The log manager to use for this output.
+
+
+
+
+
property mapping_key
+
+ The key that identifies a unique mapped output. None for regular outputs.
+
+
+
+
+
property metadata
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.0.
+ Use definition_metadata instead.
+
+ :::
+
+ used definition_metadata instead.
+
+ Type: Deprecated
+
+
+
+
+
+
property name
+
+ The name of the output that produced the output.
+
+
+
+
+
property op_def
+
+ The definition of the op that produced the output.
+
+
+
+
+
property output_metadata
+
+ A dict of the metadata that is assigned to the output at execution time.
+
+
+
+
+
property partition_key
+
+
+ The partition key for the current run.
+
+ Raises an error if the current run is not a partitioned run.
+
+
+
+
+
+
+
property resource_config
+
+ The config associated with the resource that initializes the InputManager.
+
+
+
+
+
property resources
+
+ The resources required by the output manager, specified by the required_resource_keys
+ parameter.
+
+
+
+
+
property run_id
+
+ The id of the run that produced the output.
+
+
+
+
+
property step_key
+
+ The step_key for the compute step that produced the output.
+
+
+
+
+
property version
+
+ (Experimental) The version of the output.
+
+
+
+
+
+
+
+
+
dagster.build_input_context
+
+
+ Builds input context from provided parameters.
+
+ `build_input_context` can be used as either a function, or a context manager. If resources
+ that are also context managers are provided, then `build_input_context` must be used as a
+ context manager.
+
+ Parameters:
+ - name (Optional[str]) – The name of the input that we’re loading.
+ - config (Optional[Any]) – The config attached to the input that we’re loading.
+ - definition_metadata (Optional[Dict[str, Any]]) – A dict of metadata that is assigned to the
+ - upstream_output (Optional[[*OutputContext*](#dagster.OutputContext)]) – Info about the output that produced the object
+ - dagster_type (Optional[[*DagsterType*](types.mdx#dagster.DagsterType)]) – The type of this input.
+ - resource_config (Optional[Dict[str, Any]]) – The resource config to make available from the
+ - resources (Optional[Dict[str, Any]]) – The resources to make available from the context.
+ - asset_key (Optional[Union[[*AssetKey*](assets.mdx#dagster.AssetKey), Sequence[str], str]]) – The asset key attached to the InputDefinition.
+ - op_def (Optional[[*OpDefinition*](ops.mdx#dagster.OpDefinition)]) – The definition of the op that’s loading the input.
+ - step_context (Optional[[*StepExecutionContext*](internals.mdx#dagster.StepExecutionContext)]) – For internal use.
+ - partition_key (Optional[str]) – String value representing partition key to execute with.
+ - asset_partition_key_range (Optional[[*PartitionKeyRange*](partitions.mdx#dagster.PartitionKeyRange)]) – The range of asset partition keys
+ - asset_partitions_def – Optional[PartitionsDefinition]: The PartitionsDefinition of the asset
+
+
+ Examples:
+
+ ```python
+ build_input_context()
+
+ with build_input_context(resources={"foo": context_manager_resource}) as context:
+ do_something
+ ```
+
+
+
+
+
+
dagster.build_output_context
+
+
+ Builds output context from provided parameters.
+
+ `build_output_context` can be used as either a function, or a context manager. If resources
+ that are also context managers are provided, then `build_output_context` must be used as a
+ context manager.
+
+ Parameters:
+ - step_key (Optional[str]) – The step_key for the compute step that produced the output.
+ - name (Optional[str]) – The name of the output that produced the output.
+ - definition_metadata (Optional[Mapping[str, Any]]) – A dict of the metadata that is assigned to the
+ - mapping_key (Optional[str]) – The key that identifies a unique mapped output. None for regular outputs.
+ - config (Optional[Any]) – The configuration for the output.
+ - dagster_type (Optional[[*DagsterType*](types.mdx#dagster.DagsterType)]) – The type of this output.
+ - version (Optional[str]) – (Experimental) The version of the output.
+ - resource_config (Optional[Mapping[str, Any]]) – The resource config to make available from the
+ - resources (Optional[Resources]) – The resources to make available from the context.
+ - op_def (Optional[[*OpDefinition*](ops.mdx#dagster.OpDefinition)]) – The definition of the op that produced the output.
+ - asset_key – Optional[Union[AssetKey, Sequence[str], str]]: The asset key corresponding to the
+ - partition_key – Optional[str]: String value representing partition key to execute with.
+ - metadata (Optional[Mapping[str, Any]]) – deprecateddefinition_metadata instead.) Deprecated. Use definition_metadata instead.
+
+
+ Examples:
+
+ ```python
+ build_output_context()
+
+ with build_output_context(resources={"foo": context_manager_resource}) as context:
+ do_something
+ ```
+
+
+
+
+
+
+
+
+
+
+## Built-in IO Managers
+
+
+
dagster.FilesystemIOManager IOManagerDefinition
+
+
+ Built-in filesystem IO manager that stores and retrieves values using pickling.
+
+ The base directory that the pickle files live inside is determined by:
+
+ - The IO manager’s “base_dir” configuration value, if specified. Otherwise…
+ - A “storage/” directory underneath the value for “local_artifact_storage” in your dagster.yaml
+ - A “storage/” directory underneath the directory that the DAGSTER_HOME environment variable
+ - A temporary directory.
+
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ So, with a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ 1. Attach an IO manager to a set of assets using the reserved resource key `"io_manager"`.
+ ```python
+ from dagster import Definitions, asset, FilesystemIOManager
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": FilesystemIOManager(base_dir="/my/base/path")
+ },
+ )
+ ```
+ 2. Specify a job-level IO manager using the reserved resource key `"io_manager"`,
+ which will set the given IO manager on all ops in a job.
+
+ ```python
+ from dagster import FilesystemIOManager, job, op
+
+ @op
+ def op_a():
+ # create df ...
+ return df
+
+ @op
+ def op_b(df):
+ return df[:5]
+
+ @job(
+ resource_defs={
+ "io_manager": FilesystemIOManager(base_dir="/my/base/path")
+ }
+ )
+ def job():
+ op_b(op_a())
+ ```
+ 3. Specify IO manager on [`Out`](ops.mdx#dagster.Out), which allows you to set different IO managers on
+ different step outputs.
+
+ ```python
+ from dagster import FilesystemIOManager, job, op, Out
+
+ @op(out=Out(io_manager_key="my_io_manager"))
+ def op_a():
+ # create df ...
+ return df
+
+ @op
+ def op_b(df):
+ return df[:5]
+
+ @job(resource_defs={"my_io_manager": FilesystemIOManager()})
+ def job():
+ op_b(op_a())
+ ```
+
+
+
+
+
+
dagster.InMemoryIOManager IOManagerDefinition
+
+ I/O manager that stores and retrieves values in memory. After execution is complete, the values will
+ be garbage-collected. Note that this means that each run will not have access to values from previous runs.
+
+
+
+The `UPathIOManager` can be used to easily define filesystem-based IO Managers.
+
+
+
class dagster.UPathIOManager
+
+
+ Abstract IOManager base class compatible with local and cloud storage via universal-pathlib and fsspec.
+
+ Features:
+ - handles partitioned assets
+ - handles loading a single upstream partition
+ - handles loading multiple upstream partitions (with respect to [`PartitionMapping`](partitions.mdx#dagster.PartitionMapping))
+ - supports loading multiple partitions concurrently with async load_from_path method
+ - the get_metadata method can be customized to add additional metadata to the output
+ - the allow_missing_partitions metadata value can be set to True to skip missing partitions
+
+
+
+
+
+
+
+
+
+
+
+
+## Input Managers (Experimental)
+
+Input managers load inputs from either upstream outputs or from provided default values.
+
+
+
@dagster.input_manager
+
+
+ Define an input manager.
+
+ Input managers load op inputs, either from upstream outputs or by providing default values.
+
+ The decorated function should accept a [`InputContext`](#dagster.InputContext) and resource config, and return
+ a loaded object that will be passed into one of the inputs of an op.
+
+ The decorator produces an [`InputManagerDefinition`](#dagster.InputManagerDefinition).
+
+ Parameters:
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the resource-level config. If not
+ - description (Optional[str]) – A human-readable description of the resource.
+ - input_config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – A schema for the input-level config. Each
+ - required_resource_keys (Optional[Set[str]]) – Keys for the resources required by the input
+ - version (Optional[str]) – (Experimental) the version of the input manager definition.
+
+
+ Examples:
+
+ ```python
+ from dagster import input_manager, op, job, In
+
+ @input_manager
+ def csv_loader(_):
+ return read_csv("some/path")
+
+ @op(ins={"input1": In(input_manager_key="csv_loader_key")})
+ def my_op(_, input1):
+ do_stuff(input1)
+
+ @job(resource_defs={"csv_loader_key": csv_loader})
+ def my_job():
+ my_op()
+
+ @input_manager(config_schema={"base_dir": str})
+ def csv_loader(context):
+ return read_csv(context.resource_config["base_dir"] + "/some/path")
+
+ @input_manager(input_config_schema={"path": str})
+ def csv_loader(context):
+ return read_csv(context.config["path"])
+ ```
+
+
+
+
+
+
class dagster.InputManager
+
+ Base interface for classes that are responsible for loading solid inputs.
+
+
+
+
+
class dagster.InputManagerDefinition
+
+
+ Definition of an input manager resource.
+
+ Input managers load op inputs.
+
+ An InputManagerDefinition is a [`ResourceDefinition`](resources.mdx#dagster.ResourceDefinition) whose resource_fn returns an
+ [`InputManager`](#dagster.InputManager).
+
+ The easiest way to create an InputManagerDefinition is with the
+ [`@input_manager`](#dagster.input_manager) decorator.
+
+
+
+
+
+
+
+
+
+
+## Legacy
+
+
+
dagster.fs_io_manager IOManagerDefinition
+
+
+ Built-in filesystem IO manager that stores and retrieves values using pickling.
+
+ The base directory that the pickle files live inside is determined by:
+
+ - The IO manager’s “base_dir” configuration value, if specified. Otherwise…
+ - A “storage/” directory underneath the value for “local_artifact_storage” in your dagster.yaml
+ - A “storage/” directory underneath the directory that the DAGSTER_HOME environment variable
+ - A temporary directory.
+
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ So, with a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ 1. Attach an IO manager to a set of assets using the reserved resource key `"io_manager"`.
+ ```python
+ from dagster import Definitions, asset, fs_io_manager
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": fs_io_manager.configured({"base_dir": "/my/base/path"})
+ },
+ )
+ ```
+ 2. Specify a job-level IO manager using the reserved resource key `"io_manager"`,
+ which will set the given IO manager on all ops in a job.
+
+ ```python
+ from dagster import fs_io_manager, job, op
+
+ @op
+ def op_a():
+ # create df ...
+ return df
+
+ @op
+ def op_b(df):
+ return df[:5]
+
+ @job(
+ resource_defs={
+ "io_manager": fs_io_manager.configured({"base_dir": "/my/base/path"})
+ }
+ )
+ def job():
+ op_b(op_a())
+ ```
+ 3. Specify IO manager on [`Out`](ops.mdx#dagster.Out), which allows you to set different IO managers on
+ different step outputs.
+
+ ```python
+ from dagster import fs_io_manager, job, op, Out
+
+ @op(out=Out(io_manager_key="my_io_manager"))
+ def op_a():
+ # create df ...
+ return df
+
+ @op
+ def op_b(df):
+ return df[:5]
+
+ @job(resource_defs={"my_io_manager": fs_io_manager})
+ def job():
+ op_b(op_a())
+ ```
+
+
+
+
+
+
dagster.mem_io_manager IOManagerDefinition
+
+ Built-in IO manager that stores and retrieves values in memory.
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/jobs.mdx b/docs/docs-beta/docs/api/python-api/jobs.mdx
new file mode 100644
index 0000000000000..3deb6ef9bfa62
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/jobs.mdx
@@ -0,0 +1,387 @@
+---
+title: 'jobs'
+title_meta: 'jobs API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'jobs Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Jobs
+
+A `Job` binds a `Graph` and the resources it needs to be executable.
+
+Jobs are created by calling `GraphDefinition.to_job()` on a graph instance, or using the `job` decorator.
+
+
+
@dagster.job
+
+
+ Creates a job with the specified parameters from the decorated graph/op invocation function.
+
+ Using this decorator allows you to build an executable job by writing a function that invokes
+ ops (or graphs).
+
+ Parameters:
+ - (Callable[... (compose_fn) – The decorated function. The body should contain op or graph invocations. Unlike op
+ - Any] – The decorated function. The body should contain op or graph invocations. Unlike op
+ - name (Optional[str]) – The name for the Job. Defaults to the name of the this graph.
+ - resource_defs (Optional[Mapping[str, object]]) – Resources that are required by this graph for execution.
+ - config –
+
+ Describes how the job is parameterized at runtime.
+
+ If no value is provided, then the schema for the job’s run config is a standard
+ format based on its ops and resources.
+
+ If a dictionary is provided, then it must conform to the standard config schema, and
+ it will be used as the job’s run config for the job whenever the job is executed.
+ The values provided will be viewable and editable in the Dagster UI, so be
+ careful with secrets.
+
+ If a [`RunConfig`](config.mdx#dagster.RunConfig) object is provided, then it will be used directly as the run config
+ for the job whenever the job is executed, similar to providing a dictionary.
+
+ If a [`ConfigMapping`](config.mdx#dagster.ConfigMapping) object is provided, then the schema for the job’s run config is
+ determined by the config mapping, and the ConfigMapping, which should return
+ configuration in the standard format to configure the job.
+
+ - tags (Optional[Mapping[str, object]]) – A set of key-value tags that annotate the job and can
+ - run_tags (Optional[Mapping[str, object]]) – A set of key-value tags that will be automatically attached to runs launched by this
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary information that will be attached to the JobDefinition and be viewable in the Dagster UI.
+ - logger_defs (Optional[Dict[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – A dictionary of string logger identifiers to their implementations.
+ - executor_def (Optional[[*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]) – How this Job will be executed. Defaults to [`multiprocess_executor`](execution.mdx#dagster.multiprocess_executor) .
+ - op_retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The default retry policy for all ops in this job.
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines a discrete set of partition keys
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of a job.
+
+
+ Examples:
+
+ ```python
+ @op
+ def return_one():
+ return 1
+
+ @op
+ def add_one(in1):
+ return in1 + 1
+
+ @job
+ def job1():
+ add_one(return_one())
+ ```
+
+
+
+
+
+
class dagster.JobDefinition
+
+
+ Defines a Dagster job.
+
+
+
execute_in_process
+
+
+ Execute the Job in-process, gathering results in-memory.
+
+ The executor_def on the Job will be ignored, and replaced with the in-process executor.
+ If using the default io_manager, it will switch from filesystem to in-memory.
+
+ Parameters:
+ - (Optional[Mapping[str (run_config) – The configuration for the run
+ - Any]] – The configuration for the run
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The instance to execute against, an ephemeral one will be used if none provided.
+ - partition_key – (Optional[str])
+ - raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur.
+ - op_selection (Optional[Sequence[str]]) – A list of op selection queries (including single op
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of the job. Input
+ - resources (Optional[Mapping[str, Any]]) – The resources needed if any are required. Can provide resource instances directly,
+
+
+ Returns: [`ExecuteInProcessResult`](execution.mdx#dagster.ExecuteInProcessResult)
+
+
+
+
+
+
run_request_for_partition
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.0.
+ Directly instantiate `RunRequest(partition_key=...)` instead..
+
+ :::
+
+ Creates a RunRequest object for a run that processes the given partition.
+
+ Parameters:
+ - partition_key – The key of the partition to request a run for.
+ - run_key (Optional[str]) – A string key to identify this launched run. For sensors, ensures that
+ - tags (Optional[Dict[str, str]]) – A dictionary of tags (string key-value pairs) to attach
+ - (Optional[Mapping[str (run_config) – Configuration for the run. If the job has
+ - Any]] – Configuration for the run. If the job has
+ - current_time (Optional[datetime]) – Used to determine which time-partitions exist.
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: an object that requests a run to process the given partition.Return type: [RunRequest](schedules-sensors.mdx#dagster.RunRequest)
+
+
+
+
+
+
with_hooks
+
+ Apply a set of hooks to all op instances within the job.
+
+
+
+
+
with_top_level_resources
+
+ Apply a set of resources to all op instances within the job.
+
+
+
+
+
property config_mapping
+
+
+ The config mapping for the job, if it has one.
+
+ A config mapping defines a way to map a top-level config schema to run config for the job.
+
+
+
+
+
+
+
property executor_def
+
+
+ Returns the default [`ExecutorDefinition`](internals.mdx#dagster.ExecutorDefinition) for the job.
+
+ If the user has not specified an executor definition, then this will default to the
+ [`multi_or_in_process_executor()`](execution.mdx#dagster.multi_or_in_process_executor). If a default is specified on the
+ [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to, then that will be used instead.
+
+
+
+
+
+
+
property has_specified_executor
+
+ Returns True if this job has explicitly specified an executor, and False if the executor
+ was inherited through defaults or the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to.
+
+
+
+
+
property has_specified_loggers
+
+ Returns true if the job explicitly set loggers, and False if loggers were inherited
+ through defaults or the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to.
+
+
+
+
+
property loggers
+
+
+ Returns the set of LoggerDefinition objects specified on the job.
+
+ If the user has not specified a mapping of [`LoggerDefinition`](loggers.mdx#dagster.LoggerDefinition) objects, then this
+ will default to the `colored_console_logger()` under the key console. If a default
+ is specified on the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to, then that will
+ be used instead.
+
+
+
+
+
+
+
property partitioned_config
+
+
+ The partitioned config for the job, if it has one.
+
+ A partitioned config defines a way to map partition keys to run config for the job.
+
+
+
+
+
+
+
property partitions_def
+
+
+ Returns the [`PartitionsDefinition`](partitions.mdx#dagster.PartitionsDefinition) for the job, if it has one.
+
+ A partitions definition defines the set of partition keys the job operates on.
+
+
+
+
+
+
+
property resource_defs
+
+
+ Returns the set of ResourceDefinition objects specified on the job.
+
+ This may not be the complete set of resources required by the job, since those can also be
+ provided on the [`Definitions`](definitions.mdx#dagster.Definitions) object the job may be provided to.
+
+
+
+
+
+
+
+
+
+
+
+## Reconstructable jobs
+
+
+
class dagster.reconstructable
+
+
+ Create a `ReconstructableJob` from a
+ function that returns a [`JobDefinition`](#dagster.JobDefinition)/[`JobDefinition`](#dagster.JobDefinition),
+ or a function decorated with [`@job`](#dagster.job).
+
+ When your job must cross process boundaries, e.g., for execution on multiple nodes or
+ in different systems (like `dagstermill`), Dagster must know how to reconstruct the job
+ on the other side of the process boundary.
+
+ Passing a job created with `~dagster.GraphDefinition.to_job` to `reconstructable()`,
+ requires you to wrap that job’s definition in a module-scoped function, and pass that function
+ instead:
+
+ ```python
+ from dagster import graph, reconstructable
+
+ @graph
+ def my_graph():
+ ...
+
+ def define_my_job():
+ return my_graph.to_job()
+
+ reconstructable(define_my_job)
+ ```
+ This function implements a very conservative strategy for reconstruction, so that its behavior
+ is easy to predict, but as a consequence it is not able to reconstruct certain kinds of jobs
+ or jobs, such as those defined by lambdas, in nested scopes (e.g., dynamically within a method
+ call), or in interactive environments such as the Python REPL or Jupyter notebooks.
+
+ If you need to reconstruct objects constructed in these ways, you should use
+ `build_reconstructable_job()` instead, which allows you to
+ specify your own reconstruction strategy.
+
+ Examples:
+
+ ```python
+ from dagster import job, reconstructable
+
+ @job
+ def foo_job():
+ ...
+
+ reconstructable_foo_job = reconstructable(foo_job)
+
+
+ @graph
+ def foo():
+ ...
+
+ def make_bar_job():
+ return foo.to_job()
+
+ reconstructable_bar_job = reconstructable(make_bar_job)
+ ```
+
+
+
+
+
+
dagster.build_reconstructable_job
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Create a `dagster._core.definitions.reconstructable.ReconstructableJob`.
+
+ When your job must cross process boundaries, e.g., for execution on multiple nodes or in
+ different systems (like `dagstermill`), Dagster must know how to reconstruct the job
+ on the other side of the process boundary.
+
+ This function allows you to use the strategy of your choice for reconstructing jobs, so
+ that you can reconstruct certain kinds of jobs that are not supported by
+ [`reconstructable()`](execution.mdx#dagster.reconstructable), such as those defined by lambdas, in nested scopes (e.g.,
+ dynamically within a method call), or in interactive environments such as the Python REPL or
+ Jupyter notebooks.
+
+ If you need to reconstruct jobs constructed in these ways, use this function instead of
+ [`reconstructable()`](execution.mdx#dagster.reconstructable).
+
+ Parameters:
+ - reconstructor_module_name (str) – The name of the module containing the function to use to
+ - reconstructor_function_name (str) – The name of the function to use to reconstruct the
+ - reconstructable_args (Tuple) – Args to the function to use to reconstruct the job.
+ - reconstructable_kwargs (Dict[str, Any]) – Kwargs to the function to use to reconstruct the
+
+
+ Examples:
+
+ ```python
+ # module: mymodule
+
+ from dagster import JobDefinition, job, build_reconstructable_job
+
+ class JobFactory:
+ def make_job(*args, **kwargs):
+
+ @job
+ def _job(...):
+ ...
+
+ return _job
+
+ def reconstruct_job(*args):
+ factory = JobFactory()
+ return factory.make_job(*args)
+
+ factory = JobFactory()
+
+ foo_job_args = (...,...)
+
+ foo_job_kwargs = {...:...}
+
+ foo_job = factory.make_job(*foo_job_args, **foo_job_kwargs)
+
+ reconstructable_foo_job = build_reconstructable_job(
+ 'mymodule',
+ 'reconstruct_job',
+ foo_job_args,
+ foo_job_kwargs,
+ )
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-airbyte.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-airbyte.mdx
new file mode 100644
index 0000000000000..f03dcce678eb4
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-airbyte.mdx
@@ -0,0 +1,462 @@
+---
+title: 'airbyte (dagster-airbyte)'
+title_meta: 'airbyte (dagster-airbyte) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'airbyte (dagster-airbyte) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Airbyte (dagster-airbyte)
+
+This library provides a Dagster integration with [Airbyte](https://www.airbyte.com/).
+
+For more information on getting started, see the [Airbyte integration guide](https://docs.dagster.io/integrations/libraries/airbyte/).
+
+
+
+
+## Assets (Airbyte API)
+
+
+
class dagster_airbyte.AirbyteCloudWorkspace
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This class represents a Airbyte Cloud workspace and provides utilities
+ to interact with Airbyte APIs.
+
+
+
sync_and_poll
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Executes a sync and poll process to materialize Airbyte Cloud assets.
+ This method can only be used in the context of an asset execution.
+
+ Parameters: context ([*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)) – The execution context
+ from within @airbyte_assets.Returns:
+ An iterator of MaterializeResult
+ or AssetMaterialization.
+
+ Return type: Iterator[Union[[AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [MaterializeResult](../assets.mdx#dagster.MaterializeResult)]]
+
+
+
+
+
+
+
+
+
+
class dagster_airbyte.DagsterAirbyteTranslator
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Translator class which converts a AirbyteConnectionTableProps object into AssetSpecs.
+ Subclass this class to implement custom logic how to translate Airbyte content into asset spec.
+
+
+
+
+
+
+
dagster_airbyte.load_airbyte_cloud_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Airbyte content in the workspace.
+
+ Parameters:
+ - workspace ([*AirbyteCloudWorkspace*](#dagster_airbyte.AirbyteCloudWorkspace)) – The Airbyte Cloud workspace to fetch assets from.
+ - dagster_airbyte_translator (Optional[[*DagsterAirbyteTranslator*](#dagster_airbyte.DagsterAirbyteTranslator)], optional) – The translator to use
+
+
+ Returns: The set of assets representing the Airbyte content in the workspace.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+ Examples:
+
+ Loading the asset specs for a given Airbyte Cloud workspace:
+
+ ```python
+ from dagster_airbyte import AirbyteCloudWorkspace, load_airbyte_cloud_asset_specs
+
+ import dagster as dg
+
+ airbyte_cloud_workspace = AirbyteCloudWorkspace(
+ workspace_id=dg.EnvVar("AIRBYTE_CLOUD_WORKSPACE_ID"),
+ client_id=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_ID"),
+ client_secret=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_SECRET"),
+ )
+
+
+ airbyte_cloud_specs = load_airbyte_cloud_asset_specs(airbyte_cloud_workspace)
+ defs = dg.Definitions(assets=airbyte_cloud_specs)
+ ```
+
+
+
+
+
+
@dagster_airbyte.airbyte_assets
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Create a definition for how to sync the tables of a given Airbyte connection.
+
+ Parameters:
+ - connection_id (str) – The Airbyte Connection ID.
+ - workspace ([*AirbyteCloudWorkspace*](#dagster_airbyte.AirbyteCloudWorkspace)) – The Airbyte workspace to fetch assets from.
+ - name (Optional[str], optional) – The name of the op.
+ - group_name (Optional[str], optional) – The name of the asset group.
+ - dagster_airbyte_translator (Optional[[*DagsterAirbyteTranslator*](#dagster_airbyte.DagsterAirbyteTranslator)], optional) – The translator to use
+
+
+ Examples:
+
+ Sync the tables of an Airbyte connection:
+
+ ```python
+ from dagster_airbyte import AirbyteCloudWorkspace, airbyte_assets
+
+ import dagster as dg
+
+ airbyte_workspace = AirbyteCloudWorkspace(
+ workspace_id=dg.EnvVar("AIRBYTE_CLOUD_WORKSPACE_ID"),
+ client_id=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_ID"),
+ client_secret=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_SECRET"),
+ )
+
+
+ @airbyte_assets(
+ connection_id="airbyte_connection_id",
+ workspace=airbyte_workspace,
+ )
+ def airbyte_connection_assets(context: dg.AssetExecutionContext, airbyte: AirbyteCloudWorkspace):
+ yield from airbyte.sync_and_poll(context=context)
+
+
+ defs = dg.Definitions(
+ assets=[airbyte_connection_assets],
+ resources={"airbyte": airbyte_workspace},
+ )
+ ```
+ Sync the tables of an Airbyte connection with a custom translator:
+
+ ```python
+ from dagster_airbyte import (
+ DagsterAirbyteTranslator,
+ AirbyteConnectionTableProps,
+ AirbyteCloudWorkspace,
+ airbyte_assets
+ )
+
+ import dagster as dg
+
+ class CustomDagsterAirbyteTranslator(DagsterAirbyteTranslator):
+ def get_asset_spec(self, props: AirbyteConnectionTableProps) -> dg.AssetSpec:
+ default_spec = super().get_asset_spec(props)
+ return default_spec.merge_attributes(
+ metadata={"custom": "metadata"},
+ )
+
+ airbyte_workspace = AirbyteCloudWorkspace(
+ workspace_id=dg.EnvVar("AIRBYTE_CLOUD_WORKSPACE_ID"),
+ client_id=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_ID"),
+ client_secret=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_SECRET"),
+ )
+
+
+ @airbyte_assets(
+ connection_id="airbyte_connection_id",
+ workspace=airbyte_workspace,
+ dagster_airbyte_translator=CustomDagsterAirbyteTranslator()
+ )
+ def airbyte_connection_assets(context: dg.AssetExecutionContext, airbyte: AirbyteCloudWorkspace):
+ yield from airbyte.sync_and_poll(context=context)
+
+
+ defs = dg.Definitions(
+ assets=[airbyte_connection_assets],
+ resources={"airbyte": airbyte_workspace},
+ )
+ ```
+
+
+
+
+
+
dagster_airbyte.build_airbyte_assets_definitions
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ The list of AssetsDefinition for all connections in the Airbyte workspace.
+
+ Parameters:
+ - workspace ([*AirbyteCloudWorkspace*](#dagster_airbyte.AirbyteCloudWorkspace)) – The Airbyte workspace to fetch assets from.
+ - dagster_airbyte_translator (Optional[[*DagsterAirbyteTranslator*](#dagster_airbyte.DagsterAirbyteTranslator)], optional) – The translator to use
+
+
+ Returns: The list of AssetsDefinition for all connections in the Airbyte workspace.Return type: List[[AssetsDefinition](../assets.mdx#dagster.AssetsDefinition)]
+ Examples:
+
+ Sync the tables of a Airbyte connection:
+
+ ```python
+ from dagster_airbyte import AirbyteCloudWorkspace, build_airbyte_assets_definitions
+
+ import dagster as dg
+
+ airbyte_workspace = AirbyteCloudWorkspace(
+ workspace_id=dg.EnvVar("AIRBYTE_CLOUD_WORKSPACE_ID"),
+ client_id=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_ID"),
+ client_secret=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_SECRET"),
+ )
+
+
+ airbyte_assets = build_airbyte_assets_definitions(workspace=workspace)
+
+ defs = dg.Definitions(
+ assets=airbyte_assets,
+ resources={"airbyte": airbyte_workspace},
+ )
+ ```
+ Sync the tables of a Airbyte connection with a custom translator:
+
+ ```python
+ from dagster_airbyte import (
+ DagsterAirbyteTranslator,
+ AirbyteConnectionTableProps,
+ AirbyteCloudWorkspace,
+ build_airbyte_assets_definitions
+ )
+
+ import dagster as dg
+
+ class CustomDagsterAirbyteTranslator(DagsterAirbyteTranslator):
+ def get_asset_spec(self, props: AirbyteConnectionTableProps) -> dg.AssetSpec:
+ default_spec = super().get_asset_spec(props)
+ return default_spec.merge_attributes(
+ metadata={"custom": "metadata"},
+ )
+
+ airbyte_workspace = AirbyteCloudWorkspace(
+ workspace_id=dg.EnvVar("AIRBYTE_CLOUD_WORKSPACE_ID"),
+ client_id=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_ID"),
+ client_secret=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_SECRET"),
+ )
+
+
+ airbyte_assets = build_airbyte_assets_definitions(
+ workspace=workspace,
+ dagster_airbyte_translator=CustomDagsterAirbyteTranslator()
+ )
+
+ defs = dg.Definitions(
+ assets=airbyte_assets,
+ resources={"airbyte": airbyte_workspace},
+ )
+ ```
+
+
+
+ This resource allows users to programatically interface with the Airbyte REST API to launch
+ syncs and monitor their progress.
+
+ Examples:
+
+ ```python
+ from dagster import job, EnvVar
+ from dagster_airbyte import AirbyteResource
+
+ my_airbyte_resource = AirbyteResource(
+ host=EnvVar("AIRBYTE_HOST"),
+ port=EnvVar("AIRBYTE_PORT"),
+ # If using basic auth
+ username=EnvVar("AIRBYTE_USERNAME"),
+ password=EnvVar("AIRBYTE_PASSWORD"),
+ )
+
+ airbyte_assets = build_airbyte_assets(
+ connection_id="87b7fe85-a22c-420e-8d74-b30e7ede77df",
+ destination_tables=["releases", "tags", "teams"],
+ )
+
+ defs = Definitions(
+ assets=[airbyte_assets],
+ resources={"airbyte": my_airbyte_resource},
+ )
+ ```
+
+
+
+
+
+
dagster_airbyte.load_assets_from_airbyte_instance
+
+
+ Loads Airbyte connection assets from a configured AirbyteResource instance. This fetches information
+ about defined connections at initialization time, and will error on workspace load if the Airbyte
+ instance is not reachable.
+
+ Parameters:
+ - airbyte ([*ResourceDefinition*](../resources.mdx#dagster.ResourceDefinition)) – An AirbyteResource configured with the appropriate connection
+ - workspace_id (Optional[str]) – The ID of the Airbyte workspace to load connections from. Only
+ - key_prefix (Optional[CoercibleToAssetKeyPrefix]) – A prefix for the asset keys created.
+ - create_assets_for_normalization_tables (bool) – If True, assets will be created for tables
+ - connection_to_group_fn (Optional[Callable[[str], Optional[str]]]) – Function which returns an asset
+ - connection_meta_to_group_fn (Optional[Callable[[AirbyteConnectionMetadata], Optional[str]]]) – Function which
+ - io_manager_key (Optional[str]) – The I/O manager key to use for all assets. Defaults to “io_manager”.
+ - connection_to_io_manager_key_fn (Optional[Callable[[str], Optional[str]]]) – Function which returns an
+ - connection_filter (Optional[Callable[[AirbyteConnectionMetadata], bool]]) – Optional function which takes
+ - connection_to_asset_key_fn (Optional[Callable[[AirbyteConnectionMetadata, str], [*AssetKey*](../assets.mdx#dagster.AssetKey)]]) – Optional function which
+ - connection_to_freshness_policy_fn (Optional[Callable[[AirbyteConnectionMetadata], Optional[FreshnessPolicy]]]) – Optional function
+ - connection_to_auto_materialize_policy_fn (Optional[Callable[[AirbyteConnectionMetadata], Optional[AutoMaterializePolicy]]]) – Optional
+
+
+ Examples:
+
+ Loading all Airbyte connections as assets:
+
+ ```python
+ from dagster_airbyte import airbyte_resource, load_assets_from_airbyte_instance
+
+ airbyte_instance = airbyte_resource.configured(
+ {
+ "host": "localhost",
+ "port": "8000",
+ }
+ )
+ airbyte_assets = load_assets_from_airbyte_instance(airbyte_instance)
+ ```
+ Filtering the set of loaded connections:
+
+ ```python
+ from dagster_airbyte import airbyte_resource, load_assets_from_airbyte_instance
+
+ airbyte_instance = airbyte_resource.configured(
+ {
+ "host": "localhost",
+ "port": "8000",
+ }
+ )
+ airbyte_assets = load_assets_from_airbyte_instance(
+ airbyte_instance,
+ connection_filter=lambda meta: "snowflake" in meta.name,
+ )
+ ```
+
+
+
+
+
+
dagster_airbyte.build_airbyte_assets
+
+
+ Builds a set of assets representing the tables created by an Airbyte sync operation.
+
+ Parameters:
+ - connection_id (str) – The Airbyte Connection ID that this op will sync. You can retrieve this
+ - destination_tables (List[str]) – The names of the tables that you want to be represented
+ - destination_database (Optional[str]) – The name of the destination database.
+ - destination_schema (Optional[str]) – The name of the destination schema.
+ - normalization_tables (Optional[Mapping[str, List[str]]]) – If you are using Airbyte’s
+ - asset_key_prefix (Optional[List[str]]) – A prefix for the asset keys inside this asset.
+ - deps (Optional[Sequence[Union[[*AssetsDefinition*](../assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](../assets.mdx#dagster.SourceAsset), str, [*AssetKey*](../assets.mdx#dagster.AssetKey)]]]) – A list of assets to add as sources.
+ - upstream_assets (Optional[Set[[*AssetKey*](../assets.mdx#dagster.AssetKey)]]) – Deprecated, use deps instead. A list of assets to add as sources.
+ - freshness_policy (Optional[FreshnessPolicy]) – A freshness policy to apply to the assets
+ - stream_to_asset_map (Optional[Mapping[str, str]]) – A mapping of an Airbyte stream name to a Dagster asset.
+ - auto_materialize_policy (Optional[AutoMaterializePolicy]) – An auto materialization policy to apply to the assets.
+
+
+
+
+
+
+
+
dagster_airbyte.airbyte_sync_op `=` \
+
+
+
+
+ Executes a Airbyte job sync for a given `connection_id`, and polls until that sync
+ completes, raising an error if it is unsuccessful. It outputs a AirbyteOutput which contains
+ the job details for a given `connection_id`.
+
+ It requires the use of the `airbyte_resource`, which allows it to
+ communicate with the Airbyte API.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_airbyte import airbyte_resource, airbyte_sync_op
+
+ my_airbyte_resource = airbyte_resource.configured(
+ {
+ "host": {"env": "AIRBYTE_HOST"},
+ "port": {"env": "AIRBYTE_PORT"},
+ }
+ )
+
+ sync_foobar = airbyte_sync_op.configured({"connection_id": "foobar"}, name="sync_foobar")
+
+ @job(resource_defs={"airbyte": my_airbyte_resource})
+ def my_simple_airbyte_job():
+ sync_foobar()
+
+ @job(resource_defs={"airbyte": my_airbyte_resource})
+ def my_composed_airbyte_job():
+ final_foobar_state = sync_foobar(start_after=some_op())
+ other_op(final_foobar_state)
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-airlift.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-airlift.mdx
new file mode 100644
index 0000000000000..2f6c9de25ea00
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-airlift.mdx
@@ -0,0 +1,920 @@
+---
+title: 'airlift (dagster-airlift)'
+title_meta: 'airlift (dagster-airlift) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'airlift (dagster-airlift) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Airlift (dagster-airlift)
+
+
+
+
+## Core (dagster_airlift.core)
+
+
+
+
+### AirflowInstance
+
+
+
class dagster_airlift.core.AirflowInstance
+
+
+ A class that represents a running Airflow Instance and provides methods for interacting with its REST API.
+
+ Parameters:
+ - auth_backend ([*AirflowAuthBackend*](#dagster_airlift.core.AirflowAuthBackend)) – The authentication backend to use when making requests to the Airflow instance.
+ - name (str) – The name of the Airflow instance. This will be prefixed to any assets automatically created using this instance.
+ - batch_task_instance_limit (int) – The number of task instances to query at a time when fetching task instances. Defaults to 100.
+ - batch_dag_runs_limit (int) – The number of dag runs to query at a time when fetching dag runs. Defaults to 100.
+
+
+
+
get_run_state
+
+
+ Given a run ID of an airflow dag, return the state of that run.
+
+ Parameters:
+ - dag_id (str) – The dag id.
+ - run_id (str) – The run id.
+
+
+ Returns: The state of the run. Will be one of the states defined by Airflow.Return type: str
+
+
+
+
+
+
trigger_dag
+
+
+ Trigger a dag run for the given dag_id.
+
+ Does not wait for the run to finish. To wait for the completed run to finish, use [`wait_for_run_completion()`](#dagster_airlift.core.AirflowInstance.wait_for_run_completion).
+
+ Parameters:
+ - dag_id (str) – The dag id to trigger.
+ - logical_date (Optional[datetime.datetime]) – The Airflow logical_date to use for the dag run. If not provided, the current time will be used. Previously known as execution_date in Airflow; find more information in the Airflow docs: [https://airflow.apache.org/docs/apache-airflow/stable/faq.html#what-does-execution-date-mean](https://airflow.apache.org/docs/apache-airflow/stable/faq.html#what-does-execution-date-mean)
+
+
+ Returns: The dag run id.Return type: str
+
+
+
+
+
+
wait_for_run_completion
+
+
+ Given a run ID of an airflow dag, wait for that run to reach a completed state.
+
+ Parameters:
+ - dag_id (str) – The dag id.
+ - run_id (str) – The run id.
+ - timeout (int) – The number of seconds to wait before timing out.
+
+
+ Returns: None
+
+
+
+
+
+
+
+
+
+
class dagster_airlift.core.AirflowAuthBackend
+
+
+ An abstract class that represents an authentication backend for an Airflow instance.
+
+ Requires two methods to be implemented by subclasses:
+ - get_session: Returns a requests.Session object that can be used to make requests to the Airflow instance, and handles authentication.
+ - get_webserver_url: Returns the base URL of the Airflow webserver.
+
+ The dagster-airlift package provides the following default implementations:
+ - `dagster-airlift.core.AirflowBasicAuthBackend`: An authentication backend that uses Airflow’s basic auth to authenticate with the Airflow instance.
+ - `dagster-airlift.mwaa.MwaaSessionAuthBackend`: An authentication backend that uses AWS MWAA’s web login token to authenticate with the Airflow instance (requires dagster-airlift[mwaa]).
+
+
+
+
+
+
+
class dagster_airlift.core.AirflowBasicAuthBackend
+
+
+ A [`dagster_airlift.core.AirflowAuthBackend`](#dagster_airlift.core.AirflowAuthBackend) that authenticates using basic auth.
+
+ Parameters:
+ - webserver_url (str) – The URL of the webserver.
+ - username (str) – The username to authenticate with.
+ - password (str) – The password to authenticate with.
+
+
+ Examples:
+
+ Creating a [`AirflowInstance`](#dagster_airlift.core.AirflowInstance) using this backend.
+
+ ```python
+ from dagster_airlift.core import AirflowInstance, AirflowBasicAuthBackend
+
+ af_instance = AirflowInstance(
+ name="my-instance",
+ auth_backend=AirflowBasicAuthBackend(
+ webserver_url="https://my-webserver-hostname",
+ username="my-username",
+ password="my-password"
+ )
+ )
+ ```
+
+
+
+ Builds a [`dagster.Definitions`](../definitions.mdx#dagster.Definitions) object from an Airflow instance.
+
+ For every DAG in the Airflow instance, this function will create a Dagster asset for the DAG
+ with an asset key instance_name/dag/dag_id. It will also create a sensor that polls the Airflow
+ instance for DAG runs and emits Dagster events for each successful run.
+
+ An optional defs argument can be provided, where the user can pass in a [`dagster.Definitions`](../definitions.mdx#dagster.Definitions)
+ object containing assets which are mapped to Airflow DAGs and tasks. These assets will be enriched with
+ metadata from the Airflow instance, and placed upstream of the automatically generated DAG assets.
+
+ An optional event_transformer_fn can be provided, which allows the user to modify the Dagster events
+ produced by the sensor. The function takes the Dagster events produced by the sensor and returns a sequence
+ of Dagster events.
+
+ An optional dag_selector_fn can be provided, which allows the user to filter which DAGs assets are created for.
+ The function takes a [`dagster_airlift.core.serialization.serialized_data.DagInfo`](#dagster_airlift.core.DagInfo) object and returns a
+ boolean indicating whether the DAG should be included.
+
+ Parameters:
+ - airflow_instance ([*AirflowInstance*](#dagster_airlift.core.AirflowInstance)) – The Airflow instance to build assets and the sensor from.
+ - defs – Optional[Definitions]: A [`dagster.Definitions`](../definitions.mdx#dagster.Definitions) object containing assets that are
+ - sensor_minimum_interval_seconds (int) – The minimum interval in seconds between sensor runs.
+ - event_transformer_fn (DagsterEventTransformerFn) – A function that allows for modifying the Dagster events
+ - dag_selector_fn (Optional[DagSelectorFn]) – A function that allows for filtering which DAGs assets are created for.
+
+
+ Returns: A [`dagster.Definitions`](../definitions.mdx#dagster.Definitions) object containing the assets and sensor.Return type: [Definitions](../definitions.mdx#dagster.Definitions)
+ Examples:
+
+ Building a [`dagster.Definitions`](../definitions.mdx#dagster.Definitions) object from an Airflow instance.
+
+ ```python
+ from dagster_airlift.core import (
+ AirflowInstance,
+ AirflowBasicAuthBackend,
+ build_defs_from_airflow_instance,
+ )
+
+ from .constants import AIRFLOW_BASE_URL, AIRFLOW_INSTANCE_NAME, PASSWORD, USERNAME
+
+ airflow_instance = AirflowInstance(
+ auth_backend=AirflowBasicAuthBackend(
+ webserver_url=AIRFLOW_BASE_URL, username=USERNAME, password=PASSWORD
+ ),
+ name=AIRFLOW_INSTANCE_NAME,
+ )
+
+
+ defs = build_defs_from_airflow_instance(airflow_instance=airflow_instance)
+ ```
+ Providing task-mapped assets to the function.
+
+ ```python
+ from dagster import Definitions
+ from dagster_airlift.core import (
+ AirflowInstance,
+ AirflowBasicAuthBackend,
+ assets_with_task_mappings,
+ build_defs_from_airflow_instance,
+ )
+ ...
+
+
+ defs = build_defs_from_airflow_instance(
+ airflow_instance=airflow_instance, # same as above
+ defs=Definitions(
+ assets=assets_with_task_mappings(
+ dag_id="rebuild_iris_models",
+ task_mappings={
+ "my_task": [AssetSpec("my_first_asset"), AssetSpec("my_second_asset")],
+ },
+ ),
+ ),
+ )
+ ```
+ Providing a custom event transformer function.
+
+ ```python
+ from typing import Sequence
+ from dagster import Definitions, SensorEvaluationContext
+ from dagster_airlift.core import (
+ AirflowInstance,
+ AirflowBasicAuthBackend,
+ AssetEvent,
+ assets_with_task_mappings,
+ build_defs_from_airflow_instance,
+ AirflowDefinitionsData,
+ )
+ ...
+
+ def add_tags_to_events(
+ context: SensorEvaluationContext,
+ defs_data: AirflowDefinitionsData,
+ events: Sequence[AssetEvent]
+ ) -> Sequence[AssetEvent]:
+ altered_events = []
+ for event in events:
+ altered_events.append(event._replace(tags={"my_tag": "my_value"}))
+ return altered_events
+
+ defs = build_defs_from_airflow_instance(
+ airflow_instance=airflow_instance, # same as above
+ event_transformer_fn=add_tags_to_events,
+ )
+ ```
+ Filtering which DAGs assets are created for.
+
+ ```python
+ from dagster import Definitions
+ from dagster_airlift.core import (
+ AirflowInstance,
+ AirflowBasicAuthBackend,
+ AssetEvent,
+ assets_with_task_mappings,
+ build_defs_from_airflow_instance,
+ DagInfo,
+ )
+ ...
+
+ def only_include_dag(dag_info: DagInfo) -> bool:
+ return dag_info.dag_id == "my_dag_id"
+
+ defs = build_defs_from_airflow_instance(
+ airflow_instance=airflow_instance, # same as above
+ dag_selector_fn=only_include_dag,
+ )
+ ```
+
+
+
+ Modify assets to be associated with a particular task in Airlift tooling.
+
+ Used in concert with build_defs_from_airflow_instance to observe an airflow
+ instance to monitor the tasks that are associated with the assets and
+ keep their materialization histories up to date.
+
+ Concretely this adds metadata to all asset specs in the provided definitions
+ with the provided dag_id and task_id. The dag_id comes from the dag_id argument;
+ the task_id comes from the key of the provided task_mappings dictionary.
+ There is a single metadata key “airlift/task-mapping” that is used to store
+ this information. It is a list of dictionaries with keys “dag_id” and “task_id”.
+
+ Example:
+
+ ```python
+ from dagster import AssetSpec, Definitions, asset
+ from dagster_airlift.core import assets_with_task_mappings
+
+ @asset
+ def asset_one() -> None: ...
+
+ defs = Definitions(
+ assets=assets_with_task_mappings(
+ dag_id="dag_one",
+ task_mappings={
+ "task_one": [asset_one],
+ "task_two": [AssetSpec(key="asset_two"), AssetSpec(key="asset_three")],
+ },
+ )
+ )
+ ```
+
+
+
+
+
+
dagster_airlift.core.assets_with_dag_mappings
+
+
+ Modify assets to be associated with a particular dag in Airlift tooling.
+
+ Used in concert with build_defs_from_airflow_instance to observe an airflow
+ instance to monitor the dags that are associated with the assets and
+ keep their materialization histories up to date.
+
+ In contrast with assets_with_task_mappings, which maps assets on a per-task basis, this is used in concert with
+ proxying_to_dagster dag-level mappings where an entire dag is migrated at once.
+
+ Concretely this adds metadata to all asset specs in the provided definitions
+ with the provided dag_id. The dag_id comes from the key of the provided dag_mappings dictionary.
+ There is a single metadata key “airlift/dag-mapping” that is used to store
+ this information. It is a list of strings, where each string is a dag_id which the asset is associated with.
+
+ Example:
+
+ ```python
+ from dagster import AssetSpec, Definitions, asset
+ from dagster_airlift.core import assets_with_dag_mappings
+
+ @asset
+ def asset_one() -> None: ...
+
+ defs = Definitions(
+ assets=assets_with_dag_mappings(
+ dag_mappings={
+ "dag_one": [asset_one],
+ "dag_two": [AssetSpec(key="asset_two"), AssetSpec(key="asset_three")],
+ },
+ )
+ )
+ ```
+
+
+
+ Given an asset or assets definition, return a new asset or assets definition with metadata
+ that indicates that it is targeted by multiple airflow tasks. An example of this would
+ be a separate weekly and daily dag that contains a task that targets a single asset.
+
+ ```python
+ from dagster import Definitions, AssetSpec, asset
+ from dagster_airlift import (
+ build_defs_from_airflow_instance,
+ targeted_by_multiple_tasks,
+ assets_with_task_mappings,
+ )
+
+ # Asset maps to a single task.
+ @asset
+ def other_asset(): ...
+
+ # Asset maps to a physical entity which is produced by two different airflow tasks.
+ @asset
+ def scheduled_twice(): ...
+
+ defs = build_defs_from_airflow_instance(
+ airflow_instance=airflow_instance,
+ defs=Definitions(
+ assets=[
+ *assets_with_task_mappings(
+ dag_id="other_dag",
+ task_mappings={
+ "task1": [other_asset]
+ },
+ ),
+ *assets_with_multiple_task_mappings(
+ assets=[scheduled_twice],
+ task_handles=[
+ {"dag_id": "weekly_dag", "task_id": "task1"},
+ {"dag_id": "daily_dag", "task_id": "task1"},
+ ],
+ ),
+ ]
+ ),
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+#### Annotations for customizable components:
+
+
+
dagster_airlift.core.DagSelectorFn
+
+ alias of `Callable`[[[`DagInfo`](#dagster_airlift.core.DagInfo)], `bool`]
+
+
+
+
+
dagster_airlift.core.DagsterEventTransformerFn
+
+ alias of `Callable`[[`SensorEvaluationContext`, [`AirflowDefinitionsData`](#dagster_airlift.core.AirflowDefinitionsData), `Sequence`[[`AssetMaterialization`](../ops.mdx#dagster.AssetMaterialization)]], `Iterable`[[`AssetMaterialization`](../ops.mdx#dagster.AssetMaterialization) | [`AssetObservation`](../assets.mdx#dagster.AssetObservation) | `AssetCheckEvaluation`]]
+
+
+
+
+
class dagster_airlift.core.TaskHandleDict
+
+
+
+
+
+
+
+
+
+
+
+#### Objects for retrieving information about the Airflow/Dagster mapping:
+
+
+
class dagster_airlift.core.DagInfo
+
+
+ A record containing information about a given airflow dag.
+
+ Users should not instantiate this class directly. It is provided when customizing which DAGs are included
+ in the generated definitions using the dag_selector_fn argument of [`build_defs_from_airflow_instance()`](#dagster_airlift.core.build_defs_from_airflow_instance).
+
+
+
metadata
+
+
+ The metadata associated with the dag, retrieved by the Airflow REST API:
+ [https://airflow.apache.org/docs/apache-airflow/stable/stable-rest-api-ref.html#operation/get_dags](https://airflow.apache.org/docs/apache-airflow/stable/stable-rest-api-ref.html#operation/get_dags)
+
+ Type: Dict[str, Any]
+
+
+
+
+
+
+
+
+
+
class dagster_airlift.core.AirflowDefinitionsData
+
+
+ A class that holds data about the assets that are mapped to Airflow dags and tasks, and
+ provides methods for retrieving information about the mappings.
+ The user should not instantiate this class directly. It is provided when customizing the events
+ that are generated by the Airflow sensor using the event_transformer_fn argument of
+ [`build_defs_from_airflow_instance()`](#dagster_airlift.core.build_defs_from_airflow_instance).
+
+
+
asset_keys_in_task
+
+
+ Returns the asset keys that are mapped to the given task.
+
+ Parameters:
+ - dag_id (str) – The dag id.
+ - task_id (str) – The task id.
+
+
+
+
+
+
+
+
task_ids_in_dag
+
+
+ Returns the task ids within the given dag_id.
+
+ Parameters: dag_id (str) – The dag id.
+
+
+
+
+
+
property instance_name
+
+ The name of the Airflow instance.
+
+
+
+
+
+
+
+
+
+
+
+
+
+## MWAA (dagster_airlift.mwaa)
+
+
+
class dagster_airlift.mwaa.MwaaSessionAuthBackend
+
+
+ A [`dagster_airlift.core.AirflowAuthBackend`](#dagster_airlift.core.AirflowAuthBackend) that authenticates to AWS MWAA.
+
+ Under the hood, this class uses the MWAA boto3 session to request a web login token and then
+ uses the token to authenticate to the MWAA web server.
+
+ Parameters:
+ - mwaa_session (boto3.Session) – The boto3 MWAA session
+ - env_name (str) – The name of the MWAA environment
+
+
+ Examples:
+
+ Creating an AirflowInstance pointed at a MWAA environment.
+
+ ```python
+ import boto3
+ from dagster_airlift.mwaa import MwaaSessionAuthBackend
+ from dagster_airlift.core import AirflowInstance
+
+ boto_client = boto3.client("mwaa")
+ af_instance = AirflowInstance(
+ name="my-mwaa-instance",
+ auth_backend=MwaaSessionAuthBackend(
+ mwaa_client=boto_client,
+ env_name="my-mwaa-env"
+ )
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+## In Airflow (dagster_airlift.in_airflow)
+
+
+
+### Proxying
+
+
+
dagster_airlift.in_airflow.proxying_to_dagster
+
+
+ Proxies tasks and dags to Dagster based on provided proxied state.
+ Expects a dictionary of in-scope global variables to be provided (typically retrieved with globals()), and a proxied state dictionary
+ (typically retrieved with [`load_proxied_state_from_yaml()`](#dagster_airlift.in_airflow.load_proxied_state_from_yaml)) for dags in that global state. This function will modify in-place the
+ dictionary of global variables to replace proxied tasks with appropriate Dagster operators.
+
+ In the case of task-level proxying, the proxied tasks will be replaced with new operators that are constructed by the provided build_from_task_fn.
+ A default implementation of this function is provided in DefaultProxyTaskToDagsterOperator.
+ In the case of dag-level proxying, the entire dag structure will be replaced with a single task that is constructed by the provided build_from_dag_fn.
+ A default implementation of this function is provided in DefaultProxyDAGToDagsterOperator.
+
+ Parameters:
+ - global_vars (Dict[str, Any]) – The global variables in the current context. In most cases, retrieved with globals() (no import required).
+ - proxied_state (AirflowMigrationState) – The proxied state for the dags.
+ - logger (Optional[logging.Logger]) – The logger to use. Defaults to logging.getLogger(“dagster_airlift”).
+
+
+ Examples:
+
+ Typical usage of this function is to be called at the end of a dag file, retrieving proxied_state from an accompanying proxied_state path.
+
+ ```python
+ from pathlib import Path
+
+ from airflow import DAG
+ from airflow.operators.python import PythonOperator
+ from dagster._time import get_current_datetime_midnight
+ from dagster_airlift.in_airflow import proxying_to_dagster
+ from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml
+
+
+ with DAG(
+ dag_id="daily_interval_dag",
+ ...,
+ ) as minute_dag:
+ PythonOperator(task_id="my_task", python_callable=...)
+
+ # At the end of the dag file, so we can ensure dags are loaded into globals.
+ proxying_to_dagster(
+ proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"),
+ global_vars=globals(),
+ )
+ ```
+ You can also provide custom implementations of the build_from_task_fn function to customize the behavior of task-level proxying.
+
+ ```python
+ from dagster_airlift.in_airflow import proxying_to_dagster, BaseProxyTaskToDagsterOperator
+ from airflow.models.operator import BaseOperator
+
+ ... # Dag code here
+
+ class CustomAuthTaskProxyOperator(BaseProxyTaskToDagsterOperator):
+ def get_dagster_session(self, context: Context) -> requests.Session:
+ # Add custom headers to the session
+ return requests.Session(headers={"Authorization": "Bearer my_token"})
+
+ def get_dagster_url(self, context: Context) -> str:
+ # Use a custom environment variable for the dagster url
+ return os.environ["CUSTOM_DAGSTER_URL"]
+
+ @classmethod
+ def build_from_task(cls, task: BaseOperator) -> "CustomAuthTaskProxyOperator":
+ # Custom logic to build the operator from the task (task_id should remain the same)
+ if task.task_id == "my_task_needs_more_retries":
+ return CustomAuthTaskProxyOperator(task_id=task_id, retries=3)
+ else:
+ return CustomAuthTaskProxyOperator(task_id=task_id)
+
+ proxying_to_dagster(
+ proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"),
+ global_vars=globals(),
+ build_from_task_fn=CustomAuthTaskProxyOperator.build_from_task,
+ )
+ ```
+ You can do the same for dag-level proxying by providing a custom implementation of the build_from_dag_fn function.
+
+ ```python
+ from dagster_airlift.in_airflow import proxying_to_dagster, BaseProxyDAGToDagsterOperator
+ from airflow.models.dag import DAG
+
+ ... # Dag code here
+
+ class CustomAuthDAGProxyOperator(BaseProxyDAGToDagsterOperator):
+ def get_dagster_session(self, context: Context) -> requests.Session:
+ # Add custom headers to the session
+ return requests.Session(headers={"Authorization": "Bearer my_token"})
+
+ def get_dagster_url(self, context: Context) -> str:
+ # Use a custom environment variable for the dagster url
+ return os.environ["CUSTOM_DAGSTER_URL"]
+
+ @classmethod
+ def build_from_dag(cls, dag: DAG) -> "CustomAuthDAGProxyOperator":
+ # Custom logic to build the operator from the dag (DAG id should remain the same)
+ if dag.dag_id == "my_dag_needs_more_retries":
+ return CustomAuthDAGProxyOperator(task_id="custom override", retries=3, dag=dag)
+ else:
+ return CustomAuthDAGProxyOperator(task_id="basic_override", dag=dag)
+
+ proxying_to_dagster(
+ proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"),
+ global_vars=globals(),
+ build_from_dag_fn=CustomAuthDAGProxyOperator.build_from_dag,
+ )
+ ```
+
+
+
+
+
+
+
class dagster_airlift.in_airflow.BaseDagsterAssetsOperator
+
+
+ Interface for an operator which materializes dagster assets.
+
+ This operator needs to implement the following methods:
+
+ >
+
+ - get_dagster_session: Returns a requests session that can be used to make requests to the Dagster API.
+ - get_dagster_url: Returns the URL for the Dagster instance.
+ - filter_asset_nodes: Filters asset nodes (which are returned from Dagster’s graphql API) to only include those
+
+
+
+
+Optionally, these methods can be overridden as well:
+
+>
+
+ - get_partition_key: Determines the partition key to use to trigger the dagster run. This method will only be
+
+
+
+
+
+
+
+ Loads the proxied state from a directory of yaml files.
+
+ Expects the directory to contain yaml files, where each file corresponds to the id of a dag (ie: dag_id.yaml).
+ This directory is typically constructed using the dagster-airlift CLI:
+
+ >
+
+ ```bash
+ AIRFLOW_HOME=... dagster-airlift proxy scaffold
+ ```
+
+
+The file should have either of the following structure.
+In the case of task-level proxying:
+
+>
+
+ ```yaml
+ tasks:
+ - id: task_id
+ proxied: true
+ - id: task_id
+ proxied: false
+ ```
+
+
+In the case of dag-level proxying:
+
+>
+
+ ```yaml
+ proxied: true
+ ```
+
+
+Parameters: proxied_yaml_path (Path) – The path to the directory containing the yaml files.Returns: The proxied state of the dags and tasks in Airflow.Return type: [AirflowProxiedState](#dagster_airlift.in_airflow.AirflowProxiedState)
+
+
+
+
+
+
+
+#### Proxying State
+
+
+
class dagster_airlift.in_airflow.AirflowProxiedState
+
+
+ A class to store the proxied state of dags and tasks in Airflow.
+ Typically, this is constructed by [`load_proxied_state_from_yaml()`](#dagster_airlift.in_airflow.load_proxied_state_from_yaml).
+
+ Parameters: dags (Dict[str, [*DagProxiedState*](#dagster_airlift.in_airflow.DagProxiedState)]) – A dictionary of dag_id to DagProxiedState.
+
+
+
+
+
+
class dagster_airlift.in_airflow.DagProxiedState
+
+
+ A class to store the proxied state of tasks in a dag.
+
+ Parameters:
+ - tasks (Dict[str, [*TaskProxiedState*](#dagster_airlift.in_airflow.TaskProxiedState)]) – A dictionary of task_id to TaskProxiedState. If the entire dag is proxied, or proxied state
+ - proxied (Optional[bool]) – A boolean indicating whether the entire dag is proxied. If this is None, then the dag proxies at the task level (or
+ - all). (proxying state has not been set at)
+
+
+
+
+
+
+
+
class dagster_airlift.in_airflow.TaskProxiedState
+
+
+ A class to store the proxied state of a task.
+
+ Parameters:
+ - task_id (str) – The id of the task.
+ - proxied (bool) – A boolean indicating whether the task is proxied.
+
+
+
+
+
+
+
+
+
+
+#### Task-level Proxying
+
+
+
+
class dagster_airlift.in_airflow.BaseProxyTaskToDagsterOperator
+
+
+ An operator that proxies task execution to Dagster assets with metadata that map to this task’s dag ID and task ID.
+
+ For the DAG ID and task ID that this operator proxies, it expects there to be corresponding assets
+ in the linked Dagster deployment that have metadata entries with the key dagster-airlift/task-mapping that
+ map to this DAG ID and task ID. This metadata is typically set using the
+ [`dagster_airlift.core.assets_with_task_mappings()`](#dagster_airlift.core.assets_with_task_mappings) function.
+
+ The following methods must be implemented by subclasses:
+
+ >
+
+ - `get_dagster_session()` (inherited from [`BaseDagsterAssetsOperator`](#dagster_airlift.in_airflow.BaseDagsterAssetsOperator))
+ - `get_dagster_url()` (inherited from [`BaseDagsterAssetsOperator`](#dagster_airlift.in_airflow.BaseDagsterAssetsOperator))
+ - `build_from_task()` A class method which takes the task to be proxied, and constructs
+
+
+
+
+There is a default implementation of this operator, [`DefaultProxyTaskToDagsterOperator`](#dagster_airlift.in_airflow.DefaultProxyTaskToDagsterOperator),
+which is used by [`proxying_to_dagster()`](#dagster_airlift.in_airflow.proxying_to_dagster) if no override operator is provided.
+
+
+
+
+
+
+
class dagster_airlift.in_airflow.DefaultProxyTaskToDagsterOperator
+
+
+ The default task proxying operator - which opens a blank session and expects the dagster URL to be set in the environment.
+ The dagster url is expected to be set in the environment as DAGSTER_URL.
+
+ This operator should not be instantiated directly - it is instantiated by [`proxying_to_dagster()`](#dagster_airlift.in_airflow.proxying_to_dagster) if no
+ override operator is provided.
+
+
+
+
+
+
+
+
+
+#### Dag-level Proxying
+
+
+
+
class dagster_airlift.in_airflow.BaseProxyDAGToDagsterOperator
+
+
+ An operator base class that proxies the entire DAG’s execution to Dagster assets with
+ metadata that map to the DAG id used by this task.
+
+ For the Dag ID that this operator proxies, it expects there to be corresponding assets
+ in the linked Dagster deployment that have metadata entries with the key dagster-airlift/dag-mapping that
+ map to this Dag ID. This metadata is typically set using the
+ [`dagster_airlift.core.assets_with_dag_mappings()`](#dagster_airlift.core.assets_with_dag_mappings) function.
+
+ The following methods must be implemented by subclasses:
+
+ >
+
+ - `get_dagster_session()` (inherited from [`BaseDagsterAssetsOperator`](#dagster_airlift.in_airflow.BaseDagsterAssetsOperator))
+ - `get_dagster_url()` (inherited from [`BaseDagsterAssetsOperator`](#dagster_airlift.in_airflow.BaseDagsterAssetsOperator))
+ - `build_from_dag()` A class method which takes the DAG to be proxied, and constructs
+
+
+
+
+There is a default implementation of this operator, [`DefaultProxyDAGToDagsterOperator`](#dagster_airlift.in_airflow.DefaultProxyDAGToDagsterOperator),
+which is used by [`proxying_to_dagster()`](#dagster_airlift.in_airflow.proxying_to_dagster) if no override operator is provided.
+
+
+
+
+
+
+
class dagster_airlift.in_airflow.DefaultProxyDAGToDagsterOperator
+
+
+ The default task proxying operator - which opens a blank session and expects the dagster URL to be set in the environment.
+ The dagster url is expected to be set in the environment as DAGSTER_URL.
+
+ This operator should not be instantiated directly - it is instantiated by [`proxying_to_dagster()`](#dagster_airlift.in_airflow.proxying_to_dagster) if no
+ override operator is provided.
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-aws.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-aws.mdx
new file mode 100644
index 0000000000000..16d39be4ff5a5
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-aws.mdx
@@ -0,0 +1,1316 @@
+---
+title: 'aws (dagster-aws)'
+title_meta: 'aws (dagster-aws) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'aws (dagster-aws) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# AWS (dagster-aws)
+
+Utilities for interfacing with AWS with Dagster.
+
+
+
+
+## S3
+
+
+
dagster_aws.s3.S3Resource ResourceDefinition
+
+
+
+
+ Resource that gives access to S3.
+
+ The underlying S3 session is created by calling
+ `boto3.session.Session(profile_name)`.
+ The returned resource object is an S3 client, an instance of botocore.client.S3.
+
+ Example:
+
+ ```python
+ from dagster import job, op, Definitions
+ from dagster_aws.s3 import S3Resource
+
+ @op
+ def example_s3_op(s3: S3Resource):
+ return s3.get_client().list_objects_v2(
+ Bucket='my-bucket',
+ Prefix='some-key'
+ )
+
+ @job
+ def example_job():
+ example_s3_op()
+
+ defs = Definitions(
+ jobs=[example_job],
+ resources={'s3': S3Resource(region_name='us-west-1')}
+ )
+ ```
+
+
+
+ Persistent IO manager using S3 for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for S3 and the backing bucket.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ ```python
+ from dagster import asset, Definitions
+ from dagster_aws.s3 import S3PickleIOManager, S3Resource
+
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": S3PickleIOManager(
+ s3_resource=S3Resource(),
+ s3_bucket="my-cool-bucket",
+ s3_prefix="my-cool-prefix",
+ )
+ }
+ )
+ ```
+
+
+
+
+
+
class dagster_aws.s3.S3ComputeLogManager
+
+
+ Logs compute function stdout and stderr to S3.
+
+ Users should not instantiate this class directly. Instead, use a YAML block in `dagster.yaml`
+ such as the following:
+
+ ```YAML
+ compute_logs:
+ module: dagster_aws.s3.compute_log_manager
+ class: S3ComputeLogManager
+ config:
+ bucket: "mycorp-dagster-compute-logs"
+ local_dir: "/tmp/cool"
+ prefix: "dagster-test-"
+ use_ssl: true
+ verify: true
+ verify_cert_path: "/path/to/cert/bundle.pem"
+ endpoint_url: "http://alternate-s3-host.io"
+ skip_empty_files: true
+ upload_interval: 30
+ upload_extra_args:
+ ServerSideEncryption: "AES256"
+ show_url_only: false
+ region: "us-west-1"
+ ```
+ Parameters:
+ - bucket (str) – The name of the s3 bucket to which to log.
+ - local_dir (Optional[str]) – Path to the local directory in which to stage logs. Default:
+ - prefix (Optional[str]) – Prefix for the log file keys.
+ - use_ssl (Optional[bool]) – Whether or not to use SSL. Default True.
+ - verify (Optional[bool]) – Whether or not to verify SSL certificates. Default True.
+ - verify_cert_path (Optional[str]) – A filename of the CA cert bundle to use. Only used if
+ - endpoint_url (Optional[str]) – Override for the S3 endpoint url.
+ - skip_empty_files – (Optional[bool]): Skip upload of empty log files.
+ - upload_interval – (Optional[int]): Interval in seconds to upload partial log files to S3. By default, will only upload when the capture is complete.
+ - upload_extra_args – (Optional[dict]): Extra args for S3 file upload
+ - show_url_only – (Optional[bool]): Only show the URL of the log file in the UI, instead of fetching and displaying the full content. Default False.
+ - region – (Optional[str]): The region of the S3 bucket. If not specified, will use the default region of the AWS session.
+ - inst_data (Optional[[*ConfigurableClassData*](../internals.mdx#dagster._serdes.ConfigurableClassData)]) – Serializable representation of the compute
+
+
+
+
+
+
+
+
dagster_aws.s3.S3Coordinate DagsterType
+
+
+ A [`dagster.DagsterType`](../types.mdx#dagster.DagsterType) intended to make it easier to pass information about files on S3
+ from op to op. Objects of this type should be dicts with `'bucket'` and `'key'` keys,
+ and may be hydrated from config in the intuitive way, e.g., for an input with the name
+ `s3_file`:
+
+ ```YAML
+ inputs:
+ s3_file:
+ value:
+ bucket: my-bucket
+ key: my-key
+ ```
+
+
+
+ Base class for Dagster resources that utilize structured config.
+
+ This class is a subclass of both `ResourceDefinition` and `Config`.
+
+ Example definition:
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ prefix: str
+
+ def output(self, text: str) -> None:
+ print(f"{self.prefix}{text}")
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def asset_that_uses_writer(writer: WriterResource):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[asset_that_uses_writer],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+ You can optionally use this class to model configuration only and vend an object
+ of a different type for use at runtime. This is useful for those who wish to
+ have a separate object that manages configuration and a separate object at runtime. Or
+ where you want to directly use a third-party class that you do not control.
+
+ To do this you override the create_resource methods to return a different object.
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ str: prefix
+
+ def create_resource(self, context: InitResourceContext) -> Writer:
+ # Writer is pre-existing class defined else
+ return Writer(self.prefix)
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def use_preexisting_writer_as_resource(writer: ResourceParam[Writer]):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[use_preexisting_writer_as_resource],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+## ECS
+
+
+
dagster_aws.ecs.EcsRunLauncher RunLauncher
+
+
+
+
+ RunLauncher that starts a task in ECS for each Dagster job run.
+
+
+
+
+
+
+
dagster_aws.ecs.ecs_executor ExecutorDefinition
+
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Executor which launches steps as ECS tasks.
+
+ To use the ecs_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster_aws.ecs import ecs_executor
+
+ from dagster import job, op
+
+
+ @op(
+ tags={"ecs/cpu": "256", "ecs/memory": "512"},
+ )
+ def ecs_op():
+ pass
+
+
+ @job(executor_def=ecs_executor)
+ def ecs_job():
+ ecs_op()
+
+
+ ```
+ Then you can configure the executor with run config as follows:
+
+ ```YAML
+ execution:
+ config:
+ cpu: 1024
+ memory: 2048
+ ephemeral_storage: 10
+ task_overrides:
+ containerOverrides:
+ - name: run
+ environment:
+ - name: MY_ENV_VAR
+ value: "my_value"
+ ```
+ max_concurrent limits the number of ECS tasks that will execute concurrently for one run. By default
+ there is no limit- it will maximally parallel as allowed by the DAG. Note that this is not a
+ global limit.
+
+ Configuration set on the ECS tasks created by the ECSRunLauncher will also be
+ set on the tasks created by the ecs_executor.
+
+ Configuration set using tags on a @job will only apply to the run level. For configuration
+ to apply at each step it must be set using tags for each @op.
+
+
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ While there is no plan to remove this functionality, for new projects, we recommend using Dagster Pipes. For more information, see https://docs.dagster.io/guides/build/external-pipelines/.
+
+ :::
+
+ - spark_config:
+ - cluster_id: Name of the job flow (cluster) on which to execute.
+ - region_name: The AWS region that the cluster is in.
+ - action_on_failure: The EMR action to take when the cluster step fails: [https://docs.aws.amazon.com/emr/latest/APIReference/API_StepConfig.html](https://docs.aws.amazon.com/emr/latest/APIReference/API_StepConfig.html)
+ - staging_bucket: S3 bucket to use for passing files between the plan process and EMR process.
+ - staging_prefix: S3 key prefix inside the staging_bucket to use for files passed the plan process and EMR process
+ - wait_for_logs: If set, the system will wait for EMR logs to appear on S3. Note that logs are copied every 5 minutes, so enabling this will add several minutes to the job runtime.
+ - local_job_package_path: Absolute path to the package that contains the job definition(s) whose steps will execute remotely on EMR. This is a path on the local fileystem of the process executing the job. The expectation is that this package will also be available on the python path of the launched process running the Spark step on EMR, either deployed on step launch via the deploy_local_job_package option, referenced on s3 via the s3_job_package_path option, or installed on the cluster via bootstrap actions.
+ - local_pipeline_package_path: (legacy) Absolute path to the package that contains the pipeline definition(s) whose steps will execute remotely on EMR. This is a path on the local fileystem of the process executing the pipeline. The expectation is that this package will also be available on the python path of the launched process running the Spark step on EMR, either deployed on step launch via the deploy_local_pipeline_package option, referenced on s3 via the s3_pipeline_package_path option, or installed on the cluster via bootstrap actions.
+ - deploy_local_job_package: If set, before every step run, the launcher will zip up all the code in local_job_package_path, upload it to s3, and pass it to spark-submit’s –py-files option. This gives the remote process access to up-to-date user code. If not set, the assumption is that some other mechanism is used for distributing code to the EMR cluster. If this option is set to True, s3_job_package_path should not also be set.
+ - deploy_local_pipeline_package: (legacy) If set, before every step run, the launcher will zip up all the code in local_job_package_path, upload it to s3, and pass it to spark-submit’s –py-files option. This gives the remote process access to up-to-date user code. If not set, the assumption is that some other mechanism is used for distributing code to the EMR cluster. If this option is set to True, s3_job_package_path should not also be set.
+ - s3_job_package_path: If set, this path will be passed to the –py-files option of spark-submit. This should usually be a path to a zip file. If this option is set, deploy_local_job_package should not be set to True.
+ - s3_pipeline_package_path: If set, this path will be passed to the –py-files option of spark-submit. This should usually be a path to a zip file. If this option is set, deploy_local_pipeline_package should not be set to True.
+
+
+
+
+
+ Core class for defining loggers.
+
+ Loggers are job-scoped logging handlers, which will be automatically invoked whenever
+ dagster messages are logged from within a job.
+
+ Parameters:
+ - logger_fn (Callable[[[*InitLoggerContext*](../loggers.mdx#dagster.InitLoggerContext)], logging.Logger]) – User-provided function to
+ - config_schema (Optional[[*ConfigSchema*](../config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of this logger.
+
+
+
+
+
+
+
+
+
+
+
+
+## SecretsManager
+
+Resources which surface SecretsManager secrets for use in Dagster resources and jobs.
+
+
+
+ Resource that provides a dict which maps selected SecretsManager secrets to
+ their string values. Also optionally sets chosen secrets as environment variables.
+
+ Example:
+
+ ```python
+ import os
+ from dagster import build_op_context, job, op, ResourceParam
+ from dagster_aws.secretsmanager import SecretsManagerSecretsResource
+
+ @op
+ def example_secretsmanager_secrets_op(secrets: SecretsManagerSecretsResource):
+ return secrets.fetch_secrets().get("my-secret-name")
+
+ @op
+ def example_secretsmanager_secrets_op_2(secrets: SecretsManagerSecretsResource):
+ with secrets.secrets_in_environment():
+ return os.getenv("my-other-secret-name")
+
+ @job
+ def example_job():
+ example_secretsmanager_secrets_op()
+ example_secretsmanager_secrets_op_2()
+
+ defs = Definitions(
+ jobs=[example_job],
+ resources={
+ 'secrets': SecretsManagerSecretsResource(
+ region_name='us-west-1',
+ secrets_tag="dagster",
+ add_to_environment=True,
+ )
+ }
+ )
+ ```
+ Note that your ops must also declare that they require this resource with or it will not be initialized
+ for the execution of their compute functions.
+
+
+
+
+
+
+
+
+
+
+
+## Pipes
+
+
+
+
+### Context Injectors
+
+
+
class dagster_aws.pipes.PipesS3ContextInjector
+
+
+ A context injector that injects context by writing to a temporary S3 location.
+
+ Parameters:
+ - bucket (str) – The S3 bucket to write to.
+ - client (boto3.client) – A boto3 client to use to write to S3.
+ - key_prefix (Optional[str]) – An optional prefix to use for the S3 key. Defaults to a random
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesLambdaEventContextInjector
+
+ Injects context via AWS Lambda event input.
+ Should be paired with :py:class`~dagster_pipes.PipesMappingParamsLoader` on the Lambda side.
+
+
+
+
+
+
+
+
+
+### Message Readers
+
+
+
class dagster_aws.pipes.PipesS3MessageReader
+
+
+ Message reader that reads messages by periodically reading message chunks from a specified S3
+ bucket.
+
+ If log_readers is passed, this reader will also start the passed readers
+ when the first message is received from the external process.
+
+ Parameters:
+ - interval (float) – interval in seconds between attempts to download a chunk
+ - bucket (str) – The S3 bucket to read from.
+ - client (WorkspaceClient) – A boto3 client.
+ - log_readers (Optional[Sequence[PipesLogReader]]) – A set of log readers for logs on S3.
+ - include_stdio_in_messages (bool) – Whether to send stdout/stderr to Dagster via Pipes messages. Defaults to False.
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesCloudWatchMessageReader
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Message reader that consumes AWS CloudWatch logs to read pipes messages.
+
+
+
+
+
+
+
+
+
+
+
+### Clients
+
+
+
class dagster_aws.pipes.PipesLambdaClient
+
+
+ A pipes client for invoking AWS lambda.
+
+ By default context is injected via the lambda input event and messages are parsed out of the
+ 4k tail of logs.
+
+ Parameters:
+ - client (boto3.client) – The boto lambda client used to call invoke.
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+
+
+
+
run
+
+
+ Synchronously invoke a lambda function, enriched with the pipes protocol.
+
+ Parameters:
+ - function_name (str) – The name of the function to use.
+ - event (Mapping[str, Any]) – A JSON serializable object to pass as input to the lambda.
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesGlueClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client for invoking AWS Glue jobs.
+
+ Parameters:
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - client (Optional[boto3.client]) – The boto Glue client used to launch the Glue job
+ - forward_termination (bool) – Whether to cancel the Glue job run when the Dagster process receives a termination signal.
+
+
+
+
run
+
+
+ Start a Glue job, enriched with the pipes protocol.
+
+ See also: [AWS API Documentation](https://docs.aws.amazon.com/goto/WebAPI/glue-2017-03-31/StartJobRun)
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+ - start_job_run_params (Dict) – Parameters for the `start_job_run` boto3 Glue client call.
+ - extras (Optional[Dict[str, Any]]) – Additional Dagster metadata to pass to the Glue job.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesECSClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client for running AWS ECS tasks.
+
+ Parameters:
+ - client (Any) – The boto ECS client used to launch the ECS task
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - forward_termination (bool) – Whether to cancel the ECS task when the Dagster process receives a termination signal.
+
+
+
+
run
+
+
+ Run ECS tasks, enriched with the pipes protocol.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+ - run_task_params (dict) – Parameters for the `run_task` boto3 ECS client call.
+ - extras (Optional[Dict[str, Any]]) – Additional information to pass to the Pipes session in the external process.
+ - pipes_container_name (Optional[str]) – If running more than one container in the task,
+ - waiter_config (Optional[WaiterConfig]) – Optional waiter configuration to use. Defaults to 70 days (Delay: 6, MaxAttempts: 1000000).
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesEMRClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client for running jobs on AWS EMR.
+
+ Parameters:
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - client (Optional[boto3.client]) – The boto3 EMR client used to interact with AWS EMR.
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - forward_termination (bool) – Whether to cancel the EMR job if the Dagster process receives a termination signal.
+ - wait_for_s3_logs_seconds (int) – The number of seconds to wait for S3 logs to be written after execution completes.
+ - s3_application_logs_prefix (str) – The prefix to use when looking for application logs in S3.
+
+
+
+
run
+
+
+ Run a job on AWS EMR, enriched with the pipes protocol.
+
+ Starts a new EMR cluster for each invocation.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+ - run_job_flow_params (Optional[dict]) – Parameters for the `run_job_flow` boto3 EMR client call.
+ - extras (Optional[Dict[str, Any]]) – Additional information to pass to the Pipes session in the external process.
+
+
+ Returns: Wrapper containing results reported by the external process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesEMRContainersClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client for running workloads on AWS EMR Containers.
+
+ Parameters:
+ - client (Optional[boto3.client]) – The boto3 AWS EMR containers client used to interact with AWS EMR Containers.
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - forward_termination (bool) – Whether to cancel the AWS EMR Containers workload if the Dagster process receives a termination signal.
+ - pipes_params_bootstrap_method (Literal["args", "env"]) – The method to use to inject parameters into the AWS EMR Containers workload. Defaults to “args”.
+ - waiter_config (Optional[WaiterConfig]) – Optional waiter configuration to use. Defaults to 70 days (Delay: 6, MaxAttempts: 1000000).
+
+
+
+
run
+
+
+ Run a workload on AWS EMR Containers, enriched with the pipes protocol.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+ - params (dict) – Parameters for the `start_job_run` boto3 AWS EMR Containers client call.
+ - extras (Optional[Dict[str, Any]]) – Additional information to pass to the Pipes session in the external process.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesEMRServerlessClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client for running workloads on AWS EMR Serverless.
+
+ Parameters:
+ - client (Optional[boto3.client]) – The boto3 AWS EMR Serverless client used to interact with AWS EMR Serverless.
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - forward_termination (bool) – Whether to cancel the AWS EMR Serverless workload if the Dagster process receives a termination signal.
+ - poll_interval (float) – The interval in seconds to poll the AWS EMR Serverless workload for status updates. Defaults to 5 seconds.
+
+
+
+
run
+
+
+ Run a workload on AWS EMR Serverless, enriched with the pipes protocol.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+ - params (dict) – Parameters for the `start_job_run` boto3 AWS EMR Serverless client call.
+ - extras (Optional[Dict[str, Any]]) – Additional information to pass to the Pipes session in the external process.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Please use S3PickleIOManager instead..
+
+ :::
+
+ Renamed to S3PickleIOManager. See S3PickleIOManager for documentation.
+
+
+
+
+
+
+
dagster_aws.s3.s3_resource ResourceDefinition
+
+
+
+
+ Resource that gives access to S3.
+
+ The underlying S3 session is created by calling
+ `boto3.session.Session(profile_name)`.
+ The returned resource object is an S3 client, an instance of botocore.client.S3.
+
+ Example:
+
+ ```python
+ from dagster import build_op_context, job, op
+ from dagster_aws.s3 import s3_resource
+
+ @op(required_resource_keys={'s3'})
+ def example_s3_op(context):
+ return context.resources.s3.list_objects_v2(
+ Bucket='my-bucket',
+ Prefix='some-key'
+ )
+
+ @job(resource_defs={'s3': s3_resource})
+ def example_job():
+ example_s3_op()
+
+ example_job.execute_in_process(
+ run_config={
+ 'resources': {
+ 's3': {
+ 'config': {
+ 'region_name': 'us-west-1',
+ }
+ }
+ }
+ }
+ )
+ ```
+ Note that your ops must also declare that they require this resource with
+ required_resource_keys, or it will not be initialized for the execution of their compute
+ functions.
+
+ You may configure this resource as follows:
+
+ ```YAML
+ resources:
+ s3:
+ config:
+ region_name: "us-west-1"
+ # Optional[str]: Specifies a custom region for the S3 session. Default is chosen
+ # through the ordinary boto credential chain.
+ use_unsigned_session: false
+ # Optional[bool]: Specifies whether to use an unsigned S3 session. Default: True
+ endpoint_url: "http://localhost"
+ # Optional[str]: Specifies a custom endpoint for the S3 session. Default is None.
+ profile_name: "dev"
+ # Optional[str]: Specifies a custom profile for S3 session. Default is default
+ # profile as specified in ~/.aws/credentials file
+ use_ssl: true
+ # Optional[bool]: Whether or not to use SSL. By default, SSL is used.
+ verify: None
+ # Optional[str]: Whether or not to verify SSL certificates. By default SSL certificates are verified.
+ # You can also specify this argument if you want to use a different CA cert bundle than the one used by botocore."
+ aws_access_key_id: None
+ # Optional[str]: The access key to use when creating the client.
+ aws_secret_access_key: None
+ # Optional[str]: The secret key to use when creating the client.
+ aws_session_token: None
+ # Optional[str]: The session token to use when creating the client.
+ ```
+
+
+
+ Persistent IO manager using S3 for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for S3 and the backing bucket.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ 1. Attach this IO manager to a set of assets.
+ ```python
+ from dagster import Definitions, asset
+ from dagster_aws.s3 import s3_pickle_io_manager, s3_resource
+
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": s3_pickle_io_manager.configured(
+ {"s3_bucket": "my-cool-bucket", "s3_prefix": "my-cool-prefix"}
+ ),
+ "s3": s3_resource,
+ },
+ )
+ ```
+ 2. Attach this IO manager to your job to make it available to your ops.
+ ```python
+ from dagster import job
+ from dagster_aws.s3 import s3_pickle_io_manager, s3_resource
+
+ @job(
+ resource_defs={
+ "io_manager": s3_pickle_io_manager.configured(
+ {"s3_bucket": "my-cool-bucket", "s3_prefix": "my-cool-prefix"}
+ ),
+ "s3": s3_resource,
+ },
+ )
+ def my_job():
+ ...
+ ```
+
+
+
+
+
+
dagster_aws.s3.s3_file_manager ResourceDefinition
+
+
+
+
+ FileManager that provides abstract access to S3.
+
+ Implements the [`FileManager`](../internals.mdx#dagster._core.storage.file_manager.FileManager) API.
+
+
+
+
+ Resource that provides a dict which maps selected SecretsManager secrets to
+ their string values. Also optionally sets chosen secrets as environment variables.
+
+ Example:
+
+ ```python
+ import os
+ from dagster import build_op_context, job, op
+ from dagster_aws.secretsmanager import secretsmanager_secrets_resource
+
+ @op(required_resource_keys={'secrets'})
+ def example_secretsmanager_secrets_op(context):
+ return context.resources.secrets.get("my-secret-name")
+
+ @op(required_resource_keys={'secrets'})
+ def example_secretsmanager_secrets_op_2(context):
+ return os.getenv("my-other-secret-name")
+
+ @job(resource_defs={'secrets': secretsmanager_secrets_resource})
+ def example_job():
+ example_secretsmanager_secrets_op()
+ example_secretsmanager_secrets_op_2()
+
+ example_job.execute_in_process(
+ run_config={
+ 'resources': {
+ 'secrets': {
+ 'config': {
+ 'region_name': 'us-west-1',
+ 'secrets_tag': 'dagster',
+ 'add_to_environment': True,
+ }
+ }
+ }
+ }
+ )
+ ```
+ Note that your ops must also declare that they require this resource with
+ required_resource_keys, or it will not be initialized for the execution of their compute
+ functions.
+
+ You may configure this resource as follows:
+
+ ```YAML
+ resources:
+ secretsmanager:
+ config:
+ region_name: "us-west-1"
+ # Optional[str]: Specifies a custom region for the SecretsManager session. Default is chosen
+ # through the ordinary boto credential chain.
+ profile_name: "dev"
+ # Optional[str]: Specifies a custom profile for SecretsManager session. Default is default
+ # profile as specified in ~/.aws/credentials file
+ secrets: ["arn:aws:secretsmanager:region:aws_account_id:secret:appauthexample-AbCdEf"]
+ # Optional[List[str]]: Specifies a list of secret ARNs to pull from SecretsManager.
+ secrets_tag: "dagster"
+ # Optional[str]: Specifies a tag, all secrets which have the tag set will be pulled
+ # from SecretsManager.
+ add_to_environment: true
+ # Optional[bool]: Whether to set the selected secrets as environment variables. Defaults
+ # to false.
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-azure.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-azure.mdx
new file mode 100644
index 0000000000000..5bce54f557cf4
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-azure.mdx
@@ -0,0 +1,448 @@
+---
+title: 'azure (dagster-azure)'
+title_meta: 'azure (dagster-azure) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'azure (dagster-azure) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Azure (dagster-azure)
+
+Utilities for using Azure Storage Accounts with Dagster. This is mostly aimed at Azure Data Lake
+Storage Gen 2 (ADLS2) but also contains some utilities for Azure Blob Storage.
+
+
+
+ Resource containing clients to access Azure Data Lake Storage Gen2.
+
+ Contains a client for both the Data Lake and Blob APIs, to work around the limitations
+ of each.
+
+ Example usage:
+
+ Attach this resource to your Definitions to be used by assets and jobs.
+
+ ```python
+ from dagster import Definitions, asset, job, op
+ from dagster_azure.adls2 import ADLS2Resource, ADLS2SASToken
+
+ @asset
+ def asset1(adls2: ADLS2Resource):
+ adls2.adls2_client.list_file_systems()
+ ...
+
+ @op
+ def my_op(adls2: ADLS2Resource):
+ adls2.adls2_client.list_file_systems()
+ ...
+
+ @job
+ def my_job():
+ my_op()
+
+ defs = Definitions(
+ assets=[asset1],
+ jobs=[my_job],
+ resources={
+ "adls2": ADLS2Resource(
+ storage_account="my-storage-account",
+ credential=ADLS2SASToken(token="my-sas-token"),
+ )
+ },
+ )
+ ```
+ Attach this resource to your job to make it available to your ops.
+
+ ```python
+ from dagster import job, op
+ from dagster_azure.adls2 import ADLS2Resource, ADLS2SASToken
+
+ @op
+ def my_op(adls2: ADLS2Resource):
+ adls2.adls2_client.list_file_systems()
+ ...
+
+ @job(
+ resource_defs={
+ "adls2": ADLS2Resource(
+ storage_account="my-storage-account",
+ credential=ADLS2SASToken(token="my-sas-token"),
+ )
+ },
+ )
+ def my_job():
+ my_op()
+ ```
+
+
+
+ Stateful mock of an ADLS2Resource for testing.
+
+ Wraps a `mock.MagicMock`. Containers are implemented using an in-memory dict.
+
+
+
+
+
+
+
class dagster_azure.blob.AzureBlobComputeLogManager
+
+
+ Logs op compute function stdout and stderr to Azure Blob Storage.
+
+ This is also compatible with Azure Data Lake Storage.
+
+ Users should not instantiate this class directly. Instead, use a YAML block in `dagster.yaml`. Examples provided below
+ will show how to configure with various credentialing schemes.
+
+ Parameters:
+ - storage_account (str) – The storage account name to which to log.
+ - container (str) – The container (or ADLS2 filesystem) to which to log.
+ - secret_credential (Optional[dict]) – Secret credential for the storage account. This should be
+ - access_key_or_sas_token (Optional[str]) – Access key or SAS token for the storage account.
+ - default_azure_credential (Optional[dict]) – Use and configure DefaultAzureCredential.
+ - local_dir (Optional[str]) – Path to the local directory in which to stage logs. Default:
+ - prefix (Optional[str]) – Prefix for the log file keys.
+ - upload_interval (Optional[int]) – Interval in seconds to upload partial log files blob storage. By default, will only upload when the capture is complete.
+ - show_url_only (bool) – Only show the URL of the log file in the UI, instead of fetching and displaying the full content. Default False.
+ - inst_data (Optional[[*ConfigurableClassData*](../internals.mdx#dagster._serdes.ConfigurableClassData)]) – Serializable representation of the compute
+
+
+ Examples:
+ Using an Azure Blob Storage account with an [AzureSecretCredential](https://learn.microsoft.com/en-us/python/api/azure-identity/azure.identity.clientsecretcredential?view=azure-python):
+
+ ```YAML
+ compute_logs:
+ module: dagster_azure.blob.compute_log_manager
+ class: AzureBlobComputeLogManager
+ config:
+ storage_account: my-storage-account
+ container: my-container
+ secret_credential:
+ client_id: my-client-id
+ client_secret: my-client-secret
+ tenant_id: my-tenant-id
+ prefix: "dagster-test-"
+ local_dir: "/tmp/cool"
+ upload_interval: 30
+ show_url_only: false
+ ```
+ Using an Azure Blob Storage account with a [DefaultAzureCredential](https://learn.microsoft.com/en-us/python/api/azure-identity/azure.identity.defaultazurecredential?view=azure-python):
+
+ ```YAML
+ compute_logs:
+ module: dagster_azure.blob.compute_log_manager
+ class: AzureBlobComputeLogManager
+ config:
+ storage_account: my-storage-account
+ container: my-container
+ default_azure_credential:
+ exclude_environment_credential: false
+ prefix: "dagster-test-"
+ local_dir: "/tmp/cool"
+ upload_interval: 30
+ show_url_only: false
+ ```
+ Using an Azure Blob Storage account with an access key:
+
+ ```YAML
+ compute_logs:
+ module: dagster_azure.blob.compute_log_manager
+ class: AzureBlobComputeLogManager
+ config:
+ storage_account: my-storage-account
+ container: my-container
+ access_key_or_sas_token: my-access-key
+ prefix: "dagster-test-"
+ local_dir: "/tmp/cool"
+ upload_interval: 30
+ show_url_only: false
+ ```
+
+
+
+ Persistent IO manager using Azure Data Lake Storage Gen2 for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for ADLS and the backing
+ container.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ 1. Attach this IO manager to a set of assets.
+ ```python
+ from dagster import Definitions, asset
+ from dagster_azure.adls2 import ADLS2PickleIOManager, ADLS2Resource, ADLS2SASToken
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return df[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": ADLS2PickleIOManager(
+ adls2_file_system="my-cool-fs",
+ adls2_prefix="my-cool-prefix",
+ adls2=ADLS2Resource(
+ storage_account="my-storage-account",
+ credential=ADLS2SASToken(token="my-sas-token"),
+ ),
+ ),
+ },
+ )
+ ```
+ 2. Attach this IO manager to your job to make it available to your ops.
+ ```python
+ from dagster import job
+ from dagster_azure.adls2 import ADLS2PickleIOManager, ADLS2Resource, ADLS2SASToken
+
+ @job(
+ resource_defs={
+ "io_manager": ADLS2PickleIOManager(
+ adls2_file_system="my-cool-fs",
+ adls2_prefix="my-cool-prefix",
+ adls2=ADLS2Resource(
+ storage_account="my-storage-account",
+ credential=ADLS2SASToken(token="my-sas-token"),
+ ),
+ ),
+ },
+ )
+ def my_job():
+ ...
+ ```
+
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Please use ADLS2PickleIOManager instead..
+
+ :::
+
+ Renamed to ADLS2PickleIOManager. See ADLS2PickleIOManager for documentation.
+
+
+
+
+ Resource that gives ops access to Azure Data Lake Storage Gen2.
+
+ The underlying client is a `DataLakeServiceClient`.
+
+ Attach this resource definition to a [`JobDefinition`](../jobs.mdx#dagster.JobDefinition) in order to make it
+ available to your ops.
+
+ Example:
+
+ ```python
+ from dagster import job, op
+ from dagster_azure.adls2 import adls2_resource
+
+ @op(required_resource_keys={'adls2'})
+ def example_adls2_op(context):
+ return list(context.resources.adls2.adls2_client.list_file_systems())
+
+ @job(resource_defs={"adls2": adls2_resource})
+ def my_job():
+ example_adls2_op()
+ ```
+ Note that your ops must also declare that they require this resource with
+ required_resource_keys, or it will not be initialized for the execution of their compute
+ functions.
+
+ You may pass credentials to this resource using either a SAS token, a key or by passing the
+ DefaultAzureCredential object.
+
+ ```YAML
+ resources:
+ adls2:
+ config:
+ storage_account: my_storage_account
+ # str: The storage account name.
+ credential:
+ sas: my_sas_token
+ # str: the SAS token for the account.
+ key:
+ env: AZURE_DATA_LAKE_STORAGE_KEY
+ # str: The shared access key for the account.
+ DefaultAzureCredential: {}
+ # dict: The keyword arguments used for DefaultAzureCredential
+ # or leave the object empty for no arguments
+ DefaultAzureCredential:
+ exclude_environment_credential: true
+ ```
+
+
+
+ Persistent IO manager using Azure Data Lake Storage Gen2 for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for ADLS and the backing
+ container.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ Attach this IO manager to a set of assets.
+
+ ```python
+ from dagster import Definitions, asset
+ from dagster_azure.adls2 import adls2_pickle_io_manager, adls2_resource
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return df[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": adls2_pickle_io_manager.configured(
+ {"adls2_file_system": "my-cool-fs", "adls2_prefix": "my-cool-prefix"}
+ ),
+ "adls2": adls2_resource,
+ },
+ )
+ ```
+ Attach this IO manager to your job to make it available to your ops.
+
+ ```python
+ from dagster import job
+ from dagster_azure.adls2 import adls2_pickle_io_manager, adls2_resource
+
+ @job(
+ resource_defs={
+ "io_manager": adls2_pickle_io_manager.configured(
+ {"adls2_file_system": "my-cool-fs", "adls2_prefix": "my-cool-prefix"}
+ ),
+ "adls2": adls2_resource,
+ },
+ )
+ def my_job():
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-celery-docker.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-celery-docker.mdx
new file mode 100644
index 0000000000000..2c29c99f768f5
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-celery-docker.mdx
@@ -0,0 +1,88 @@
+---
+title: 'orchestration on celery + docker'
+title_meta: 'orchestration on celery + docker API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'orchestration on celery + docker Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+ Celery-based executor which launches tasks in docker containers.
+
+ The Celery executor exposes config settings for the underlying Celery app under
+ the `config_source` key. This config corresponds to the “new lowercase settings” introduced
+ in Celery version 4.0 and the object constructed from config will be passed to the
+ `celery.Celery` constructor as its `config_source` argument.
+ (See [https://docs.celeryq.dev/en/stable/userguide/configuration.html](https://docs.celeryq.dev/en/stable/userguide/configuration.html) for details.)
+
+ The executor also exposes the `broker`, backend, and `include` arguments to the
+ `celery.Celery` constructor.
+
+ In the most common case, you may want to modify the `broker` and `backend` (e.g., to use
+ Redis instead of RabbitMQ). We expect that `config_source` will be less frequently
+ modified, but that when op executions are especially fast or slow, or when there are
+ different requirements around idempotence or retry, it may make sense to execute jobs
+ with variations on these settings.
+
+ To use the celery_docker_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster import job
+ from dagster_celery_docker.executor import celery_docker_executor
+
+ @job(executor_def=celery_docker_executor)
+ def celery_enabled_job():
+ pass
+ ```
+ Then you can configure the executor as follows:
+
+ ```YAML
+ execution:
+ config:
+ docker:
+ image: 'my_repo.com/image_name:latest'
+ registry:
+ url: 'my_repo.com'
+ username: 'my_user'
+ password: {env: 'DOCKER_PASSWORD'}
+ env_vars: ["DAGSTER_HOME"] # environment vars to pass from celery worker to docker
+ container_kwargs: # keyword args to be passed to the container. example:
+ volumes: ['/home/user1/:/mnt/vol2','/var/www:/mnt/vol1']
+
+ broker: 'pyamqp://guest@localhost//' # Optional[str]: The URL of the Celery broker
+ backend: 'rpc://' # Optional[str]: The URL of the Celery results backend
+ include: ['my_module'] # Optional[List[str]]: Modules every worker should import
+ config_source: # Dict[str, Any]: Any additional parameters to pass to the
+ #... # Celery workers. This dict will be passed as the `config_source`
+ #... # argument of celery.Celery().
+ ```
+ Note that the YAML you provide here must align with the configuration with which the Celery
+ workers on which you hope to run were started. If, for example, you point the executor at a
+ different broker than the one your workers are listening to, the workers will never be able to
+ pick up tasks for execution.
+
+ In deployments where the celery_docker_job_executor is used all appropriate celery and dagster_celery
+ commands must be invoked with the -A dagster_celery_docker.app argument.
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-celery-k8s.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-celery-k8s.mdx
new file mode 100644
index 0000000000000..ef1f37ca2462e
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-celery-k8s.mdx
@@ -0,0 +1,120 @@
+---
+title: 'orchestration on celery + kubernetes'
+title_meta: 'orchestration on celery + kubernetes API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'orchestration on celery + kubernetes Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+ In contrast to the `K8sRunLauncher`, which launches dagster runs as single K8s
+ Jobs, this run launcher is intended for use in concert with
+ [`dagster_celery_k8s.celery_k8s_job_executor()`](#dagster_celery_k8s.celery_k8s_job_executor).
+
+ With this run launcher, execution is delegated to:
+
+ >
+
+ 1. A run worker Kubernetes Job, which traverses the dagster run execution plan and
+ submits steps to Celery queues for execution;
+ 2. The step executions which are submitted to Celery queues are picked up by Celery workers,
+ and each step execution spawns a step execution Kubernetes Job. See the implementation
+ defined in `dagster_celery_k8.executor.create_k8s_job_task()`.
+
+
+You can configure a Dagster instance to use this RunLauncher by adding a section to your
+`dagster.yaml` like the following:
+
+ ```yaml
+ run_launcher:
+ module: dagster_k8s.launcher
+ class: CeleryK8sRunLauncher
+ config:
+ instance_config_map: "dagster-k8s-instance-config-map"
+ dagster_home: "/some/path"
+ postgres_password_secret: "dagster-k8s-pg-password"
+ broker: "some_celery_broker_url"
+ backend: "some_celery_backend_url"
+ ```
+
+
+
+ Celery-based executor which launches tasks as Kubernetes Jobs.
+
+ The Celery executor exposes config settings for the underlying Celery app under
+ the `config_source` key. This config corresponds to the “new lowercase settings” introduced
+ in Celery version 4.0 and the object constructed from config will be passed to the
+ `celery.Celery` constructor as its `config_source` argument.
+ (See [https://docs.celeryq.dev/en/stable/userguide/configuration.html](https://docs.celeryq.dev/en/stable/userguide/configuration.html) for details.)
+
+ The executor also exposes the `broker`, backend, and `include` arguments to the
+ `celery.Celery` constructor.
+
+ In the most common case, you may want to modify the `broker` and `backend` (e.g., to use
+ Redis instead of RabbitMQ). We expect that `config_source` will be less frequently
+ modified, but that when op executions are especially fast or slow, or when there are
+ different requirements around idempotence or retry, it may make sense to execute dagster jobs
+ with variations on these settings.
+
+ To use the celery_k8s_job_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster import job
+ from dagster_celery_k8s.executor import celery_k8s_job_executor
+
+
+ @job(executor_def=celery_k8s_job_executor)
+ def celery_enabled_job():
+ pass
+ ```
+ Then you can configure the executor as follows:
+
+ ```YAML
+ execution:
+ config:
+ job_image: 'my_repo.com/image_name:latest'
+ job_namespace: 'some-namespace'
+ broker: 'pyamqp://guest@localhost//' # Optional[str]: The URL of the Celery broker
+ backend: 'rpc://' # Optional[str]: The URL of the Celery results backend
+ include: ['my_module'] # Optional[List[str]]: Modules every worker should import
+ config_source: # Dict[str, Any]: Any additional parameters to pass to the
+ #... # Celery workers. This dict will be passed as the `config_source`
+ #... # argument of celery.Celery().
+ ```
+ Note that the YAML you provide here must align with the configuration with which the Celery
+ workers on which you hope to run were started. If, for example, you point the executor at a
+ different broker than the one your workers are listening to, the workers will never be able to
+ pick up tasks for execution.
+
+ In deployments where the celery_k8s_job_executor is used all appropriate celery and dagster_celery
+ commands must be invoked with the -A dagster_celery_k8s.app argument.
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-celery.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-celery.mdx
new file mode 100644
index 0000000000000..2f02160b1a903
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-celery.mdx
@@ -0,0 +1,290 @@
+---
+title: 'celery (dagster-celery)'
+title_meta: 'celery (dagster-celery) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'celery (dagster-celery) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Celery (dagster-celery)
+
+
+
+
+## Quickstart
+
+To get a local rabbitmq broker started and available via the default
+`pyamqp://guest@localhost:5672`, in the `dagster/python_modules/libraries/dagster-celery/`
+directory run:
+
+ ```bash
+ docker-compose up
+ ```
+To run a celery worker:
+
+ ```bash
+ celery -A dagster_celery.app worker -l info
+ ```
+To start multiple workers in the background, run:
+
+ ```bash
+ celery multi start w2 -A dagster_celery.app -l info
+ ```
+To execute a job using the celery-backed executor, you’ll need to set the job’s `executor_def` to
+the celery_executor.
+
+ ```python
+ from dagster import job
+ from dagster_celery import celery_executor
+
+ @job(executor_def=celery_executor)
+ def my_job():
+ pass
+ ```
+
+
+
+### Monitoring your Celery tasks
+
+We advise using [Flower](https://celery.readthedocs.io/en/latest/userguide/monitoring.html#flower-real-time-celery-web-monitor):
+
+ ```bash
+ celery -A dagster_celery.app flower
+ ```
+
+
+
+
+
+
+### Customizing the Celery broker, backend, and other app configuration
+
+By default this will use `amqp://guest:**@localhost:5672//` as the Celery broker URL and
+`rpc://` as the results backend. In production, you will want to change these values. Pending the
+introduction of a dagster_celery CLI, that would entail writing a Python module `my_module` as
+follows:
+
+ ```python
+ from celery import Celery
+
+ from dagster_celery.tasks import create_task
+
+ app = Celery('dagster', broker_url='some://custom@value', ...)
+
+ execute_plan = create_task(app)
+
+ if __name__ == '__main__':
+ app.worker_main()
+ ```
+You can then run the celery worker using:
+
+ ```bash
+ celery -A my_module worker --loglevel=info
+ ```
+This customization mechanism is used to implement dagster_celery_k8s and dagster_celery_k8s which delegate the execution of steps to ephemeral kubernetes pods and docker containers, respectively.
+
+
+
+
+
+
+
+## API
+
+
+
dagster_celery.celery_executor ExecutorDefinition
+
+
+
+
+ Celery-based executor.
+
+ The Celery executor exposes config settings for the underlying Celery app under
+ the `config_source` key. This config corresponds to the “new lowercase settings” introduced
+ in Celery version 4.0 and the object constructed from config will be passed to the
+ `celery.Celery` constructor as its `config_source` argument.
+ (See [https://docs.celeryq.dev/en/stable/userguide/configuration.html](https://docs.celeryq.dev/en/stable/userguide/configuration.html) for details.)
+
+ The executor also exposes the `broker`, backend, and `include` arguments to the
+ `celery.Celery` constructor.
+
+ In the most common case, you may want to modify the `broker` and `backend` (e.g., to use
+ Redis instead of RabbitMQ). We expect that `config_source` will be less frequently
+ modified, but that when solid executions are especially fast or slow, or when there are
+ different requirements around idempotence or retry, it may make sense to execute jobs
+ with variations on these settings.
+
+ To use the celery_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster import job
+ from dagster_celery import celery_executor
+
+ @job(executor_def=celery_executor)
+ def celery_enabled_job():
+ pass
+ ```
+ Then you can configure the executor as follows:
+
+ ```YAML
+ execution:
+ config:
+ broker: 'pyamqp://guest@localhost//' # Optional[str]: The URL of the Celery broker
+ backend: 'rpc://' # Optional[str]: The URL of the Celery results backend
+ include: ['my_module'] # Optional[List[str]]: Modules every worker should import
+ config_source: # Dict[str, Any]: Any additional parameters to pass to the
+ #... # Celery workers. This dict will be passed as the `config_source`
+ #... # argument of celery.Celery().
+ ```
+ Note that the YAML you provide here must align with the configuration with which the Celery
+ workers on which you hope to run were started. If, for example, you point the executor at a
+ different broker than the one your workers are listening to, the workers will never be able to
+ pick up tasks for execution.
+
+
+
+
+
+
+
+
+
+
+
+## CLI
+
+The `dagster-celery` CLI lets you start, monitor, and terminate workers.
+
+
+ The name of the worker. Defaults to a unique name prefixed with “dagster-” and ending with the hostname.
+
+
+
+
+
-y, --config-yaml \
+
+ Specify the path to a config YAML file with options for the worker. This is the same config block that you provide to dagster_celery.celery_executor when configuring a job for execution with Celery, with, e.g., the URL of the broker to use.
+
+
+
+
+
-q, --queue \
+
+ Names of the queues on which this worker should listen for tasks. Provide multiple -q arguments to specify multiple queues. Note that each celery worker may listen on no more than four queues.
+
+
+
+
+
-d, --background
+
+ Set this flag to run the worker in the background.
+
+
+
+
+
-i, --includes \
+
+ Python modules the worker should import. Provide multiple -i arguments to specify multiple modules.
+
+
+
+
+
-l, --loglevel \
+
+ Log level for the worker.
+
+
+
+
+
-A, --app \
+
+
+
+
+
+Arguments:
+
+
+
ADDITIONAL_ARGS
+
+ Optional argument(s)
+
+
+
+
+
+
+
+
+
+### dagster-celery worker list
+
+List running dagster-celery workers. Note that we use the broker to contact the workers.
+
+ ```shell
+ dagster-celery worker list [OPTIONS]
+ ```
+Options:
+
+
+
-y, --config-yaml \
+
+ Specify the path to a config YAML file with options for the workers you are trying to manage. This is the same config block that you provide to dagster_celery.celery_executor when configuring a job for execution with Celery, with, e.g., the URL of the broker to use. Without this config file, you will not be able to find your workers (since the CLI won’t know how to reach the broker).
+
+
+
+
+
+
+
+
+### dagster-celery worker terminate
+
+Shut down dagster-celery workers. Note that we use the broker to send signals to the workers to terminate – if the broker is not running, this command is a no-op. Provide the argument NAME to terminate a specific worker by name.
+
+ ```shell
+ dagster-celery worker terminate [OPTIONS] [NAME]
+ ```
+Options:
+
+
+
-a, --all
+
+ Set this flag to terminate all running workers.
+
+
+
+
+
-y, --config-yaml \
+
+ Specify the path to a config YAML file with options for the workers you are trying to manage. This is the same config block that you provide to dagster_celery.celery_executor when configuring a job for execution with Celery, with, e.g., the URL of the broker to use. Without this config file, you will not be able to terminate your workers (since the CLI won’t know how to reach the broker).
+
+
+
+Arguments:
+
+
+
NAME
+
+ Optional argument
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-census.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-census.mdx
new file mode 100644
index 0000000000000..9bd967bd14f25
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-census.mdx
@@ -0,0 +1,132 @@
+---
+title: 'census (dagster-census)'
+title_meta: 'census (dagster-census) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'census (dagster-census) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Census (dagster-census)
+
+This library provides an integration with Census.
+
+
+
+ Executes a Census sync for a given `sync_id` and polls until that sync completes, raising
+ an error if it is unsuccessful.
+
+ It outputs a [`CensusOutput`](#dagster_census.CensusOutput) which contains the details of the Census
+ sync after it successfully completes.
+
+ It requires the use of the [`census_resource`](#dagster_census.census_resource), which allows it to
+ communicate with the Census API.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_census import census_resource, census_sync_op
+
+ my_census_resource = census_resource.configured(
+ {
+ "api_key": {"env": "CENSUS_API_KEY"},
+ }
+ )
+
+ sync_foobar = census_sync_op.configured({"sync_id": "foobar"}, name="sync_foobar")
+
+ @job(resource_defs={"census": my_census_resource})
+ def my_simple_census_job():
+ sync_foobar()
+ ```
+
+
+
+
+
+
dagster_census.census_resource ResourceDefinition
+
+
+
+
+ This resource allows users to programatically interface with the Census REST API to launch
+ syncs and monitor their progress. This currently implements only a subset of the functionality
+ exposed by the API.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_census import census_resource
+
+ my_census_resource = census_resource.configured(
+ {
+ "api_key": {"env": "CENSUS_API_KEY"},
+ }
+ )
+
+ @job(resource_defs={"census":my_census_resource})
+ def my_census_job():
+ ...
+ ```
+
+
+
+
+
+
class dagster_census.CensusResource
+
+ This class exposes methods on top of the Census REST API.
+
+
+
+
+
class dagster_census.CensusOutput
+
+
+ Contains recorded information about the state of a Census sync after a sync completes.
+
+
+
sync_run
+
+
+ The details of the specific sync run.
+
+ Type: Dict[str, Any]
+
+
+
+
+
+
source
+
+
+ Information about the source for the Census sync.
+
+ Type: Dict[str, Any]
+
+
+
+
+
+
destination
+
+
+ Information about the destination for the Census sync.
+
+ Type: Dict[str, Any]
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-dask.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-dask.mdx
new file mode 100644
index 0000000000000..a30381cfe6b6c
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-dask.mdx
@@ -0,0 +1,57 @@
+---
+title: 'dask (dagster-dask)'
+title_meta: 'dask (dagster-dask) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dask (dagster-dask) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Dask (dagster-dask)
+
+See also the [Dask deployment guide](https://docs.dagster.io/deployment/guides/dask/).
+
+
+
dagster_dask.dask_executor ExecutorDefinition
+
+
+
+
+ Dask-based executor.
+
+ The ‘cluster’ can be one of the following:
+ (‘existing’, ‘local’, ‘yarn’, ‘ssh’, ‘pbs’, ‘moab’, ‘sge’, ‘lsf’, ‘slurm’, ‘oar’, ‘kube’).
+
+ If the Dask executor is used without providing executor-specific config, a local Dask cluster
+ will be created (as when calling `dask.distributed.Client()`
+ with `dask.distributed.LocalCluster()`).
+
+ The Dask executor optionally takes the following config:
+
+ ```none
+ cluster:
+ {
+ local?: # takes distributed.LocalCluster parameters
+ {
+ timeout?: 5, # Timeout duration for initial connection to the scheduler
+ n_workers?: 4 # Number of workers to start
+ threads_per_worker?: 1 # Number of threads per each worker
+ }
+ }
+ ```
+ To use the dask_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster import job
+ from dagster_dask import dask_executor
+
+ @job(executor_def=dask_executor)
+ def dask_enabled_job():
+ pass
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-databricks.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-databricks.mdx
new file mode 100644
index 0000000000000..73e55601e0b95
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-databricks.mdx
@@ -0,0 +1,359 @@
+---
+title: 'databricks (dagster-databricks)'
+title_meta: 'databricks (dagster-databricks) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'databricks (dagster-databricks) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Databricks (dagster-databricks)
+
+The `dagster_databricks` package provides these main pieces of functionality:
+
+ - A resource, `databricks_pyspark_step_launcher`, which will execute a op within a Databricks
+ - An op factory, `create_databricks_run_now_op`, which creates an op that launches an existing
+ - A op factory, `create_databricks_submit_run_op`, which creates an op that submits a one-time run
+
+
+Note that, for the `databricks_pyspark_step_launcher`, either S3 or Azure Data Lake Storage config
+must be specified for ops to succeed, and the credentials for this storage must also be
+stored as a Databricks Secret and stored in the resource config so that the Databricks cluster can
+access storage.
+
+
+
+ Resource which provides a Python client for interacting with Databricks within an
+ op or asset.
+
+
+
+
+
+
+
class dagster_databricks.DatabricksClient
+
+
+ A thin wrapper over the Databricks REST API.
+
+
+
property workspace_client
+
+
+ Retrieve a reference to the underlying Databricks Workspace client. For more information,
+ see the [Databricks SDK for Python](https://docs.databricks.com/dev-tools/sdk-python.html).
+
+ Examples:
+
+ ```python
+ from dagster import op
+ from databricks.sdk import WorkspaceClient
+
+ @op(required_resource_keys={"databricks_client"})
+ def op1(context):
+ # Initialize the Databricks Jobs API
+ client = context.resources.databricks_client.api_client
+
+ # Example 1: Run a Databricks job with some parameters.
+ client.jobs.run_now(...)
+
+ # Example 2: Trigger a one-time run of a Databricks workload.
+ client.jobs.submit(...)
+
+ # Example 3: Get an existing run.
+ client.jobs.get_run(...)
+
+ # Example 4: Cancel a run.
+ client.jobs.cancel_run(...)
+ ```
+ Returns: The authenticated Databricks SDK Workspace Client.Return type: WorkspaceClient
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Ops
+
+
+
dagster_databricks.create_databricks_run_now_op
+
+
+ Creates an op that launches an existing databricks job.
+
+ As config, the op accepts a blob of the form described in Databricks’ Job API:
+ [https://docs.databricks.com/api/workspace/jobs/runnow](https://docs.databricks.com/api/workspace/jobs/runnow). The only required field is
+ `job_id`, which is the ID of the job to be executed. Additional fields can be used to specify
+ override parameters for the Databricks Job.
+
+ Parameters:
+ - databricks_job_id (int) – The ID of the Databricks Job to be executed.
+ - databricks_job_configuration (dict) – Configuration for triggering a new job run of a
+ - poll_interval_seconds (float) – How often to poll the Databricks API to check whether the
+ - max_wait_time_seconds (float) – How long to wait for the Databricks job to finish running
+ - name (Optional[str]) – The name of the op. If not provided, the name will be
+ - databricks_resource_key (str) – The name of the resource key used by this op. If not
+
+
+ Returns: An op definition to run the Databricks Job.Return type: [OpDefinition](../ops.mdx#dagster.OpDefinition)
+ Example:
+
+ ```python
+ from dagster import job
+ from dagster_databricks import create_databricks_run_now_op, DatabricksClientResource
+
+ DATABRICKS_JOB_ID = 1234
+
+
+ run_now_op = create_databricks_run_now_op(
+ databricks_job_id=DATABRICKS_JOB_ID,
+ databricks_job_configuration={
+ "python_params": [
+ "--input",
+ "schema.db.input_table",
+ "--output",
+ "schema.db.output_table",
+ ],
+ },
+ )
+
+ @job(
+ resource_defs={
+ "databricks": DatabricksClientResource(
+ host=EnvVar("DATABRICKS_HOST"),
+ token=EnvVar("DATABRICKS_TOKEN")
+ )
+ }
+ )
+ def do_stuff():
+ run_now_op()
+ ```
+
+
+
+ Creates an op that submits a one-time run of a set of tasks on Databricks.
+
+ As config, the op accepts a blob of the form described in Databricks’ Job API:
+ [https://docs.databricks.com/api/workspace/jobs/submit](https://docs.databricks.com/api/workspace/jobs/submit).
+
+ Parameters:
+ - databricks_job_configuration (dict) – Configuration for submitting a one-time run of a set
+ - poll_interval_seconds (float) – How often to poll the Databricks API to check whether the
+ - max_wait_time_seconds (float) – How long to wait for the Databricks job to finish running
+ - name (Optional[str]) – The name of the op. If not provided, the name will be
+ - databricks_resource_key (str) – The name of the resource key used by this op. If not
+
+
+ Returns: An op definition to submit a one-time run of a set of tasks on Databricks.Return type: [OpDefinition](../ops.mdx#dagster.OpDefinition)
+ Example:
+
+ ```python
+ from dagster import job
+ from dagster_databricks import create_databricks_submit_run_op, DatabricksClientResource
+
+
+ submit_run_op = create_databricks_submit_run_op(
+ databricks_job_configuration={
+ "new_cluster": {
+ "spark_version": '2.1.0-db3-scala2.11',
+ "num_workers": 2
+ },
+ "notebook_task": {
+ "notebook_path": "/Users/dagster@example.com/PrepareData",
+ },
+ }
+ )
+
+ @job(
+ resource_defs={
+ "databricks": DatabricksClientResource(
+ host=EnvVar("DATABRICKS_HOST"),
+ token=EnvVar("DATABRICKS_TOKEN")
+ )
+ }
+ )
+ def do_stuff():
+ submit_run_op()
+ ```
+
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ While there is no plan to remove this functionality, for new projects, we recommend using Dagster Pipes. For more information, see https://docs.dagster.io/guides/build/external-pipelines/.
+
+ :::
+
+ Resource for running ops as a Databricks Job.
+
+ When this resource is used, the op will be executed in Databricks using the ‘Run Submit’
+ API. Pipeline code will be zipped up and copied to a directory in DBFS along with the op’s
+ execution context.
+
+ Use the ‘run_config’ configuration to specify the details of the Databricks cluster used, and
+ the ‘storage’ key to configure persistent storage on that cluster. Storage is accessed by
+ setting the credentials in the Spark context, as documented [here for S3](https://docs.databricks.com/data/data-sources/aws/amazon-s3.html#alternative-1-set-aws-keys-in-the-spark-context) and [here for ADLS](https://docs.microsoft.com/en-gb/azure/databricks/data/data-sources/azure/azure-datalake-gen2#--access-directly-using-the-storage-account-access-key).
+
+
+
+
+
+
+
+
+
+
+
+## Pipes
+
+
+
class dagster_databricks.PipesDatabricksClient
+
+
+ Pipes client for databricks.
+
+ Parameters:
+ - client (WorkspaceClient) – A databricks WorkspaceClient object.
+ - (Optional[Mapping[str (env) – An optional dict of environment
+ - str]] – An optional dict of environment
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - poll_interval_seconds (float) – How long to sleep between checking the status of the job run.
+ - forward_termination (bool) – Whether to cancel the Databricks job if the orchestration process
+
+
+
+
+
+
+
+
class dagster_databricks.PipesDbfsContextInjector
+
+
+ A context injector that injects context into a Databricks job by writing a JSON file to DBFS.
+
+ Parameters: client (WorkspaceClient) – A databricks WorkspaceClient object.
+
+
+
+
+
+
class dagster_databricks.PipesDbfsMessageReader
+
+
+ Message reader that reads messages by periodically reading message chunks from an
+ automatically-generated temporary directory on DBFS.
+
+ If log_readers is passed, this reader will also start the passed readers
+ when the first message is received from the external process.
+
+ Parameters:
+ - interval (float) – interval in seconds between attempts to download a chunk
+ - client (WorkspaceClient) – A databricks WorkspaceClient object.
+ - cluster_log_root (Optional[str]) – The root path on DBFS where the cluster logs are written.
+ - include_stdio_in_messages (bool) – Whether to send stdout/stderr to Dagster via Pipes messages. Defaults to False.
+ - log_readers (Optional[Sequence[PipesLogReader]]) – A set of log readers for logs on DBFS.
+
+
+
+
+
+
+
+
class dagster_databricks.PipesDbfsLogReader
+
+
+ Reader that reads a log file from DBFS.
+
+ Parameters:
+ - interval (float) – interval in seconds between attempts to download a log chunk
+ - remote_log_name (Literal["stdout", "stderr"]) – The name of the log file to read.
+ - target_stream (TextIO) – The stream to which to forward log chunks that have been read.
+ - client (WorkspaceClient) – A databricks WorkspaceClient object.
+ - debug_info (Optional[str]) – An optional message containing debug information about the log reader.
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-datadog.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-datadog.mdx
new file mode 100644
index 0000000000000..0f71e87fd3fcf
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-datadog.mdx
@@ -0,0 +1,126 @@
+---
+title: 'datadog (dagster-datadog)'
+title_meta: 'datadog (dagster-datadog) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'datadog (dagster-datadog) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Datadog (dagster-datadog)
+
+This library provides an integration with Datadog, to support publishing metrics to Datadog from
+within Dagster ops.
+
+We use the Python [datadogpy](https://github.com/DataDog/datadogpy) library. To use it, you’ll
+first need to create a DataDog account and get both [API and Application keys](https://docs.datadoghq.com/account_management/api-app-keys/).
+
+The integration uses [DogStatsD](https://docs.datadoghq.com/developers/dogstatsd/), so you’ll need
+to ensure the datadog agent is running on the host you’re sending metrics from.
+
+
+
+ This legacy resource is a thin wrapper over the
+ [dogstatsd library](https://datadogpy.readthedocs.io/en/latest/).
+
+ Prefer using [`DatadogResource`](#dagster_datadog.DatadogResource).
+
+ As such, we directly mirror the public API methods of DogStatsd here; you can refer to the
+ [DataDog documentation](https://docs.datadoghq.com/developers/dogstatsd/) for how to use this
+ resource.
+
+ Examples:
+
+ ```python
+ @op(required_resource_keys={'datadog'})
+ def datadog_op(context):
+ dd = context.resources.datadog
+
+ dd.event('Man down!', 'This server needs assistance.')
+ dd.gauge('users.online', 1001, tags=["protocol:http"])
+ dd.increment('page.views')
+ dd.decrement('page.views')
+ dd.histogram('album.photo.count', 26, tags=["gender:female"])
+ dd.distribution('album.photo.count', 26, tags=["color:blue"])
+ dd.set('visitors.uniques', 999, tags=["browser:ie"])
+ dd.service_check('svc.check_name', dd.WARNING)
+ dd.timing("query.response.time", 1234)
+
+ # Use timed decorator
+ @dd.timed('run_fn')
+ def run_fn():
+ pass
+
+ run_fn()
+
+ @job(resource_defs={'datadog': datadog_resource})
+ def dd_job():
+ datadog_op()
+
+ result = dd_job.execute_in_process(
+ run_config={'resources': {'datadog': {'config': {'api_key': 'YOUR_KEY', 'app_key': 'YOUR_KEY'}}}}
+ )
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-datahub.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-datahub.mdx
new file mode 100644
index 0000000000000..cd9ca3a41ea4e
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-datahub.mdx
@@ -0,0 +1,172 @@
+---
+title: 'datahub (dagster-datahub)'
+title_meta: 'datahub (dagster-datahub) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'datahub (dagster-datahub) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Datahub (dagster-datahub)
+
+This library provides an integration with Datahub, to support pushing metadata to Datahub from
+within Dagster ops.
+
+
+
+We use the [Datahub Python Library](https://github.com/datahub-project/datahub). To use it, you’ll
+first need to start up a Datahub Instance. [Datahub Quickstart Guide](https://datahubproject.io/docs/quickstart).
+
+
+
+ Base class for Dagster resources that utilize structured config.
+
+ This class is a subclass of both `ResourceDefinition` and `Config`.
+
+ Example definition:
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ prefix: str
+
+ def output(self, text: str) -> None:
+ print(f"{self.prefix}{text}")
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def asset_that_uses_writer(writer: WriterResource):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[asset_that_uses_writer],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+ You can optionally use this class to model configuration only and vend an object
+ of a different type for use at runtime. This is useful for those who wish to
+ have a separate object that manages configuration and a separate object at runtime. Or
+ where you want to directly use a third-party class that you do not control.
+
+ To do this you override the create_resource methods to return a different object.
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ str: prefix
+
+ def create_resource(self, context: InitResourceContext) -> Writer:
+ # Writer is pre-existing class defined else
+ return Writer(self.prefix)
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def use_preexisting_writer_as_resource(writer: ResourceParam[Writer]):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[use_preexisting_writer_as_resource],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+
+
+
+ Base class for Dagster resources that utilize structured config.
+
+ This class is a subclass of both `ResourceDefinition` and `Config`.
+
+ Example definition:
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ prefix: str
+
+ def output(self, text: str) -> None:
+ print(f"{self.prefix}{text}")
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def asset_that_uses_writer(writer: WriterResource):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[asset_that_uses_writer],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+ You can optionally use this class to model configuration only and vend an object
+ of a different type for use at runtime. This is useful for those who wish to
+ have a separate object that manages configuration and a separate object at runtime. Or
+ where you want to directly use a third-party class that you do not control.
+
+ To do this you override the create_resource methods to return a different object.
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ str: prefix
+
+ def create_resource(self, context: InitResourceContext) -> Writer:
+ # Writer is pre-existing class defined else
+ return Writer(self.prefix)
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def use_preexisting_writer_as_resource(writer: ResourceParam[Writer]):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[use_preexisting_writer_as_resource],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-dbt.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-dbt.mdx
new file mode 100644
index 0000000000000..d56591a47b60e
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-dbt.mdx
@@ -0,0 +1,2058 @@
+---
+title: 'dbt (dagster-dbt)'
+title_meta: 'dbt (dagster-dbt) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dbt (dagster-dbt) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# dbt (dagster-dbt)
+
+Dagster orchestrates dbt alongside other technologies, so you can combine dbt with Spark, Python,
+etc. in a single workflow. Dagster’s software-defined asset abstractions make it simple to define
+data assets that depend on specific dbt models, or to define the computation required to compute
+the sources that your dbt models depend on.
+
+Related documentation pages: [dbt](https://docs.dagster.io/integrations/libraries/dbt/) and
+[dbt Cloud](https://docs.dagster.io/integrations/dbt-cloud).
+
+
+
+
+## dagster-dbt
+
+
+
+
+### dagster-dbt project
+
+Commands for using a dbt project in Dagster.
+
+ ```shell
+ dagster-dbt project [OPTIONS] COMMAND [ARGS]...
+ ```
+
+
+
+#### prepare-and-package
+
+This command will invoke `prepare_and_package` on [`DbtProject`](#dagster_dbt.DbtProject) found in the target module or file.
+Note that this command runs dbt deps and dbt parse.
+
+ ```shell
+ dagster-dbt project prepare-and-package [OPTIONS]
+ ```
+Options:
+
+
+
--file \
+
+ Required The file containing DbtProject definitions to prepare.
+
+
+
+
+
+
+
+
+
+#### scaffold
+
+This command will initialize a new Dagster project and create directories and files that
+load assets from an existing dbt project.
+
+ ```shell
+ dagster-dbt project scaffold [OPTIONS]
+ ```
+Options:
+
+
+
--project-name \
+
+ Required The name of the Dagster project to initialize for your dbt project.
+
+
+
+
+
--dbt-project-dir \
+
+ The path of your dbt project directory. This path must contain a dbt_project.yml file. By default, this command will assume that the current working directory contains a dbt project, but you can set a different directory by setting this option.
+
+
+
+
+
+
+
+
+
+## dbt Core
+
+Here, we provide interfaces to manage dbt projects invoked by the local dbt command line interface
+(dbt CLI).
+
+
+
+
+### Assets (dbt Core)
+
+
+
@dagster_dbt.dbt_assets
+
+
+ Create a definition for how to compute a set of dbt resources, described by a manifest.json.
+ When invoking dbt commands using [`DbtCliResource`](#dagster_dbt.DbtCliResource)’s
+ [`cli()`](#dagster_dbt.DbtCliResource.cli) method, Dagster events are emitted by calling
+ `yield from` on the event stream returned by [`stream()`](#dagster_dbt.DbtCliInvocation.stream).
+
+ Parameters:
+ - manifest (Union[Mapping[str, Any], str, Path]) – The contents of a manifest.json file
+ - select (str) – A dbt selection string for the models in a project that you want
+ - exclude (Optional[str]) – A dbt selection string for the models in a project that you want
+ - name (Optional[str]) – The name of the op.
+ - io_manager_key (Optional[str]) – The IO manager key that will be set on each of the returned
+ - partitions_def (Optional[[*PartitionsDefinition*](../partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - dagster_dbt_translator (Optional[[*DagsterDbtTranslator*](#dagster_dbt.DagsterDbtTranslator)]) – Allows customizing how to map
+ - backfill_policy (Optional[[*BackfillPolicy*](../partitions.mdx#dagster.BackfillPolicy)]) – If a partitions_def is defined, this determines
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the assets.
+ - required_resource_keys (Optional[Set[str]]) – Set of required resource handles.
+ - project (Optional[[*DbtProject*](#dagster_dbt.DbtProject)]) – A DbtProject instance which provides a pointer to the dbt
+ - retry_policy (Optional[[*RetryPolicy*](../ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that computes the asset.
+
+
+ Examples:
+
+ Running `dbt build` for a dbt project:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ yield from dbt.cli(["build"], context=context).stream()
+ ```
+ Running dbt commands with flags:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ yield from dbt.cli(["build", "--full-refresh"], context=context).stream()
+ ```
+ Running dbt commands with `--vars`:
+
+ ```python
+ import json
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_vars = {"key": "value"}
+
+ yield from dbt.cli(["build", "--vars", json.dumps(dbt_vars)], context=context).stream()
+ ```
+ Retrieving dbt artifacts after running a dbt command:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_build_invocation = dbt.cli(["build"], context=context)
+
+ yield from dbt_build_invocation.stream()
+
+ run_results_json = dbt_build_invocation.get_artifact("run_results.json")
+ ```
+ Running multiple dbt commands for a dbt project:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ yield from dbt.cli(["run"], context=context).stream()
+ yield from dbt.cli(["test"], context=context).stream()
+ ```
+ Accessing the dbt event stream alongside the Dagster event stream:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_cli_invocation = dbt.cli(["build"], context=context)
+
+ # Each dbt event is structured: https://docs.getdbt.com/reference/events-logging
+ for dbt_event in dbt_invocation.stream_raw_events():
+ for dagster_event in dbt_event.to_default_asset_events(
+ manifest=dbt_invocation.manifest,
+ dagster_dbt_translator=dbt_invocation.dagster_dbt_translator,
+ context=dbt_invocation.context,
+ target_path=dbt_invocation.target_path,
+ ):
+ # Manipulate `dbt_event`
+ ...
+
+ # Then yield the Dagster event
+ yield dagster_event
+ ```
+ Customizing the Dagster asset definition metadata inferred from a dbt project using [`DagsterDbtTranslator`](#dagster_dbt.DagsterDbtTranslator):
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DagsterDbtTranslator, DbtCliResource, dbt_assets
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ ...
+
+
+ @dbt_assets(
+ manifest=Path("target", "manifest.json"),
+ dagster_dbt_translator=CustomDagsterDbtTranslator(),
+ )
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ yield from dbt.cli(["build"], context=context).stream()
+ ```
+ Using a custom resource key for dbt:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, my_custom_dbt_resource_key: DbtCliResource):
+ yield from my_custom_dbt_resource_key.cli(["build"], context=context).stream()
+ ```
+ Using a dynamically generated resource key for dbt using required_resource_keys:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ dbt_resource_key = "my_custom_dbt_resource_key"
+
+ @dbt_assets(manifest=Path("target", "manifest.json"), required_resource_keys={my_custom_dbt_resource_key})
+ def my_dbt_assets(context: AssetExecutionContext):
+ dbt = getattr(context.resources, dbt_resource_key)
+ yield from dbt.cli(["build"], context=context).stream()
+ ```
+ Invoking another Dagster [`ResourceDefinition`](../resources.mdx#dagster.ResourceDefinition) alongside dbt:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+ from dagster_slack import SlackResource
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource, slack: SlackResource):
+ yield from dbt.cli(["build"], context=context).stream()
+
+ slack_client = slack.get_client()
+ slack_client.chat_postMessage(channel="#my-channel", text="dbt build succeeded!")
+ ```
+ Defining and accessing Dagster [`Config`](../config.mdx#dagster.Config) alongside dbt:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext, Config
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ class MyDbtConfig(Config):
+ full_refresh: bool
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource, config: MyDbtConfig):
+ dbt_build_args = ["build"]
+ if config.full_refresh:
+ dbt_build_args += ["--full-refresh"]
+
+ yield from dbt.cli(dbt_build_args, context=context).stream()
+ ```
+ Defining Dagster `PartitionDefinition` alongside dbt:
+
+ ```python
+ import json
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext, DailyPartitionDefinition
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(
+ manifest=Path("target", "manifest.json"),
+ partitions_def=DailyPartitionsDefinition(start_date="2023-01-01")
+ )
+ def partitionshop_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ time_window = context.partition_time_window
+
+ dbt_vars = {
+ "min_date": time_window.start.isoformat(),
+ "max_date": time_window.end.isoformat()
+ }
+ dbt_build_args = ["build", "--vars", json.dumps(dbt_vars)]
+
+ yield from dbt.cli(dbt_build_args, context=context).stream()
+ ```
+
+
+
+
+
+
class dagster_dbt.DagsterDbtTranslator
+
+
+ Holds a set of methods that derive Dagster asset definition metadata given a representation
+ of a dbt resource (models, tests, sources, etc).
+
+ This class is exposed so that methods can be overriden to customize how Dagster asset metadata
+ is derived.
+
+
+
get_asset_key
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster asset key that represents that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom asset key for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: The Dagster asset key for the dbt resource.Return type: [AssetKey](../assets.mdx#dagster.AssetKey)
+ Examples:
+
+ Adding a prefix to the default asset key generated for each dbt resource:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster import AssetKey
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_asset_key(self, dbt_resource_props: Mapping[str, Any]) -> AssetKey:
+ return super().get_asset_key(dbt_resource_props).with_prefix("prefix")
+ ```
+ Adding a prefix to the default asset key generated for each dbt resource, but only for dbt sources:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster import AssetKey
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_asset_key(self, dbt_resource_props: Mapping[str, Any]) -> AssetKey:
+ asset_key = super().get_asset_key(dbt_resource_props)
+
+ if dbt_resource_props["resource_type"] == "source":
+ asset_key = asset_key.with_prefix("my_prefix")
+
+ return asset_key
+ ```
+
+
+
+
+
+
get_auto_materialize_policy
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster `dagster.AutoMaterializePolicy` for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom auto-materialize policy for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A Dagster auto-materialize policy.Return type: Optional[AutoMaterializePolicy]
+ Examples:
+
+ Set a custom auto-materialize policy for all dbt resources:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_auto_materialize_policy(self, dbt_resource_props: Mapping[str, Any]) -> Optional[AutoMaterializePolicy]:
+ return AutoMaterializePolicy.eager()
+ ```
+ Set a custom auto-materialize policy for dbt resources with a specific tag:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_auto_materialize_policy(self, dbt_resource_props: Mapping[str, Any]) -> Optional[AutoMaterializePolicy]:
+ auto_materialize_policy = None
+ if "my_custom_tag" in dbt_resource_props.get("tags", []):
+ auto_materialize_policy = AutoMaterializePolicy.eager()
+
+ return auto_materialize_policy
+ ```
+
+
+
+
+
+
get_automation_condition
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster `dagster.AutoMaterializePolicy` for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom AutomationCondition for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A Dagster auto-materialize policy.Return type: Optional[AutoMaterializePolicy]
+ Examples:
+
+ Set a custom AutomationCondition for all dbt resources:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_automation_condition(self, dbt_resource_props: Mapping[str, Any]) -> Optional[AutomationCondition]:
+ return AutomationCondition.eager()
+ ```
+ Set a custom AutomationCondition for dbt resources with a specific tag:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_automation_condition(self, dbt_resource_props: Mapping[str, Any]) -> Optional[AutomationCondition]:
+ automation_condition = None
+ if "my_custom_tag" in dbt_resource_props.get("tags", []):
+ automation_condition = AutomationCondition.eager()
+
+ return automation_condition
+ ```
+
+
+
+
+
+
get_code_version
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster code version for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom code version for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A Dagster code version.Return type: Optional[str]
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_code_version(self, dbt_resource_props: Mapping[str, Any]) -> Optional[str]:
+ return dbt_resource_props["checksum"]["checksum"]
+ ```
+
+
+
+
+
+
get_description
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster description for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom description for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: The description for the dbt resource.Return type: str
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_description(self, dbt_resource_props: Mapping[str, Any]) -> str:
+ return "custom description"
+ ```
+
+
+
+
+
+
get_freshness_policy
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster `dagster.FreshnessPolicy` for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom freshness policy for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A Dagster freshness policy.Return type: Optional[FreshnessPolicy]
+ Examples:
+
+ Set a custom freshness policy for all dbt resources:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_freshness_policy(self, dbt_resource_props: Mapping[str, Any]) -> Optional[FreshnessPolicy]:
+ return FreshnessPolicy(maximum_lag_minutes=60)
+ ```
+ Set a custom freshness policy for dbt resources with a specific tag:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_freshness_policy(self, dbt_resource_props: Mapping[str, Any]) -> Optional[FreshnessPolicy]:
+ freshness_policy = None
+ if "my_custom_tag" in dbt_resource_props.get("tags", []):
+ freshness_policy = FreshnessPolicy(maximum_lag_minutes=60)
+
+ return freshness_policy
+ ```
+
+
+
+
+
+
get_group_name
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster group name for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom group name for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A Dagster group name.Return type: Optional[str]
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_group_name(self, dbt_resource_props: Mapping[str, Any]) -> Optional[str]:
+ return "custom_group_prefix" + dbt_resource_props.get("config", {}).get("group")
+ ```
+
+
+
+
+
+
get_metadata
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster metadata for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom metadata for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A dictionary representing the Dagster metadata for the dbt resource.Return type: Mapping[str, Any]
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_metadata(self, dbt_resource_props: Mapping[str, Any]) -> Mapping[str, Any]:
+ return {"custom": "metadata"}
+ ```
+
+
+
+
+
+
get_owners
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster owners for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide custom owners for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A set of Dagster owners.Return type: Optional[Sequence[str]]
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_owners(self, dbt_resource_props: Mapping[str, Any]) -> Optional[Sequence[str]]:
+ return ["user@owner.com", "team:team@owner.com"]
+ ```
+
+
+
+
+
+
get_partition_mapping
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A function that takes two dictionaries: the first, representing properties of a dbt
+ resource; and the second, representing the properties of a parent dependency to the first
+ dbt resource. The function returns the Dagster partition mapping for the dbt dependency.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom partition mapping for a dbt dependency.
+
+ Parameters:
+ - dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt child resource.
+ - dbt_parent_resource_props (Mapping[str, Any]) – A dictionary representing the dbt parent resource, in relationship to the child.
+
+
+ Returns: The Dagster partition mapping for the dbt resource. If None is returned, the
+ default partition mapping will be used.Return type: Optional[[PartitionMapping](../partitions.mdx#dagster.PartitionMapping)]
+
+
+
+
+
+
get_tags
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster tags for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ dbt tags are strings, but Dagster tags are key-value pairs. To bridge this divide, the dbt
+ tag string is used as the Dagster tag key, and the Dagster tag value is set to the empty
+ string, “”.
+
+ Any dbt tags that don’t match Dagster’s supported tag key format (e.g. they contain
+ unsupported characters) will be ignored.
+
+ This method can be overridden to provide custom tags for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A dictionary representing the Dagster tags for the dbt resource.Return type: Mapping[str, str]
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_tags(self, dbt_resource_props: Mapping[str, Any]) -> Mapping[str, str]:
+ return {"custom": "tag"}
+ ```
+
+
+
+
+
+
+
+
+
+
class dagster_dbt.DagsterDbtTranslatorSettings
+
+
+ Settings to enable Dagster features for your dbt project.
+
+ Parameters:
+ - enable_asset_checks (bool) – Whether to load dbt tests as Dagster asset checks.
+ - enable_duplicate_source_asset_keys (bool) – Whether to allow dbt sources with duplicate
+ - enable_code_references (bool) – Whether to enable Dagster code references for dbt resources.
+ - enable_dbt_selection_by_name (bool) – Whether to enable selecting dbt resources by name,
+
+
+
+
+
+
+
+
class dagster_dbt.DbtManifestAssetSelection
+
+
+ Defines a selection of assets from a dbt manifest wrapper and a dbt selection string.
+
+ Parameters:
+ - manifest (Mapping[str, Any]) – The dbt manifest blob.
+ - select (str) – A dbt selection string to specify a set of dbt resources.
+ - exclude (Optional[str]) – A dbt selection string to exclude a set of dbt resources.
+
+
+ Examples:
+
+ ```python
+ import json
+ from pathlib import Path
+
+ from dagster_dbt import DbtManifestAssetSelection
+
+ manifest = json.loads(Path("path/to/manifest.json").read_text())
+
+ # select the dbt assets that have the tag "foo".
+ my_selection = DbtManifestAssetSelection(manifest=manifest, select="tag:foo")
+ ```
+
+
+
+
+
+
dagster_dbt.build_dbt_asset_selection
+
+
+ Build an asset selection for a dbt selection string.
+
+ See [https://docs.getdbt.com/reference/node-selection/syntax#how-does-selection-work](https://docs.getdbt.com/reference/node-selection/syntax#how-does-selection-work) for
+ more information.
+
+ Parameters:
+ - dbt_select (str) – A dbt selection string to specify a set of dbt resources.
+ - dbt_exclude (Optional[str]) – A dbt selection string to exclude a set of dbt resources.
+
+
+ Returns: An asset selection for the selected dbt nodes.Return type: [AssetSelection](../assets.mdx#dagster.AssetSelection)
+ Examples:
+
+ ```python
+ from dagster_dbt import dbt_assets, build_dbt_asset_selection
+
+ @dbt_assets(manifest=...)
+ def all_dbt_assets():
+ ...
+
+ # Select the dbt assets that have the tag "foo".
+ foo_selection = build_dbt_asset_selection([dbt_assets], dbt_select="tag:foo")
+
+ # Select the dbt assets that have the tag "foo" and all Dagster assets downstream
+ # of them (dbt-related or otherwise)
+ foo_and_downstream_selection = foo_selection.downstream()
+ ```
+ Building an asset selection on a dbt assets definition with an existing selection:
+
+ ```python
+ from dagster_dbt import dbt_assets, build_dbt_asset_selection
+
+ @dbt_assets(
+ manifest=...
+ select="bar+",
+ )
+ def bar_plus_dbt_assets():
+ ...
+
+ # Select the dbt assets that are in the intersection of having the tag "foo" and being
+ # in the existing selection "bar+".
+ bar_plus_and_foo_selection = build_dbt_asset_selection(
+ [bar_plus_dbt_assets],
+ dbt_select="tag:foo"
+ )
+
+ # Furthermore, select all assets downstream (dbt-related or otherwise).
+ bar_plus_and_foo_and_downstream_selection = bar_plus_and_foo_selection.downstream()
+ ```
+
+
+
+
+
+
dagster_dbt.build_schedule_from_dbt_selection
+
+
+ Build a schedule to materialize a specified set of dbt resources from a dbt selection string.
+
+ See [https://docs.getdbt.com/reference/node-selection/syntax#how-does-selection-work](https://docs.getdbt.com/reference/node-selection/syntax#how-does-selection-work) for
+ more information.
+
+ Parameters:
+ - job_name (str) – The name of the job to materialize the dbt resources.
+ - cron_schedule (str) – The cron schedule to define the schedule.
+ - dbt_select (str) – A dbt selection string to specify a set of dbt resources.
+ - dbt_exclude (Optional[str]) – A dbt selection string to exclude a set of dbt resources.
+ - schedule_name (Optional[str]) – The name of the dbt schedule to create.
+ - tags (Optional[Mapping[str, str]]) – A dictionary of tags (string key-value pairs) to attach
+ - config (Optional[[*RunConfig*](../config.mdx#dagster.RunConfig)]) – The config that parameterizes the execution of this schedule.
+ - execution_timezone (Optional[str]) – Timezone in which the schedule should run.
+
+
+ Returns: A definition to materialize the selected dbt resources on a cron schedule.Return type: [ScheduleDefinition](../schedules-sensors.mdx#dagster.ScheduleDefinition)
+ Examples:
+
+ ```python
+ from dagster_dbt import dbt_assets, build_schedule_from_dbt_selection
+
+ @dbt_assets(manifest=...)
+ def all_dbt_assets():
+ ...
+
+ daily_dbt_assets_schedule = build_schedule_from_dbt_selection(
+ [all_dbt_assets],
+ job_name="all_dbt_assets",
+ cron_schedule="0 0 * * *",
+ dbt_select="fqn:*",
+ )
+ ```
+
+
+
+
+
+
dagster_dbt.get_asset_key_for_model
+
+
+ Return the corresponding Dagster asset key for a dbt model, seed, or snapshot.
+
+ Parameters:
+ - dbt_assets ([*AssetsDefinition*](../assets.mdx#dagster.AssetsDefinition)) – An AssetsDefinition object produced by @dbt_assets.
+ - model_name (str) – The name of the dbt model, seed, or snapshot.
+
+
+ Returns: The corresponding Dagster asset key.Return type: [AssetKey](../assets.mdx#dagster.AssetKey)
+ Examples:
+
+ ```python
+ from dagster import asset
+ from dagster_dbt import dbt_assets, get_asset_key_for_model
+
+ @dbt_assets(manifest=...)
+ def all_dbt_assets():
+ ...
+
+
+ @asset(deps={get_asset_key_for_model([all_dbt_assets], "customers")})
+ def cleaned_customers():
+ ...
+ ```
+
+
+
+
+
+
dagster_dbt.get_asset_key_for_source
+
+
+ Returns the corresponding Dagster asset key for a dbt source with a singular table.
+
+ Parameters: source_name (str) – The name of the dbt source.Raises: DagsterInvalidInvocationError – If the source has more than one table.Returns: The corresponding Dagster asset key.Return type: [AssetKey](../assets.mdx#dagster.AssetKey)
+ Examples:
+
+ ```python
+ from dagster import asset
+ from dagster_dbt import dbt_assets, get_asset_key_for_source
+
+ @dbt_assets(manifest=...)
+ def all_dbt_assets():
+ ...
+
+ @asset(key=get_asset_key_for_source([all_dbt_assets], "my_source"))
+ def upstream_python_asset():
+ ...
+ ```
+
+
+
+ Returns the corresponding Dagster asset keys for all tables in a dbt source.
+
+ This is a convenience method that makes it easy to define a multi-asset that generates
+ all the tables for a given dbt source.
+
+ Parameters: source_name (str) – The name of the dbt source.Returns:
+ A mapping of the table name to corresponding Dagster asset key
+ for all tables in the given dbt source.
+
+ Return type: Mapping[str, [AssetKey](../assets.mdx#dagster.AssetKey)]
+ Examples:
+
+ ```python
+ from dagster import AssetOut, multi_asset
+ from dagster_dbt import dbt_assets, get_asset_keys_by_output_name_for_source
+
+ @dbt_assets(manifest=...)
+ def all_dbt_assets():
+ ...
+
+ @multi_asset(
+ outs={
+ name: AssetOut(key=asset_key)
+ for name, asset_key in get_asset_keys_by_output_name_for_source(
+ [all_dbt_assets], "raw_data"
+ ).items()
+ },
+ )
+ def upstream_python_asset():
+ ...
+ ```
+
+
+
+
+
+
class dagster_dbt.DbtProject
+
+
+ Representation of a dbt project and related settings that assist with managing the project preparation.
+
+ Using this helps achieve a setup where the dbt manifest file
+ and dbt dependencies are available and up-to-date:
+ * during development, pull the dependencies and reload the manifest at run time to pick up any changes.
+ * when deployed, expect a manifest that was created at build time to reduce start-up time.
+
+ The cli `dagster-dbt project prepare-and-package` can be used as part of the deployment process to
+ handle the project preparation.
+
+ This object can be passed directly to [`DbtCliResource`](#dagster_dbt.DbtCliResource).
+
+ Parameters:
+ - project_dir (Union[str, Path]) – The directory of the dbt project.
+ - target_path (Union[str, Path]) – The path, relative to the project directory, to output artifacts.
+ - profiles_dir (Union[str, Path]) – The path to the directory containing your dbt profiles.yml.
+ - profile (Optional[str]) – The profile from your dbt profiles.yml to use for execution, if it should be explicitly set.
+ - target (Optional[str]) – The target from your dbt profiles.yml to use for execution, if it should be explicitly set.
+ - packaged_project_dir (Optional[Union[str, Path]]) – A directory that will contain a copy of the dbt project and the manifest.json
+ - state_path (Optional[Union[str, Path]]) – The path, relative to the project directory, to reference artifacts from another run.
+
+
+ Examples:
+
+ Creating a DbtProject with by referencing the dbt project directory:
+
+ ```python
+ from pathlib import Path
+
+ from dagster_dbt import DbtProject
+
+ my_project = DbtProject(project_dir=Path("path/to/dbt_project"))
+ ```
+ Creating a DbtProject that changes target based on environment variables and uses manged state artifacts:
+
+ ```python
+ import os
+ from pathlib import Path
+ from dagster_dbt import DbtProject
+
+
+ def get_env():
+ if os.getenv("DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT", "") == "1":
+ return "BRANCH"
+ if os.getenv("DAGSTER_CLOUD_DEPLOYMENT_NAME", "") == "prod":
+ return "PROD"
+ return "LOCAL"
+
+
+ dbt_project = DbtProject(
+ project_dir=Path('path/to/dbt_project'),
+ state_path="target/managed_state",
+ target=get_env(),
+ )
+ ```
+
+
prepare_if_dev
+
+
+ Prepare a dbt project at run time during development, i.e. when dagster dev is used.
+ This method has no effect outside this development context.
+
+ The preparation process ensures that the dbt manifest file and dbt dependencies are available and up-to-date.
+ During development, it pulls the dependencies and reloads the manifest at run time to pick up any changes.
+
+ If this method returns successfully, self.manifest_path will point to a loadable manifest file.
+ This method causes errors if the manifest file has not been correctly created by the preparation process.
+
+ Examples:
+
+ Preparing a DbtProject during development:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import Definitions
+ from dagster_dbt import DbtProject
+
+ my_project = DbtProject(project_dir=Path("path/to/dbt_project"))
+ my_project.prepare_if_dev()
+
+ defs = Definitions(
+ resources={
+ "dbt": DbtCliResource(project_dir=my_project),
+ },
+ ...
+ )
+ ```
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a sequence of freshness checks constructed from the provided dbt assets.
+
+ Freshness checks can be configured on a per-model basis in the model schema configuration.
+
+ For assets which are not partitioned based on time, the freshness check configuration mirrors
+ that of the `build_last_update_freshness_checks()` function. lower_bound_delta is provided in
+ terms of seconds, and deadline_cron is optional.
+
+ For time-partitioned assets, the freshness check configuration mirrors that of the
+ `build_time_partition_freshness_checks()` function.
+
+ Below is example of configuring a non-time-partitioned dbt asset with a freshness check.
+ This code would be placed in the schema.yml file for the dbt model.
+
+ ```YAML
+ models:
+ - name: customers
+ ...
+ meta:
+ dagster:
+ freshness_check:
+ lower_bound_delta_seconds: 86400 # 1 day
+ deadline_cron: "0 0 * * *" # Optional
+ severity: "WARN" # Optional, defaults to "WARN"
+ ```
+ Below is an example of configuring a time-partitioned dbt asset with a freshness check.
+ This code would be placed in the schema.yml file for the dbt model.
+
+ ```yaml
+ models:
+ - name: customers
+ ...
+ meta:
+ dagster:
+ freshness_check:
+ deadline_cron: "0 0 * * *"
+ severity: "WARN" # Optional, defaults to "WARN"
+ ```
+ Parameters: dbt_assets (Sequence[[*AssetsDefinition*](../assets.mdx#dagster.AssetsDefinition)]) – A sequence of dbt assets to construct freshness
+ checks from.Returns:
+ A sequence of asset checks definitions representing the
+ freshness checks for the provided dbt assets.
+
+ Return type: Sequence[[AssetChecksDefinition](../asset-checks.mdx#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
+
+
+
+
+### Resources (dbt Core)
+
+
+
+
+#### CLI Resource
+
+
+
class dagster_dbt.DbtCliResource
+
+
+ A resource used to execute dbt CLI commands.
+
+
+
project_dir
+
+
+ The path to the dbt project directory. This directory should contain a
+ dbt_project.yml. See [https://docs.getdbt.com/reference/dbt_project.yml](https://docs.getdbt.com/reference/dbt_project.yml) for more
+ information.
+
+ Type: str
+
+
+
+
+
+
global_config_flags
+
+
+ A list of global flags configuration to pass to the dbt CLI
+ invocation. Invoke dbt –help to see a full list of global flags.
+
+ Type: List[str]
+
+
+
+
+
+
profiles_dir
+
+
+ The path to the directory containing your dbt profiles.yml.
+ By default, the current working directory is used, which is the dbt project directory.
+ See [https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles](https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles) for more
+ information.
+
+ Type: Optional[str]
+
+
+
+
+
+
profile
+
+
+ The profile from your dbt profiles.yml to use for execution. See
+ [https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles](https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles) for more
+ information.
+
+ Type: Optional[str]
+
+
+
+
+
+
target
+
+
+ The target from your dbt profiles.yml to use for execution. See
+ [https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles](https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles) for more
+ information.
+
+ Type: Optional[str]
+
+
+
+
+
+
dbt_executable
+
+
+ The path to the dbt executable. By default, this is dbt.
+
+ Type: str
+
+
+
+
+
+
state_path
+
+
+ The path, relative to the project directory, to a directory of
+ dbt artifacts to be used with –state / –defer-state.
+
+ Type: Optional[str]
+
+
+
+
+ Examples:
+
+ Creating a dbt resource with only a reference to `project_dir`:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(project_dir="/path/to/dbt/project")
+ ```
+ Creating a dbt resource with a custom `profiles_dir`:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(
+ project_dir="/path/to/dbt/project",
+ profiles_dir="/path/to/dbt/project/profiles",
+ )
+ ```
+ Creating a dbt resource with a custom `profile` and `target`:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(
+ project_dir="/path/to/dbt/project",
+ profiles_dir="/path/to/dbt/project/profiles",
+ profile="jaffle_shop",
+ target="dev",
+ )
+ ```
+ Creating a dbt resource with global configs, e.g. disabling colored logs with `--no-use-color`:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(
+ project_dir="/path/to/dbt/project",
+ global_config_flags=["--no-use-color"],
+ )
+ ```
+ Creating a dbt resource with custom dbt executable path:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(
+ project_dir="/path/to/dbt/project",
+ dbt_executable="/path/to/dbt/executable",
+ )
+ ```
+
+
cli
+
+
+ Create a subprocess to execute a dbt CLI command.
+
+ Parameters:
+ - args (Sequence[str]) – The dbt CLI command to execute.
+ - raise_on_error (bool) – Whether to raise an exception if the dbt CLI command fails.
+ - manifest (Optional[Union[Mapping[str, Any], str, Path]]) – The dbt manifest blob. If an
+ - dagster_dbt_translator (Optional[[*DagsterDbtTranslator*](#dagster_dbt.DagsterDbtTranslator)]) – The translator to link dbt
+ - context (Optional[Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]]) – The execution context from within @dbt_assets.
+ - target_path (Optional[Path]) – An explicit path to a target folder to use to store and
+
+
+ Returns:
+ A invocation instance that can be used to retrieve the output of the
+ dbt CLI command.
+
+ Return type: [DbtCliInvocation](#dagster_dbt.DbtCliInvocation)
+ Examples:
+
+ Streaming Dagster events for dbt asset materializations and observations:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ yield from dbt.cli(["run"], context=context).stream()
+ ```
+ Retrieving a dbt artifact after streaming the Dagster events:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_run_invocation = dbt.cli(["run"], context=context)
+
+ yield from dbt_run_invocation.stream()
+
+ # Retrieve the `run_results.json` dbt artifact as a dictionary:
+ run_results_json = dbt_run_invocation.get_artifact("run_results.json")
+
+ # Retrieve the `run_results.json` dbt artifact as a file path:
+ run_results_path = dbt_run_invocation.target_path.joinpath("run_results.json")
+ ```
+ Customizing the asset materialization metadata when streaming the Dagster events:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_cli_invocation = dbt.cli(["run"], context=context)
+
+ for dagster_event in dbt_cli_invocation.stream():
+ if isinstance(dagster_event, Output):
+ context.add_output_metadata(
+ metadata={
+ "my_custom_metadata": "my_custom_metadata_value",
+ },
+ output_name=dagster_event.output_name,
+ )
+
+ yield dagster_event
+ ```
+ Suppressing exceptions from a dbt CLI command when a non-zero exit code is returned:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_run_invocation = dbt.cli(["run"], context=context, raise_on_error=False)
+
+ if dbt_run_invocation.is_successful():
+ yield from dbt_run_invocation.stream()
+ else:
+ ...
+ ```
+ Invoking a dbt CLI command in a custom asset or op:
+
+ ```python
+ import json
+
+ from dagster import Nothing, Out, asset, op
+ from dagster_dbt import DbtCliResource
+
+
+ @asset
+ def my_dbt_asset(dbt: DbtCliResource):
+ dbt_macro_args = {"key": "value"}
+ dbt.cli(["run-operation", "my-macro", json.dumps(dbt_macro_args)]).wait()
+
+
+ @op(out=Out(Nothing))
+ def my_dbt_op(dbt: DbtCliResource):
+ dbt_macro_args = {"key": "value"}
+ yield from dbt.cli(["run-operation", "my-macro", json.dumps(dbt_macro_args)]).stream()
+ ```
+
+
+
+
+
+
get_defer_args
+
+
+ Build the defer arguments for the dbt CLI command, using the supplied state directory.
+ If no state directory is supplied, or the state directory does not have a manifest for.
+ comparison, an empty list of arguments is returned.
+
+ Returns: The defer arguments for the dbt CLI command.Return type: Sequence[str]
+
+
+
+
+
+
get_state_args
+
+
+ Build the state arguments for the dbt CLI command, using the supplied state directory.
+ If no state directory is supplied, or the state directory does not have a manifest for.
+ comparison, an empty list of arguments is returned.
+
+ Returns: The state arguments for the dbt CLI command.Return type: Sequence[str]
+
+
+
+
+
+
+
+
+
+
class dagster_dbt.DbtCliInvocation
+
+
+ The representation of an invoked dbt command.
+
+ Parameters:
+ - process (subprocess.Popen) – The process running the dbt command.
+ - manifest (Mapping[str, Any]) – The dbt manifest blob.
+ - project_dir (Path) – The path to the dbt project.
+ - target_path (Path) – The path to the dbt target folder.
+ - raise_on_error (bool) – Whether to raise an exception if the dbt command fails.
+
+
+
+
get_artifact
+
+
+ Retrieve a dbt artifact from the target path.
+
+ See [https://docs.getdbt.com/reference/artifacts/dbt-artifacts](https://docs.getdbt.com/reference/artifacts/dbt-artifacts) for more information.
+
+ Parameters: artifact (Union[Literal["manifest.json"], Literal["catalog.json"], Literal["run_results.json"], Literal["sources.json"]]) – The name of the artifact to retrieve.Returns: The artifact as a dictionary.Return type: Dict[str, Any]
+ Examples:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(project_dir="/path/to/dbt/project")
+
+ dbt_cli_invocation = dbt.cli(["run"]).wait()
+
+ # Retrieve the run_results.json artifact.
+ run_results = dbt_cli_invocation.get_artifact("run_results.json")
+ ```
+
+
+
+
+
+
get_error
+
+
+ Return an exception if the dbt CLI process failed.
+
+ Returns: An exception if the dbt CLI process failed, and None otherwise.Return type: Optional[Exception]
+ Examples:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(project_dir="/path/to/dbt/project")
+
+ dbt_cli_invocation = dbt.cli(["run"], raise_on_error=False)
+
+ error = dbt_cli_invocation.get_error()
+ if error:
+ logger.error(error)
+ ```
+
+
+
+
+
+
is_successful
+
+
+ Return whether the dbt CLI process completed successfully.
+
+ Returns: True, if the dbt CLI process returns with a zero exit code, and False otherwise.Return type: bool
+ Examples:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(project_dir="/path/to/dbt/project")
+
+ dbt_cli_invocation = dbt.cli(["run"], raise_on_error=False)
+
+ if dbt_cli_invocation.is_successful():
+ ...
+ ```
+
+
+
+
+
+
stream
+
+
+ Stream the events from the dbt CLI process and convert them to Dagster events.
+
+ Returns:
+ A set of corresponding Dagster events.
+
+ In a Dagster asset definition, the following are yielded:
+ - Output for refables (e.g. models, seeds, snapshots.)
+ - AssetCheckResult for dbt test results that are enabled as asset checks.
+ - AssetObservation for dbt test results that are not enabled as asset checks.
+
+ In a Dagster op definition, the following are yielded:
+ - AssetMaterialization for dbt test results that are not enabled as asset checks.
+ - AssetObservation for dbt test results.
+
+ Return type: Iterator[Union[[Output](../ops.mdx#dagster.Output), [AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [AssetObservation](../assets.mdx#dagster.AssetObservation), [AssetCheckResult](../asset-checks.mdx#dagster.AssetCheckResult)]]
+ Examples:
+
+ ```python
+ from pathlib import Path
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context, dbt: DbtCliResource):
+ yield from dbt.cli(["run"], context=context).stream()
+ ```
+
+
+
+
+
+
stream_raw_events
+
+
+ Stream the events from the dbt CLI process.
+
+ Returns: An iterator of events from the dbt CLI process.Return type: Iterator[[DbtCliEventMessage](#dagster_dbt.DbtCliEventMessage)]
+
+
+
+
+
+
wait
+
+
+ Wait for the dbt CLI process to complete.
+
+ Returns: The current representation of the dbt CLI invocation.Return type: [DbtCliInvocation](#dagster_dbt.DbtCliInvocation)
+ Examples:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(project_dir="/path/to/dbt/project")
+
+ dbt_cli_invocation = dbt.cli(["run"]).wait()
+ ```
+
+
+
+
+
+
+
+
+
+
class dagster_dbt.core.dbt_cli_invocation.DbtEventIterator
+
+
+ A wrapper around an iterator of dbt events which contains additional methods for
+ post-processing the events, such as fetching row counts for materialized tables.
+
+
+
fetch_column_metadata
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Experimental functionality which will fetch column schema metadata for dbt models in a run
+ once they’re built. It will also fetch schema information for upstream models and generate
+ column lineage metadata using sqlglot, if enabled.
+
+ Parameters: generate_column_lineage (bool) – Whether to generate column lineage metadata using sqlglot.Returns: A set of corresponding Dagster events for dbt models, with column metadata attached,
+ yielded in the order they are emitted by dbt.Return type: Iterator[Union[[Output](../ops.mdx#dagster.Output), [AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [AssetObservation](../assets.mdx#dagster.AssetObservation), [AssetCheckResult](../asset-checks.mdx#dagster.AssetCheckResult)]]
+
+
+
+
+
+
fetch_row_counts
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Experimental functionality which will fetch row counts for materialized dbt
+ models in a dbt run once they are built. Note that row counts will not be fetched
+ for views, since this requires running the view’s SQL query which may be costly.
+
+ Returns: A set of corresponding Dagster events for dbt models, with row counts attached,
+ yielded in the order they are emitted by dbt.Return type: Iterator[Union[[Output](../ops.mdx#dagster.Output), [AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [AssetObservation](../assets.mdx#dagster.AssetObservation), [AssetCheckResult](../asset-checks.mdx#dagster.AssetCheckResult)]]
+
+
+
+
+
+
with_insights
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Associate each warehouse query with the produced asset materializations for use in Dagster
+ Plus Insights. Currently supports Snowflake and BigQuery.
+
+ For more information, see the documentation for
+ dagster_cloud.dagster_insights.dbt_with_snowflake_insights and
+ dagster_cloud.dagster_insights.dbt_with_bigquery_insights.
+
+ Parameters:
+ - skip_config_check (bool) – If true, skips the check that the dbt project config is set up
+ - record_observation_usage (bool) – If True, associates the usage associated with
+
+
+ Example:
+
+ ```python
+ @dbt_assets(manifest=DBT_MANIFEST_PATH)
+ def jaffle_shop_dbt_assets(
+ context: AssetExecutionContext,
+ dbt: DbtCliResource,
+ ):
+ yield from dbt.cli(["build"], context=context).stream().with_insights()
+ ```
+
+
+
+
+
+
+
+
+
+
class dagster_dbt.DbtCliEventMessage
+
+
+ The representation of a dbt CLI event.
+
+ Parameters:
+ - raw_event (Dict[str, Any]) – The raw event dictionary.
+ - event_history_metadata (Dict[str, Any]) – A dictionary of metadata about the
+
+
+
+
to_default_asset_events
+
+
+ Convert a dbt CLI event to a set of corresponding Dagster events.
+
+ Parameters:
+ - manifest (Union[Mapping[str, Any], str, Path]) – The dbt manifest blob.
+ - dagster_dbt_translator ([*DagsterDbtTranslator*](#dagster_dbt.DagsterDbtTranslator)) – Optionally, a custom translator for
+ - context (Optional[Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]]) – The execution context.
+ - target_path (Optional[Path]) – An explicit path to a target folder used to retrieve
+
+
+ Returns:
+ A set of corresponding Dagster events.
+
+ In a Dagster asset definition, the following are yielded:
+ - Output for refables (e.g. models, seeds, snapshots.)
+ - AssetCheckResult for dbt test results that are enabled as asset checks.
+ - AssetObservation for dbt test results that are not enabled as asset checks.
+
+ In a Dagster op definition, the following are yielded:
+ - AssetMaterialization for dbt test results that are not enabled as asset checks.
+ - AssetObservation for dbt test results.
+
+ Return type: Iterator[Union[[Output](../ops.mdx#dagster.Output), [AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [AssetObservation](../assets.mdx#dagster.AssetObservation), [AssetCheckResult](../asset-checks.mdx#dagster.AssetCheckResult)]]
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## dbt Cloud
+
+Here, we provide interfaces to manage dbt projects invoked by the hosted dbt Cloud service.
+
+
+
+
+### Assets (dbt Cloud)
+
+
+
dagster_dbt.load_assets_from_dbt_cloud_job
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Loads a set of dbt models, managed by a dbt Cloud job, into Dagster assets. In order to
+ determine the set of dbt models, the project is compiled to generate the necessary artifacts
+ that define the dbt models and their dependencies.
+
+ One Dagster asset is created for each dbt model.
+
+ Parameters:
+ - dbt_cloud ([*ResourceDefinition*](../resources.mdx#dagster.ResourceDefinition)) – The dbt Cloud resource to use to connect to the dbt Cloud API.
+ - job_id (int) – The ID of the dbt Cloud job to load assets from.
+ - node_info_to_asset_key – (Mapping[str, Any] -> AssetKey): A function that takes a dictionary
+ - node_info_to_group_fn (Dict[str, Any] -> Optional[str]) – A function that takes a
+ - node_info_to_freshness_policy_fn (Dict[str, Any] -> Optional[FreshnessPolicy]) – A function
+ - node_info_to_auto_materialize_policy_fn (Dict[str, Any] -> Optional[AutoMaterializePolicy]) – A function that takes a dictionary of dbt node info and optionally returns a AutoMaterializePolicy
+ - node_info_to_definition_metadata_fn (Dict[str, Any] -> Optional[Dict[str, RawMetadataMapping]]) – A function that takes a dictionary of dbt node info and optionally returns a dictionary
+ - partitions_def (Optional[[*PartitionsDefinition*](../partitions.mdx#dagster.PartitionsDefinition)]) – experimental
+ - partition_key_to_vars_fn (Optional[str -> Dict[str, Any]]) – experimental
+
+
+ Returns: A definition for the loaded assets.Return type: CacheableAssetsDefinition
+ Examples:
+
+ ```python
+ from dagster import repository
+ from dagster_dbt import dbt_cloud_resource, load_assets_from_dbt_cloud_job
+
+ DBT_CLOUD_JOB_ID = 1234
+
+ dbt_cloud = dbt_cloud_resource.configured(
+ {
+ "auth_token": {"env": "DBT_CLOUD_API_TOKEN"},
+ "account_id": {"env": "DBT_CLOUD_ACCOUNT_ID"},
+ }
+ )
+
+ dbt_cloud_assets = load_assets_from_dbt_cloud_job(
+ dbt_cloud=dbt_cloud, job_id=DBT_CLOUD_JOB_ID
+ )
+
+
+ @repository
+ def dbt_cloud_sandbox():
+ return [dbt_cloud_assets]
+ ```
+
+
+
+
+
+
+
+
+
+
+### Ops (dbt Cloud)
+
+
+
dagster_dbt.dbt_cloud_run_op `=` \
+
+
+
+
+ Initiates a run for a dbt Cloud job, then polls until the run completes. If the job
+ fails or is otherwised stopped before succeeding, a dagster.Failure exception will be raised,
+ and this op will fail.
+
+ It requires the use of a ‘dbt_cloud’ resource, which is used to connect to the dbt Cloud API.
+
+ Config Options:
+
+ job_id (int)
+ The integer ID of the relevant dbt Cloud job. You can find this value by going to the details
+ page of your job in the dbt Cloud UI. It will be the final number in the url, e.g.:
+ `https://cloud.getdbt.com/#/accounts/\{account_id}/projects/\{project_id}/jobs/\{job_id}/`
+
+ poll_interval (float)
+ The time (in seconds) that will be waited between successive polls. Defaults to `10`.
+
+ poll_timeout (float)
+ The maximum time (in seconds) that will waited before this operation is timed out. By
+ default, this will never time out.
+
+ yield_materializations (bool)
+ If True, materializations corresponding to the results of the dbt operation will be
+ yielded when the solid executes. Defaults to `True`.
+
+ rasset_key_prefix (float)
+ If provided and yield_materializations is True, these components will be used to ”
+ prefix the generated asset keys. Defaults to [“dbt”].
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_dbt import dbt_cloud_resource, dbt_cloud_run_op
+
+ my_dbt_cloud_resource = dbt_cloud_resource.configured(
+ {"auth_token": {"env": "DBT_CLOUD_AUTH_TOKEN"}, "account_id": 77777}
+ )
+ run_dbt_nightly_sync = dbt_cloud_run_op.configured(
+ {"job_id": 54321}, name="run_dbt_nightly_sync"
+ )
+
+ @job(resource_defs={"dbt_cloud": my_dbt_cloud_resource})
+ def dbt_cloud():
+ run_dbt_nightly_sync()
+ ```
+
+
+
+
+
+
+
+
+
+
+### Resources (dbt Cloud)
+
+
+
class dagster_dbt.DbtCloudClientResource
+
+ This resource helps interact with dbt Cloud connectors.
+
+
+
+
+
+
+#### Deprecated (dbt Cloud)
+
+
+
dagster_dbt.dbt_cloud_resource ResourceDefinition
+
+
+
+
+ This resource allows users to programatically interface with the dbt Cloud Administrative REST
+ API (v2) to launch jobs and monitor their progress. This currently implements only a subset of
+ the functionality exposed by the API.
+
+ For a complete set of documentation on the dbt Cloud Administrative REST API, including expected
+ response JSON schemae, see the [dbt Cloud API Docs](https://docs.getdbt.com/dbt-cloud/api-v2).
+
+ To configure this resource, we recommend using the [configured](https://legacy-docs.dagster.io/concepts/configuration/configured) method.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_dbt import dbt_cloud_resource
+
+ my_dbt_cloud_resource = dbt_cloud_resource.configured(
+ {
+ "auth_token": {"env": "DBT_CLOUD_AUTH_TOKEN"},
+ "account_id": {"env": "DBT_CLOUD_ACCOUNT_ID"},
+ }
+ )
+
+ @job(resource_defs={"dbt_cloud": my_dbt_cloud_resource})
+ def my_dbt_cloud_job():
+ ...
+ ```
+
+
+
+
+
+
+
+
+
+
+## Errors
+
+
+
exception dagster_dbt.DagsterDbtError
+
+ The base exception of the `dagster-dbt` library.
+
+
+
+
+
exception dagster_dbt.DagsterDbtCliRuntimeError
+
+ Represents an error while executing a dbt CLI command.
+
+
+
+
+
+
+
+
+## Utils
+
+
+
dagster_dbt.default_group_from_dbt_resource_props
+
+
+ Get the group name for a dbt node.
+
+ If a Dagster group is configured in the metadata for the node, use that.
+
+ Otherwise, if a dbt group is configured for the node, use that.
+
+
+
+
+ Get the group name for a dbt node.
+
+ Has the same behavior as the default_group_from_dbt_resource_props, except for that, if no group can be determined
+ from config or metadata, falls back to using the subdirectory of the models directory that the
+ source file is in.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to Delta Lake.
+
+ Examples:
+
+ ```python
+ from dagster_deltalake import DeltaLakeIOManager
+ from dagster_deltalake_pandas import DeltaLakePandasTypeHandler
+
+ class MyDeltaLakeIOManager(DeltaLakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DeltaLakePandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema (parent folder) in Delta Lake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDeltaLakeIOManager()}
+ )
+ ```
+ If you do not provide a schema, Dagster will determine a schema based on the assets and ops using
+ the I/O Manager. For assets, the schema will be determined from the asset key, as in the above example.
+ For ops, the schema can be specified by including a “schema” entry in output metadata. If none
+ of these is provided, the schema will default to “public”.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_deltalake_pandas.DeltaLakePandasTypeHandler
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-deltalake-polars.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-deltalake-polars.mdx
new file mode 100644
index 0000000000000..0133088e962ea
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-deltalake-polars.mdx
@@ -0,0 +1,85 @@
+---
+title: 'deltalake + polars (dagster-deltalake-polars)'
+title_meta: 'deltalake + polars (dagster-deltalake-polars) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'deltalake + polars (dagster-deltalake-polars) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Deltalake + Polars (dagster-deltalake-polars)
+
+This library provides an integration with the [Delta Lake](https://delta.io/) storage framework.
+
+Related guides:
+
+ - [Using Dagster with Delta Lake guide](https://docs.dagster.io/integrations/libraries/deltalake/)
+ - [DeltaLake I/O manager reference](https://docs.dagster.io/integrations/libraries/deltalake/reference)
+
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to Delta Lake.
+
+ Examples:
+
+ ```python
+ from dagster_deltalake import DeltaLakeIOManager
+ from dagster_deltalake_pandas import DeltaLakePandasTypeHandler
+
+ class MyDeltaLakeIOManager(DeltaLakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DeltaLakePandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema (parent folder) in Delta Lake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDeltaLakeIOManager()}
+ )
+ ```
+ If you do not provide a schema, Dagster will determine a schema based on the assets and ops using
+ the I/O Manager. For assets, the schema will be determined from the asset key, as in the above example.
+ For ops, the schema can be specified by including a “schema” entry in output metadata. If none
+ of these is provided, the schema will default to “public”.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_deltalake_polars.DeltaLakePolarsTypeHandler
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-deltalake.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-deltalake.mdx
new file mode 100644
index 0000000000000..9b97c8e2767ca
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-deltalake.mdx
@@ -0,0 +1,167 @@
+---
+title: 'delta lake (dagster-deltalake)'
+title_meta: 'delta lake (dagster-deltalake) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'delta lake (dagster-deltalake) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Delta Lake (dagster-deltalake)
+
+This library provides an integration with the [Delta Lake](https://delta.io/) storage framework.
+
+Related Guides:
+
+ - [Using Dagster with Delta Lake tutorial](https://docs.dagster.io/integrations/libraries/deltalake/)
+ - [Delta Lake reference](https://docs.dagster.io/integrations/libraries/deltalake/reference)
+
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to Delta Lake.
+
+ Examples:
+
+ ```python
+ from dagster_deltalake import DeltaLakeIOManager
+ from dagster_deltalake_pandas import DeltaLakePandasTypeHandler
+
+ class MyDeltaLakeIOManager(DeltaLakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DeltaLakePandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema (parent folder) in Delta Lake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDeltaLakeIOManager()}
+ )
+ ```
+ If you do not provide a schema, Dagster will determine a schema based on the assets and ops using
+ the I/O Manager. For assets, the schema will be determined from the asset key, as in the above example.
+ For ops, the schema can be specified by including a “schema” entry in output metadata. If none
+ of these is provided, the schema will default to “public”.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to Delta Lake.
+
+ Examples:
+
+ ```python
+ from dagster_deltalake import DeltaLakeIOManager
+ from dagster_deltalake_pandas import DeltaLakePandasTypeHandler
+
+ class MyDeltaLakeIOManager(DeltaLakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DeltaLakePandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema (parent folder) in Delta Lake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDeltaLakeIOManager()}
+ )
+ ```
+ If you do not provide a schema, Dagster will determine a schema based on the assets and ops using
+ the I/O Manager. For assets, the schema will be determined from the asset key, as in the above example.
+ For ops, the schema can be specified by including a “schema” entry in output metadata. If none
+ of these is provided, the schema will default to “public”.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-dlt.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-dlt.mdx
new file mode 100644
index 0000000000000..9f36d071c8937
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-dlt.mdx
@@ -0,0 +1,359 @@
+---
+title: 'dlt (dagster-dlt)'
+title_meta: 'dlt (dagster-dlt) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dlt (dagster-dlt) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# dlt (dagster-dlt)
+
+This library provides a Dagster integration with [dlt](https://dlthub.com).
+
+For more information on getting started, see the [dlt & Dagster](https://docs.dagster.io/integrations/libraries/dlt/) documentation.
+
+
+
+
+## Assets
+
+
+
@dagster_dlt.dlt_assets
+
+
+ Asset Factory for using data load tool (dlt).
+
+ Parameters:
+ - dlt_source (DltSource) – The DltSource to be ingested.
+ - dlt_pipeline (Pipeline) – The dlt Pipeline defining the destination parameters.
+ - name (Optional[str], optional) – The name of the op.
+ - group_name (Optional[str], optional) – The name of the asset group.
+ - dagster_dlt_translator ([*DagsterDltTranslator*](#dagster_dlt.DagsterDltTranslator), optional) – Customization object for defining asset parameters from dlt resources.
+ - partitions_def (Optional[[*PartitionsDefinition*](../partitions.mdx#dagster.PartitionsDefinition)]) – Optional partitions definition.
+ - op_tags (Optional[Mapping[str, Any]]) – The tags for the underlying op.
+
+
+ Examples:
+
+ Loading Hubspot data to Snowflake with an auto materialize policy using the dlt verified source:
+
+ ```python
+ from dagster_dlt import DagsterDltResource, DagsterDltTranslator, dlt_assets
+
+
+ class HubspotDagsterDltTranslator(DagsterDltTranslator):
+ @public
+ def get_auto_materialize_policy(self, resource: DltResource) -> Optional[AutoMaterializePolicy]:
+ return AutoMaterializePolicy.eager().with_rules(
+ AutoMaterializeRule.materialize_on_cron("0 0 * * *")
+ )
+
+
+ @dlt_assets(
+ dlt_source=hubspot(include_history=True),
+ dlt_pipeline=pipeline(
+ pipeline_name="hubspot",
+ dataset_name="hubspot",
+ destination="snowflake",
+ progress="log",
+ ),
+ name="hubspot",
+ group_name="hubspot",
+ dagster_dlt_translator=HubspotDagsterDltTranslator(),
+ )
+ def hubspot_assets(context: AssetExecutionContext, dlt: DagsterDltResource):
+ yield from dlt.run(context=context)
+ ```
+ Loading Github issues to snowflake:
+
+ ```python
+ from dagster_dlt import DagsterDltResource, dlt_assets
+
+
+ @dlt_assets(
+ dlt_source=github_reactions(
+ "dagster-io", "dagster", items_per_page=100, max_items=250
+ ),
+ dlt_pipeline=pipeline(
+ pipeline_name="github_issues",
+ dataset_name="github",
+ destination="snowflake",
+ progress="log",
+ ),
+ name="github",
+ group_name="github",
+ )
+ def github_reactions_dagster_assets(context: AssetExecutionContext, dlt: DagsterDltResource):
+ yield from dlt.run(context=context)
+ ```
+
+
+
+
+
+
dagster_dlt.build_dlt_asset_specs
+
+
+ Build a list of asset specs from a dlt source and pipeline.
+
+ Parameters:
+ - dlt_source (DltSource) – dlt source object
+ - dlt_pipeline (Pipeline) – dlt pipeline object
+ - dagster_dlt_translator (Optional[[*DagsterDltTranslator*](#dagster_dlt.DagsterDltTranslator)]) – Allows customizing how to
+
+
+ Returns: List[AssetSpec] list of asset specs from dlt source and pipeline
+
+
+
+
+
+
class dagster_dlt.DagsterDltTranslator
+
+
+
+
get_asset_key
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).key` instead..
+
+ :::
+
+ Defines asset key for a given dlt resource key and dataset name.
+
+ This method can be overridden to provide custom asset key for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: AssetKey of Dagster asset derived from dlt resource
+
+
+
+
+
+
get_auto_materialize_policy
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).auto_materialize_policy` instead..
+
+ :::
+
+ Defines resource specific auto materialize policy.
+
+ This method can be overridden to provide custom auto materialize policy for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: The auto-materialize policy for a resourceReturn type: Optional[AutoMaterializePolicy]
+
+
+
+
+
+
get_automation_condition
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).automation_condition` instead..
+
+ :::
+
+ Defines resource specific automation condition.
+
+ This method can be overridden to provide custom automation condition for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: The automation condition for a resourceReturn type: Optional[[AutomationCondition](../assets.mdx#dagster.AutomationCondition)]
+
+
+
+
+
+
get_deps_asset_keys
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Iterate over `DagsterDltTranslator.get_asset_spec(...).deps` to access `AssetDep.asset_key` instead..
+
+ :::
+
+ Defines upstream asset dependencies given a dlt resource.
+
+ Defaults to a concatenation of resource.source_name and resource.name.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: The Dagster asset keys upstream of dlt_resource_key.Return type: Iterable[[AssetKey](../assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
get_description
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).description` instead..
+
+ :::
+
+ A method that takes in a dlt resource returns the Dagster description of the resource.
+
+ This method can be overridden to provide a custom description for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: The Dagster description for the dlt resource.Return type: Optional[str]
+
+
+
+
+
+
get_group_name
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).group_name` instead..
+
+ :::
+
+ A method that takes in a dlt resource and returns the Dagster group name of the resource.
+
+ This method can be overridden to provide a custom group name for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: A Dagster group name for the dlt resource.Return type: Optional[str]
+
+
+
+
+
+
get_kinds
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).kinds` instead..
+
+ :::
+
+ A method that takes in a dlt resource and returns the kinds which should be
+ attached. Defaults to the destination type and “dlt”.
+
+ This method can be overridden to provide custom kinds for a dlt resource.
+
+ Parameters:
+ - resource (DltResource) – dlt resource
+ - destination (Destination) – dlt destination
+
+
+ Returns: The kinds of the asset.Return type: Set[str]
+
+
+
+
+
+
get_metadata
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).metadata` instead..
+
+ :::
+
+ Defines resource specific metadata.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: The custom metadata entries for this resource.Return type: Mapping[str, Any]
+
+
+
+
+
+
get_owners
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).owners` instead..
+
+ :::
+
+ A method that takes in a dlt resource and returns the Dagster owners of the resource.
+
+ This method can be overridden to provide custom owners for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: A sequence of Dagster owners for the dlt resource.Return type: Optional[Sequence[str]]
+
+
+
+
+
+
get_tags
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).tags` instead..
+
+ :::
+
+ A method that takes in a dlt resource and returns the Dagster tags of the structure.
+
+ This method can be overridden to provide custom tags for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns:
+ A dictionary representing the Dagster tags for the
+ dlt resource.
+
+ Return type: Optional[Mapping[str, str]]
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Resources
+
+
+
class dagster_dlt.DagsterDltResource
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+
+
run
+
+
+ Runs the dlt pipeline with subset support.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – Asset or op execution context
+ - dlt_source (Optional[DltSource]) – optional dlt source if resource is used from an @op
+ - dlt_pipeline (Optional[Pipeline]) – optional dlt pipeline if resource is used from an @op
+ - dagster_dlt_translator (Optional[[*DagsterDltTranslator*](#dagster_dlt.DagsterDltTranslator)]) – optional dlt translator if resource is used from an @op
+ - **kwargs (dict[str, Any]) – Keyword args passed to pipeline run method
+
+
+ Returns: An iterator of MaterializeResult or AssetMaterializationReturn type: DltEventIterator[DltEventType]
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-docker.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-docker.mdx
new file mode 100644
index 0000000000000..7747673692f4a
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-docker.mdx
@@ -0,0 +1,209 @@
+---
+title: 'orchestration on docker'
+title_meta: 'orchestration on docker API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'orchestration on docker Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Orchestration on Docker
+
+
+
+
+
+
+
+# APIs
+
+
+
dagster_docker.DockerRunLauncher RunLauncher
+
+
+
+
+ Launches runs in a Docker container.
+
+
+
+
+
+
+
dagster_docker.docker_executor ExecutorDefinition
+
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Executor which launches steps as Docker containers.
+
+ To use the docker_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster_docker import docker_executor
+
+ from dagster import job
+
+ @job(executor_def=docker_executor)
+ def docker_job():
+ pass
+ ```
+ Then you can configure the executor with run config as follows:
+
+ ```YAML
+ execution:
+ config:
+ registry: ...
+ network: ...
+ networks: ...
+ container_kwargs: ...
+ ```
+ If you’re using the DockerRunLauncher, configuration set on the containers created by the run
+ launcher will also be set on the containers that are created for each step.
+
+
+
+
+
+
+
+
+## Ops
+
+
+
dagster_docker.docker_container_op `=` \
+
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ An op that runs a Docker container using the docker Python API.
+
+ Contrast with the docker_executor, which runs each Dagster op in a Dagster job in its
+ own Docker container.
+
+ This op may be useful when:
+ - You need to orchestrate a command that isn’t a Dagster op (or isn’t written in Python)
+ - You want to run the rest of a Dagster job using a specific executor, and only a single
+
+
+ For example:
+
+ ```python
+ from dagster_docker import docker_container_op
+
+ from dagster import job
+
+ first_op = docker_container_op.configured(
+ {
+ "image": "busybox",
+ "command": ["echo HELLO"],
+ },
+ name="first_op",
+ )
+ second_op = docker_container_op.configured(
+ {
+ "image": "busybox",
+ "command": ["echo GOODBYE"],
+ },
+ name="second_op",
+ )
+
+ @job
+ def full_job():
+ second_op(first_op())
+ ```
+ You can create your own op with the same implementation by calling the execute_docker_container function
+ inside your own op.
+
+
+
+
+
+
+
dagster_docker.execute_docker_container
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This function is a utility for executing a Docker container from within a Dagster op.
+
+ Parameters:
+ - image (str) – The image to use for the launched Docker container.
+ - entrypoint (Optional[Sequence[str]]) – The ENTRYPOINT to run in the launched Docker
+ - command (Optional[Sequence[str]]) – The CMD to run in the launched Docker container.
+ - networks (Optional[Sequence[str]]) – Names of the Docker networks to which to connect the
+ - registry – (Optional[Mapping[str, str]]): Information for using a non local/public Docker
+ - env_vars (Optional[Sequence[str]]) – List of environemnt variables to include in the launched
+ - container_kwargs (Optional[Dict[str[Any]]]) – key-value pairs that can be passed into
+
+
+
+
+
+
+
+
+
+
+
+## Pipes
+
+
+
class dagster_docker.PipesDockerClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client that runs external processes in docker containers.
+
+ By default context is injected via environment variables and messages are parsed out of the
+ log stream, with other logs forwarded to stdout of the orchestration process.
+
+ Parameters:
+ - env (Optional[Mapping[str, str]]) – An optional dict of environment variables to pass to the
+ - register (Optional[Mapping[str, str]]) – An optional dict of registry credentials to login to
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+
+
+
+
+
+
+
+
class dagster_docker.PipesDockerLogsMessageReader
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb-pandas.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb-pandas.mdx
new file mode 100644
index 0000000000000..a5f39aca185f0
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb-pandas.mdx
@@ -0,0 +1,218 @@
+---
+title: 'duckdb + pandas (dagster-duckdb-pandas)'
+title_meta: 'duckdb + pandas (dagster-duckdb-pandas) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'duckdb + pandas (dagster-duckdb-pandas) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# DuckDB + Pandas (dagster-duckdb-pandas)
+
+This library provides an integration with the [DuckDB](https://duckdb.org/) database and Pandas data processing library.
+
+Related guides:
+
+ - [Using Dagster with DuckDB guide](https://docs.dagster.io/integrations/libraries/duckdb/)
+ - [DuckDB I/O manager reference](https://docs.dagster.io/integrations/libraries/duckdb/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes Pandas DataFrames to DuckDB. When
+ using the DuckDBPandasIOManager, any inputs and outputs without type annotations will be loaded
+ as Pandas DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_pandas import DuckDBPandasIOManager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPandasIOManager(database="my_db.duckdb")}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPandasIOManager(database="my_db.duckdb", schema="my_schema")}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_duckdb_pandas.DuckDBPandasTypeHandler
+
+
+ Stores and loads Pandas DataFrames in DuckDB.
+
+ To use this type handler, return it from the `type_handlers` method of an I/O manager that inherits from ``DuckDBIOManager`.
+
+ Example:
+
+ ```python
+ from dagster_duckdb import DuckDBIOManager
+ from dagster_duckdb_pandas import DuckDBPandasTypeHandler
+
+ class MyDuckDBIOManager(DuckDBIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DuckDBPandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb")}
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes Pandas DataFrames to DuckDB. When
+ using the duckdb_pandas_io_manager, any inputs and outputs without type annotations will be loaded
+ as Pandas DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_pandas import duckdb_pandas_io_manager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_pandas_io_manager.configured({"database": "my_db.duckdb"})}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_pandas_io_manager.configured({"database": "my_db.duckdb", "schema": "my_schema"})}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb-polars.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb-polars.mdx
new file mode 100644
index 0000000000000..ca93e54436e7f
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb-polars.mdx
@@ -0,0 +1,221 @@
+---
+title: 'duckdb + polars (dagster-duckdb-polars)'
+title_meta: 'duckdb + polars (dagster-duckdb-polars) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'duckdb + polars (dagster-duckdb-polars) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+# DuckDB + Polars (dagster-duckdb-polars)
+
+This library provides an integration with the [DuckDB](https://duckdb.org/) database and Polars data processing library.
+
+Related guides:
+
+ - [Using Dagster with DuckDB guide](https://docs.dagster.io/integrations/libraries/duckdb/)
+ - [DuckDB I/O manager reference](https://docs.dagster.io/integrations/libraries/duckdb/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes Polars DataFrames to DuckDB. When
+ using the DuckDBPolarsIOManager, any inputs and outputs without type annotations will be loaded
+ as Polars DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_polars import DuckDBPolarsIOManager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pl.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPolarsIOManager(database="my_db.duckdb")}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPolarsIOManager(database="my_db.duckdb", schema="my_schema")}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pl.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pl.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pl.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ >
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pl.DataFrame) -> pl.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
+
+
class dagster_duckdb_polars.DuckDBPolarsTypeHandler
+
+
+ Stores and loads Polars DataFrames in DuckDB.
+
+ To use this type handler, return it from the `type_handlers` method of an I/O manager that inherits from ``DuckDBIOManager`.
+
+ Example:
+
+ ```python
+ from dagster_duckdb import DuckDBIOManager
+ from dagster_duckdb_polars import DuckDBPolarsTypeHandler
+
+ class MyDuckDBIOManager(DuckDBIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DuckDBPolarsTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pl.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb")}
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes polars dataframes to DuckDB. When
+ using the duckdb_polars_io_manager, any inputs and outputs without type annotations will be loaded
+ as Polars DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_polars import duckdb_polars_io_manager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pl.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_polars_io_manager.configured({"database": "my_db.duckdb"})}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_polars_io_manager.configured({"database": "my_db.duckdb", "schema": "my_schema"})}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pl.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pl.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pl.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pl.DataFrame) -> pl.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb-pyspark.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb-pyspark.mdx
new file mode 100644
index 0000000000000..8248340b46909
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb-pyspark.mdx
@@ -0,0 +1,218 @@
+---
+title: 'duckdb + pyspark (dagster-duckdb-pyspark)'
+title_meta: 'duckdb + pyspark (dagster-duckdb-pyspark) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'duckdb + pyspark (dagster-duckdb-pyspark) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# DuckDB + PySpark (dagster-duckdb-pyspark)
+
+This library provides an integration with the [DuckDB](https://duckdb.org/) database and PySpark data processing library.
+
+Related guides:
+
+ - [Using Dagster with DuckDB guide](https://docs.dagster.io/integrations/libraries/duckdb/)
+ - [DuckDB I/O manager reference](https://docs.dagster.io/integrations/libraries/duckdb/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to DuckDB. When
+ using the DuckDBPySparkIOManager, any inputs and outputs without type annotations will be loaded
+ as PySpark DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_pyspark import DuckDBPySparkIOManager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pyspark.sql.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPySparkIOManager(database="my_db.duckdb")}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPySparkIOManager(database="my_db.duckdb", schema="my_schema")}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pyspark.sql.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pyspark.sql.DataFrame) -> pyspark.sql.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_duckdb_pyspark.DuckDBPySparkTypeHandler
+
+
+ Stores PySpark DataFrames in DuckDB.
+
+ To use this type handler, return it from the `type_handlers` method of an I/O manager that inherits from ``DuckDBIOManager`.
+
+ Example:
+
+ ```python
+ from dagster_duckdb import DuckDBIOManager
+ from dagster_duckdb_pyspark import DuckDBPySparkTypeHandler
+
+ class MyDuckDBIOManager(DuckDBIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DuckDBPySparkTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pyspark.sql.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb")}
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to DuckDB. When
+ using the duckdb_pyspark_io_manager, any inputs and outputs without type annotations will be loaded
+ as PySpark DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_pyspark import duckdb_pyspark_io_manager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pyspark.sql.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_pyspark_io_manager.configured({"database": "my_db.duckdb"})}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_pyspark_io_manager.configured({"database": "my_db.duckdb", "schema": "my_schema"})}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pyspark.sql.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pyspark.sql.DataFrame) -> pyspark.sql.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb.mdx
new file mode 100644
index 0000000000000..7d1f0a6b5953f
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-duckdb.mdx
@@ -0,0 +1,233 @@
+---
+title: 'duckdb (dagster-duckdb)'
+title_meta: 'duckdb (dagster-duckdb) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'duckdb (dagster-duckdb) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# DuckDB (dagster-duckdb)
+
+This library provides an integration with the [DuckDB](https://duckdb.org/) database.
+
+Related Guides:
+
+ - [Using Dagster with DuckDB guide](https://docs.dagster.io/integrations/libraries/duckdb/)
+ - [DuckDB I/O manager reference](https://docs.dagster.io/integrations/libraries/duckdb/reference)
+
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to DuckDB.
+
+ Examples:
+
+ ```python
+ from dagster_duckdb import DuckDBIOManager
+ from dagster_duckdb_pandas import DuckDBPandasTypeHandler
+
+ class MyDuckDBIOManager(DuckDBIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DuckDBPandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb")}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb", schema="my_schema")}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ Set DuckDB configuration options using the connection_config field. See
+ [https://duckdb.org/docs/sql/configuration.html](https://duckdb.org/docs/sql/configuration.html) for all available settings.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb",
+ connection_config={"arrow_large_buffer_size": True})}
+ )
+ ```
+
+
+
+
+
+
dagster_duckdb.DuckDBResource ResourceDefinition
+
+
+
+
+ Resource for interacting with a DuckDB database.
+
+ Examples:
+
+ ```python
+ from dagster import Definitions, asset
+ from dagster_duckdb import DuckDBResource
+
+ @asset
+ def my_table(duckdb: DuckDBResource):
+ with duckdb.get_connection() as conn:
+ conn.execute("SELECT * from MY_SCHEMA.MY_TABLE")
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"duckdb": DuckDBResource(database="path/to/db.duckdb")}
+ )
+ ```
+
+
+
+ Builds an IO manager definition that reads inputs from and writes outputs to DuckDB.
+
+ Parameters:
+ - type_handlers (Sequence[DbTypeHandler]) – Each handler defines how to translate between
+ - default_load_type (Type) – When an input has no type annotation, load it as this type.
+
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb import build_duckdb_io_manager
+ from dagster_duckdb_pandas import DuckDBPandasTypeHandler
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ duckdb_io_manager = build_duckdb_io_manager([DuckDBPandasTypeHandler()])
+
+ defs = Definitions(
+ assets=[my_table]
+ resources={"io_manager" duckdb_io_manager.configured({"database": "my_db.duckdb"})}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={"io_manager" duckdb_io_manager.configured(
+ {"database": "my_db.duckdb", "schema": "my_schema"} # will be used as the schema
+ )}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-fivetran.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-fivetran.mdx
new file mode 100644
index 0000000000000..41c39bfb989de
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-fivetran.mdx
@@ -0,0 +1,541 @@
+---
+title: 'fivetran (dagster-fivetran)'
+title_meta: 'fivetran (dagster-fivetran) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'fivetran (dagster-fivetran) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Fivetran (dagster-fivetran)
+
+This library provides a Dagster integration with [Fivetran](https://www.fivetran.com/).
+
+
+
+
+## Assets (Fivetran API)
+
+
+
class dagster_fivetran.FivetranWorkspace
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This class represents a Fivetran workspace and provides utilities
+ to interact with Fivetran APIs.
+
+
+
sync_and_poll
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Executes a sync and poll process to materialize Fivetran assets.
+ This method can only be used in the context of an asset execution.
+
+ Parameters: context ([*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)) – The execution context
+ from within @fivetran_assets.Returns:
+ An iterator of MaterializeResult
+ or AssetMaterialization.
+
+ Return type: Iterator[Union[[AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [MaterializeResult](../assets.mdx#dagster.MaterializeResult)]]
+
+
+
+
+
+
+
+
+
+
class dagster_fivetran.DagsterFivetranTranslator
+
+ Translator class which converts a FivetranConnectorTableProps object into AssetSpecs.
+ Subclass this class to implement custom logic on how to translate Fivetran content into asset spec.
+
+
+
+
+
@dagster_fivetran.fivetran_assets
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Create a definition for how to sync the tables of a given Fivetran connector.
+
+ Parameters:
+ - connector_id (str) – The Fivetran Connector ID. You can retrieve this value from the
+ - workspace ([*FivetranWorkspace*](#dagster_fivetran.FivetranWorkspace)) – The Fivetran workspace to fetch assets from.
+ - name (Optional[str], optional) – The name of the op.
+ - group_name (Optional[str], optional) – The name of the asset group.
+ - dagster_fivetran_translator (Optional[[*DagsterFivetranTranslator*](#dagster_fivetran.DagsterFivetranTranslator)], optional) – The translator to use
+
+
+ Examples:
+
+ Sync the tables of a Fivetran connector:
+
+ ```python
+ from dagster_fivetran import FivetranWorkspace, fivetran_assets
+
+ import dagster as dg
+
+ fivetran_workspace = FivetranWorkspace(
+ account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"),
+ api_key=dg.EnvVar("FIVETRAN_API_KEY"),
+ api_secret=dg.EnvVar("FIVETRAN_API_SECRET"),
+ )
+
+ @fivetran_assets(
+ connector_id="fivetran_connector_id",
+ name="fivetran_connector_id",
+ group_name="fivetran_connector_id",
+ workspace=fivetran_workspace,
+ )
+ def fivetran_connector_assets(context: dg.AssetExecutionContext, fivetran: FivetranWorkspace):
+ yield from fivetran.sync_and_poll(context=context)
+
+ defs = dg.Definitions(
+ assets=[fivetran_connector_assets],
+ resources={"fivetran": fivetran_workspace},
+ )
+ ```
+ Sync the tables of a Fivetran connector with a custom translator:
+
+ ```python
+ from dagster_fivetran import (
+ DagsterFivetranTranslator,
+ FivetranConnectorTableProps,
+ FivetranWorkspace,
+ fivetran_assets
+ )
+
+ import dagster as dg
+ from dagster._core.definitions.asset_spec import replace_attributes
+
+ class CustomDagsterFivetranTranslator(DagsterFivetranTranslator):
+ def get_asset_spec(self, props: FivetranConnectorTableProps) -> dg.AssetSpec:
+ default_spec = super().get_asset_spec(props)
+ return default_spec.replace_attributes(
+ key=default_spec.key.with_prefix("my_prefix"),
+ )
+
+
+ fivetran_workspace = FivetranWorkspace(
+ account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"),
+ api_key=dg.EnvVar("FIVETRAN_API_KEY"),
+ api_secret=dg.EnvVar("FIVETRAN_API_SECRET"),
+ )
+
+ @fivetran_assets(
+ connector_id="fivetran_connector_id",
+ name="fivetran_connector_id",
+ group_name="fivetran_connector_id",
+ workspace=fivetran_workspace,
+ dagster_fivetran_translator=CustomDagsterFivetranTranslator(),
+ )
+ def fivetran_connector_assets(context: dg.AssetExecutionContext, fivetran: FivetranWorkspace):
+ yield from fivetran.sync_and_poll(context=context)
+
+ defs = dg.Definitions(
+ assets=[fivetran_connector_assets],
+ resources={"fivetran": fivetran_workspace},
+ )
+ ```
+
+
+
+
+
+
dagster_fivetran.load_fivetran_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Fivetran content in the workspace.
+
+ Parameters:
+ - workspace ([*FivetranWorkspace*](#dagster_fivetran.FivetranWorkspace)) – The Fivetran workspace to fetch assets from.
+ - dagster_fivetran_translator (Optional[[*DagsterFivetranTranslator*](#dagster_fivetran.DagsterFivetranTranslator)], optional) – The translator to use
+
+
+ Returns: The set of assets representing the Fivetran content in the workspace.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+ Examples:
+
+ Loading the asset specs for a given Fivetran workspace:
+
+ ```python
+ from dagster_fivetran import FivetranWorkspace, load_fivetran_asset_specs
+
+ import dagster as dg
+
+ fivetran_workspace = FivetranWorkspace(
+ account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"),
+ api_key=dg.EnvVar("FIVETRAN_API_KEY"),
+ api_secret=dg.EnvVar("FIVETRAN_API_SECRET"),
+ )
+
+ fivetran_specs = load_fivetran_asset_specs(fivetran_workspace)
+ defs = dg.Definitions(assets=[*fivetran_specs], resources={"fivetran": fivetran_workspace}
+ ```
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ The list of AssetsDefinition for all connectors in the Fivetran workspace.
+
+ Parameters:
+ - workspace ([*FivetranWorkspace*](#dagster_fivetran.FivetranWorkspace)) – The Fivetran workspace to fetch assets from.
+ - dagster_fivetran_translator (Optional[[*DagsterFivetranTranslator*](#dagster_fivetran.DagsterFivetranTranslator)], optional) – The translator to use
+
+
+ Returns: The list of AssetsDefinition for all connectors in the Fivetran workspace.Return type: List[[AssetsDefinition](../assets.mdx#dagster.AssetsDefinition)]
+ Examples:
+
+ Sync the tables of a Fivetran connector:
+
+ ```python
+ from dagster_fivetran import FivetranWorkspace, build_fivetran_assets_definitions
+
+ import dagster as dg
+
+ fivetran_workspace = FivetranWorkspace(
+ account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"),
+ api_key=dg.EnvVar("FIVETRAN_API_KEY"),
+ api_secret=dg.EnvVar("FIVETRAN_API_SECRET"),
+ )
+
+ fivetran_assets = build_fivetran_assets_definitions(workspace=workspace)
+
+ defs = dg.Definitions(
+ assets=[*fivetran_assets],
+ resources={"fivetran": fivetran_workspace},
+ )
+ ```
+ Sync the tables of a Fivetran connector with a custom translator:
+
+ ```python
+ from dagster_fivetran import (
+ DagsterFivetranTranslator,
+ FivetranConnectorTableProps,
+ FivetranWorkspace,
+ build_fivetran_assets_definitions
+ )
+
+ import dagster as dg
+ from dagster._core.definitions.asset_spec import replace_attributes
+
+ class CustomDagsterFivetranTranslator(DagsterFivetranTranslator):
+ def get_asset_spec(self, props: FivetranConnectorTableProps) -> dg.AssetSpec:
+ default_spec = super().get_asset_spec(props)
+ return default_spec.replace_attributes(
+ key=default_spec.key.with_prefix("my_prefix"),
+ )
+
+
+ fivetran_workspace = FivetranWorkspace(
+ account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"),
+ api_key=dg.EnvVar("FIVETRAN_API_KEY"),
+ api_secret=dg.EnvVar("FIVETRAN_API_SECRET"),
+ )
+
+ fivetran_assets = build_fivetran_assets_definitions(
+ workspace=workspace,
+ dagster_fivetran_translator=CustomDagsterFivetranTranslator()
+ )
+
+ defs = dg.Definitions(
+ assets=[*fivetran_assets],
+ resources={"fivetran": fivetran_workspace},
+ )
+ ```
+
+
+
+
+
+
class dagster_fivetran.fivetran_event_iterator.FivetranEventIterator
+
+
+ A wrapper around an iterator of Fivetran events which contains additional methods for
+ post-processing the events, such as fetching column metadata.
+
+
+
fetch_column_metadata
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Fetches column metadata for each table synced with the Fivetran API.
+
+ Retrieves the column schema for each destination table.
+
+ Returns: An iterator of Dagster events with column metadata attached.Return type: [FivetranEventIterator](#dagster_fivetran.fivetran_event_iterator.FivetranEventIterator)
+
+
+
+ This resource allows users to programatically interface with the Fivetran REST API to launch
+ syncs and monitor their progress. This currently implements only a subset of the functionality
+ exposed by the API.
+
+ For a complete set of documentation on the Fivetran REST API, including expected response JSON
+ schemae, see the [Fivetran API Docs](https://fivetran.com/docs/rest-api/connectors).
+
+ To configure this resource, we recommend using the [configured](https://legacy-docs.dagster.io/concepts/configuration/configured) method.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_fivetran import fivetran_resource
+
+ my_fivetran_resource = fivetran_resource.configured(
+ {
+ "api_key": {"env": "FIVETRAN_API_KEY"},
+ "api_secret": {"env": "FIVETRAN_API_SECRET"},
+ }
+ )
+
+ @job(resource_defs={"fivetran":my_fivetran_resource})
+ def my_fivetran_job():
+ ...
+ ```
+
+
+
+ Loads Fivetran connector assets from a configured FivetranResource instance. This fetches information
+ about defined connectors at initialization time, and will error on workspace load if the Fivetran
+ instance is not reachable.
+
+ Parameters:
+ - fivetran ([*ResourceDefinition*](../resources.mdx#dagster.ResourceDefinition)) – A FivetranResource configured with the appropriate connection
+ - key_prefix (Optional[CoercibleToAssetKeyPrefix]) – A prefix for the asset keys created.
+ - connector_to_group_fn (Optional[Callable[[str], Optional[str]]]) – Function which returns an asset
+ - io_manager_key (Optional[str]) – The IO manager key to use for all assets. Defaults to “io_manager”.
+ - connector_to_io_manager_key_fn (Optional[Callable[[str], Optional[str]]]) – Function which returns an
+ - connector_filter (Optional[Callable[[FivetranConnectorMetadata], bool]]) – Optional function which takes
+ - connector_to_asset_key_fn (Optional[Callable[[FivetranConnectorMetadata, str], [*AssetKey*](../assets.mdx#dagster.AssetKey)]]) – Optional function
+ - destination_ids (Optional[List[str]]) – A list of destination IDs to fetch connectors from. If None, all destinations
+ - poll_interval (float) – The time (in seconds) that will be waited between successive polls.
+ - poll_timeout (Optional[float]) – The maximum time that will waited before this operation is
+ - fetch_column_metadata (bool) – If True, will fetch column schema information for each table in the connector.
+
+
+ Examples:
+
+ Loading all Fivetran connectors as assets:
+
+ ```python
+ from dagster_fivetran import fivetran_resource, load_assets_from_fivetran_instance
+
+ fivetran_instance = fivetran_resource.configured(
+ {
+ "api_key": "some_key",
+ "api_secret": "some_secret",
+ }
+ )
+ fivetran_assets = load_assets_from_fivetran_instance(fivetran_instance)
+ ```
+ Filtering the set of loaded connectors:
+
+ ```python
+ from dagster_fivetran import fivetran_resource, load_assets_from_fivetran_instance
+
+ fivetran_instance = fivetran_resource.configured(
+ {
+ "api_key": "some_key",
+ "api_secret": "some_secret",
+ }
+ )
+ fivetran_assets = load_assets_from_fivetran_instance(
+ fivetran_instance,
+ connector_filter=lambda meta: "snowflake" in meta.name,
+ )
+ ```
+
+
+
+
+
+
+
dagster_fivetran.build_fivetran_assets
+
+
+ Build a set of assets for a given Fivetran connector.
+
+ Returns an AssetsDefinition which connects the specified `asset_keys` to the computation that
+ will update them. Internally, executes a Fivetran sync for a given `connector_id`, and
+ polls until that sync completes, raising an error if it is unsuccessful. Requires the use of the
+ [`fivetran_resource`](#dagster_fivetran.fivetran_resource), which allows it to communicate with the
+ Fivetran API.
+
+ Parameters:
+ - connector_id (str) – The Fivetran Connector ID that this op will sync. You can retrieve this
+ - destination_tables (List[str]) – schema_name.table_name for each table that you want to be
+ - poll_interval (float) – The time (in seconds) that will be waited between successive polls.
+ - poll_timeout (Optional[float]) – The maximum time that will waited before this operation is
+ - io_manager_key (Optional[str]) – The io_manager to be used to handle each of these assets.
+ - asset_key_prefix (Optional[List[str]]) – A prefix for the asset keys inside this asset.
+ - metadata_by_table_name (Optional[Mapping[str, RawMetadataMapping]]) – A mapping from destination
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. This
+ - infer_missing_tables (bool) – If True, will create asset materializations for tables specified
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset. Frameworks may expect and
+ - fetch_column_metadata (bool) – If True, will fetch column schema information for each table in the connector.
+
+
+ Examples:
+
+ Basic example:
+
+ >
+
+ ```python
+ from dagster import AssetKey, repository, with_resources
+
+ from dagster_fivetran import fivetran_resource
+ from dagster_fivetran.assets import build_fivetran_assets
+
+ my_fivetran_resource = fivetran_resource.configured(
+ {
+ "api_key": {"env": "FIVETRAN_API_KEY"},
+ "api_secret": {"env": "FIVETRAN_API_SECRET"},
+ }
+ )
+ ```
+
+
+Attaching metadata:
+
+>
+
+ ```python
+ fivetran_assets = build_fivetran_assets(
+ connector_id="foobar",
+ table_names=["schema1.table1", "schema2.table2"],
+ metadata_by_table_name={
+ "schema1.table1": {
+ "description": "This is a table that contains foo and bar",
+ },
+ "schema2.table2": {
+ "description": "This is a table that contains baz and quux",
+ },
+ },
+ )
+ ```
+
+
+
+
+
+
+
+
dagster_fivetran.fivetran_sync_op `=` \
+
+
+
+
+ Executes a Fivetran sync for a given `connector_id`, and polls until that sync
+ completes, raising an error if it is unsuccessful. It outputs a FivetranOutput which contains
+ the details of the Fivetran connector after the sync successfully completes, as well as details
+ about which tables the sync updates.
+
+ It requires the use of the [`fivetran_resource`](#dagster_fivetran.fivetran_resource), which allows it to
+ communicate with the Fivetran API.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_fivetran import fivetran_resource, fivetran_sync_op
+
+ my_fivetran_resource = fivetran_resource.configured(
+ {
+ "api_key": {"env": "FIVETRAN_API_KEY"},
+ "api_secret": {"env": "FIVETRAN_API_SECRET"},
+ }
+ )
+
+ sync_foobar = fivetran_sync_op.configured({"connector_id": "foobar"}, name="sync_foobar")
+
+ @job(resource_defs={"fivetran": my_fivetran_resource})
+ def my_simple_fivetran_job():
+ sync_foobar()
+
+ @job(resource_defs={"fivetran": my_fivetran_resource})
+ def my_composed_fivetran_job():
+ final_foobar_state = sync_foobar(start_after=some_op())
+ other_op(final_foobar_state)
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-gcp-pandas.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-gcp-pandas.mdx
new file mode 100644
index 0000000000000..3de869ce23903
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-gcp-pandas.mdx
@@ -0,0 +1,256 @@
+---
+title: 'gcp + pandas (dagster-gcp-pandas)'
+title_meta: 'gcp + pandas (dagster-gcp-pandas) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'gcp + pandas (dagster-gcp-pandas) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# GCP + Pandas (dagster-gcp-pandas)
+
+
+
+
+## Google BigQuery
+
+This library provides an integration with the [BigQuery](https://cloud.google.com/bigquery) database and Pandas data processing library.
+
+Related Guides:
+
+ - [Using Dagster with BigQuery](https://docs.dagster.io/integrations/libraries/gcp/bigquery/)
+ - [BigQuery I/O manager reference](https://docs.dagster.io/integrations/libraries/gcp/bigquery/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes pandas DataFrames to BigQuery.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_gcp_pandas import BigQueryPandasIOManager
+ from dagster import Definitions, EnvVar
+
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": BigQueryPandasIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": BigQueryPandasIOManager(project=EnvVar("GCP_PROJECT", dataset="my_dataset")
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the “gcp_credentials” configuration.
+ Dagster will store this key in a temporary file and set GOOGLE_APPLICATION_CREDENTIALS to point to the file.
+ After the run completes, the file will be deleted, and GOOGLE_APPLICATION_CREDENTIALS will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded key with this shell command: cat $GOOGLE_APPLICATION_CREDENTIALS | base64
+
+
+
+
+
+
+
class dagster_gcp_pandas.BigQueryPandasTypeHandler
+
+
+ Plugin for the BigQuery I/O Manager that can store and load Pandas DataFrames as BigQuery tables.
+
+ Examples:
+
+ ```python
+ from dagster_gcp import BigQueryIOManager
+ from dagster_bigquery_pandas import BigQueryPandasTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MyBigQueryIOManager(BigQueryIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [BigQueryPandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_dataset"] # my_dataset will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MyBigQueryIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes pandas DataFrames to BigQuery.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_gcp_pandas import bigquery_pandas_io_manager
+ from dagster import Definitions
+
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": bigquery_pandas_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ })
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": bigquery_pandas_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ "dataset": "my_dataset"
+ })
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the “gcp_credentials” configuration.
+ Dagster will store this key in a temporary file and set GOOGLE_APPLICATION_CREDENTIALS to point to the file.
+ After the run completes, the file will be deleted, and GOOGLE_APPLICATION_CREDENTIALS will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded key with this shell command: cat $GOOGLE_APPLICATION_CREDENTIALS | base64
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-gcp-pyspark.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-gcp-pyspark.mdx
new file mode 100644
index 0000000000000..297d09408566a
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-gcp-pyspark.mdx
@@ -0,0 +1,256 @@
+---
+title: 'gcp + pyspark (dagster-gcp-pyspark)'
+title_meta: 'gcp + pyspark (dagster-gcp-pyspark) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'gcp + pyspark (dagster-gcp-pyspark) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# GCP + PySpark (dagster-gcp-pyspark)
+
+
+
+
+## Google BigQuery
+
+This library provides an integration with the [BigQuery](https://cloud.google.com/bigquery) database and PySpark data processing library.
+
+Related Guides:
+
+ - [Using Dagster with BigQuery](https://docs.dagster.io/integrations/libraries/gcp/bigquery/)
+ - [BigQuery I/O manager reference](https://docs.dagster.io/integrations/libraries/gcp/bigquery/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to BigQuery.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_gcp_pyspark import BigQueryPySparkIOManager
+ from dagster import Definitions, EnvVar
+
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pyspark.sql.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": BigQueryPySparkIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": BigQueryPySparkIOManager(project=EnvVar("GCP_PROJECT", dataset="my_dataset")
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pyspark.sql.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pyspark.sql.DataFrame) -> pyspark.sql.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the “gcp_credentials” configuration.
+ Dagster will store this key in a temporary file and set GOOGLE_APPLICATION_CREDENTIALS to point to the file.
+ After the run completes, the file will be deleted, and GOOGLE_APPLICATION_CREDENTIALS will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded key with this shell command: cat $GOOGLE_APPLICATION_CREDENTIALS | base64
+
+
+
+
+
+
+
class dagster_gcp_pyspark.BigQueryPySparkTypeHandler
+
+
+ Plugin for the BigQuery I/O Manager that can store and load PySpark DataFrames as BigQuery tables.
+
+ Examples:
+
+ ```python
+ from dagster_gcp import BigQueryIOManager
+ from dagster_bigquery_pandas import BigQueryPySparkTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MyBigQueryIOManager(BigQueryIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [BigQueryPySparkTypeHandler()]
+
+ @asset(
+ key_prefix=["my_dataset"] # my_dataset will be used as the dataset in BigQuery
+ )
+ def my_table() -> pyspark.sql.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MyBigQueryIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to BigQuery.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_gcp_pyspark import bigquery_pyspark_io_manager
+ from dagster import Definitions
+
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": bigquery_pyspark_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ })
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": bigquery_pandas_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ "dataset": "my_dataset"
+ })
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pyspark.sql.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pyspark.sql.DataFrame) -> pyspark.sql.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the “gcp_credentials” configuration.
+ Dagster will store this key in a temporary file and set GOOGLE_APPLICATION_CREDENTIALS to point to the file.
+ After the run completes, the file will be deleted, and GOOGLE_APPLICATION_CREDENTIALS will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded key with this shell command: cat $GOOGLE_APPLICATION_CREDENTIALS | base64
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-gcp.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-gcp.mdx
new file mode 100644
index 0000000000000..5e30584bb9092
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-gcp.mdx
@@ -0,0 +1,820 @@
+---
+title: 'gcp (dagster-gcp)'
+title_meta: 'gcp (dagster-gcp) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'gcp (dagster-gcp) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+ Resource for interacting with Google BigQuery.
+
+ Examples:
+
+ ```python
+ from dagster import Definitions, asset
+ from dagster_gcp import BigQueryResource
+
+ @asset
+ def my_table(bigquery: BigQueryResource):
+ with bigquery.get_client() as client:
+ client.query("SELECT * FROM my_dataset.my_table")
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "bigquery": BigQueryResource(project="my-project")
+ }
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+### BigQuery I/O Manager
+
+
+
dagster_gcp.BigQueryIOManager IOManagerDefinition
+
+
+
+
+ Base class for an I/O manager definition that reads inputs from and writes outputs to BigQuery.
+
+ Examples:
+
+ ```python
+ from dagster_gcp import BigQueryIOManager
+ from dagster_bigquery_pandas import BigQueryPandasTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MyBigQueryIOManager(BigQueryIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [BigQueryPandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_dataset"] # my_dataset will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MyBigQueryIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MyBigQueryIOManager(project=EnvVar("GCP_PROJECT"), dataset="my_dataset")
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata `columns` to the
+ [`In`](../ops.mdx#dagster.In) or [`AssetIn`](../assets.mdx#dagster.AssetIn).
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the `gcp_credentials` configuration.
+ Dagster will store this key in a temporary file and set `GOOGLE_APPLICATION_CREDENTIALS` to point to the file.
+ After the run completes, the file will be deleted, and `GOOGLE_APPLICATION_CREDENTIALS` will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded with this shell command: `cat $GOOGLE_APPLICATION_CREDENTIALS | base64`
+
+
+
+
+
+
+
+
+
+
+
+### BigQuery Ops
+
+
+
dagster_gcp.bq_create_dataset
+
+
+ BigQuery Create Dataset.
+
+ This op encapsulates creating a BigQuery dataset.
+
+ Expects a BQ client to be provisioned in resources as context.resources.bigquery.
+
+
+
+
+
+
+
dagster_gcp.bq_delete_dataset
+
+
+ BigQuery Delete Dataset.
+
+ This op encapsulates deleting a BigQuery dataset.
+
+ Expects a BQ client to be provisioned in resources as context.resources.bigquery.
+
+
+
+
+
+
+
dagster_gcp.bq_op_for_queries
+
+
+ Executes BigQuery SQL queries.
+
+ Expects a BQ client to be provisioned in resources as context.resources.bigquery.
+
+
+
+
+
+
+
dagster_gcp.import_df_to_bq
+
+
+
+
+
+
+
dagster_gcp.import_file_to_bq
+
+
+
+
+
+
+
dagster_gcp.import_gcs_paths_to_bq
+
+
+
+
+
+
+
+
+
+
+
+### Data Freshness
+
+
+
dagster_gcp.fetch_last_updated_timestamps
+
+
+ Get the last updated timestamps of a list BigQuery table.
+
+ Note that this only works on BigQuery tables, and not views.
+
+ Parameters:
+ - client (bigquery.Client) – The BigQuery client.
+ - dataset_id (str) – The BigQuery dataset ID.
+ - table_ids (Sequence[str]) – The table IDs to get the last updated timestamp for.
+
+
+ Returns: A mapping of table IDs to their last updated timestamps (UTC).Return type: Mapping[str, datetime]
+
+
+
+
+
+
+
+
+
+
+### Other
+
+
+
class dagster_gcp.BigQueryError
+
+
+
+
+
+
+
+
+
+
+
+## GCS
+
+
+
+
+### GCS Resource
+
+
+
dagster_gcp.GCSResource ResourceDefinition
+
+
+
+
+ Resource for interacting with Google Cloud Storage.
+
+ Example:
+
+ ```default
+ @asset
+ def my_asset(gcs: GCSResource):
+ with gcs.get_client() as client:
+ # client is a google.cloud.storage.Client
+ ...
+ ```
+
+
+
+ Persistent IO manager using GCS for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for GCS and the backing bucket.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at `\/\`. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of `/my/base/path`, an asset with key
+ `AssetKey(["one", "two", "three"])` would be stored in a file called `three` in a directory
+ with path `/my/base/path/one/two/`.
+
+ Example usage:
+
+ 1. Attach this IO manager to a set of assets.
+ ```python
+ from dagster import asset, Definitions
+ from dagster_gcp.gcs import GCSPickleIOManager, GCSResource
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": GCSPickleIOManager(
+ gcs_bucket="my-cool-bucket",
+ gcs_prefix="my-cool-prefix",
+ gcs=GCSResource(project="my-cool-project")
+ ),
+
+ }
+ )
+ ```
+ 2. Attach this IO manager to your job to make it available to your ops.
+ ```python
+ from dagster import job
+ from dagster_gcp.gcs import GCSPickleIOManager, GCSResource
+
+ @job(
+ resource_defs={
+ "io_manager": GCSPickleIOManager(
+ gcs=GCSResource(project="my-cool-project")
+ gcs_bucket="my-cool-bucket",
+ gcs_prefix="my-cool-prefix"
+ ),
+ }
+ )
+ def my_job():
+ ...
+ ```
+
+
+
+
+
+
+
+
+
+
+### GCS Sensor
+
+
+
dagster_gcp.gcs.sensor.get_gcs_keys
+
+
+ Return a list of updated keys in a GCS bucket.
+
+ Parameters:
+ - bucket (str) – The name of the GCS bucket.
+ - prefix (Optional[str]) – The prefix to filter the keys by.
+ - since_key (Optional[str]) – The key to start from. If provided, only keys updated after this key will be returned.
+ - gcs_session (Optional[google.cloud.storage.client.Client]) – A GCS client session. If not provided, a new session will be created.
+
+
+ Returns: A list of keys in the bucket, sorted by update time, that are newer than the since_key.Return type: List[str]
+ Example:
+
+ ```python
+ @resource
+ def google_cloud_storage_client(context):
+ return storage.Client().from_service_account_json("my-service-account.json")
+
+ @sensor(job=my_job, required_resource_keys={"google_cloud_storage_client"})
+ def my_gcs_sensor(context):
+ since_key = context.cursor or None
+ new_gcs_keys = get_gcs_keys(
+ "my-bucket",
+ prefix="data",
+ since_key=since_key,
+ gcs_session=context.resources.google_cloud_storage_client
+ )
+
+ if not new_gcs_keys:
+ return SkipReason("No new gcs files found for bucket 'my-bucket'.")
+
+ for gcs_key in new_gcs_keys:
+ yield RunRequest(run_key=gcs_key, run_config={
+ "ops": {
+ "gcs_files": {
+ "config": {
+ "gcs_key": gcs_key
+ }
+ }
+ }
+ })
+
+ last_key = new_gcs_keys[-1]
+ context.update_cursor(last_key)
+ ```
+
+
+
+ FileManager that provides abstract access to GCS.
+
+
+
+
+
+
+
+
+
+
+
+### GCS Compute Log Manager
+
+
+
class dagster_gcp.gcs.GCSComputeLogManager
+
+
+ Logs op compute function stdout and stderr to GCS.
+
+ Users should not instantiate this class directly. Instead, use a YAML block in `dagster.yaml`
+ such as the following:
+
+ ```YAML
+ compute_logs:
+ module: dagster_gcp.gcs.compute_log_manager
+ class: GCSComputeLogManager
+ config:
+ bucket: "mycorp-dagster-compute-logs"
+ local_dir: "/tmp/cool"
+ prefix: "dagster-test-"
+ upload_interval: 30
+ ```
+ There are more configuration examples in the instance documentation guide: [https://docs.dagster.io/guides/deploy/dagster-instance-configuration#compute-log-storage](https://docs.dagster.io/guides/deploy/dagster-instance-configuration#compute-log-storage)
+
+ Parameters:
+ - bucket (str) – The name of the GCS bucket to which to log.
+ - local_dir (Optional[str]) – Path to the local directory in which to stage logs. Default:
+ - prefix (Optional[str]) – Prefix for the log file keys.
+ - json_credentials_envvar (Optional[str]) – Environment variable that contains the JSON with a private key
+ - upload_interval – (Optional[int]): Interval in seconds to upload partial log files to GCS. By default, will only upload when the capture is complete.
+ - show_url_only – (Optional[bool]): Only show the URL of the log file in the UI, instead of fetching and displaying the full content. Default False.
+ - inst_data (Optional[[*ConfigurableClassData*](../internals.mdx#dagster._serdes.ConfigurableClassData)]) – Serializable representation of the compute
+
+
+
+
+
+
+
+
+
+
+
+
+## Dataproc
+
+
+
+
+### Dataproc Resource
+
+
+
dagster_gcp.DataprocResource ResourceDefinition
+
+
+
+
+ Resource for connecting to a Dataproc cluster.
+
+ Example:
+
+ ```default
+ @asset
+ def my_asset(dataproc: DataprocResource):
+ with dataproc.get_client() as client:
+ # client is a dagster_gcp.DataprocClient
+ ...
+ ```
+
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Please use GCSPickleIOManager instead..
+
+ :::
+
+ Renamed to GCSPickleIOManager. See GCSPickleIOManager for documentation.
+
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Builds an I/O manager definition that reads inputs from and writes outputs to BigQuery.
+
+ Parameters:
+ - type_handlers (Sequence[DbTypeHandler]) – Each handler defines how to translate between
+ - default_load_type (Type) – When an input has no type annotation, load it as this type.
+
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_gcp import build_bigquery_io_manager
+ from dagster_bigquery_pandas import BigQueryPandasTypeHandler
+ from dagster import Definitions
+
+ @asset(
+ key_prefix=["my_prefix"],
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ @asset(
+ key_prefix=["my_dataset"] # my_dataset will be used as the dataset in BigQuery
+ )
+ def my_second_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ bigquery_io_manager = build_bigquery_io_manager([BigQueryPandasTypeHandler()])
+
+ defs = Definitions(
+ assets=[my_table, my_second_table],
+ resources={
+ "io_manager": bigquery_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ })
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": bigquery_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ "dataset": "my_dataset"
+ })
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata `columns` to the
+ [`In`](../ops.mdx#dagster.In) or [`AssetIn`](../assets.mdx#dagster.AssetIn).
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the `gcp_credentials` configuration.
+ Dagster willstore this key in a temporary file and set `GOOGLE_APPLICATION_CREDENTIALS` to point to the file.
+ After the run completes, the file will be deleted, and `GOOGLE_APPLICATION_CREDENTIALS` will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded with this shell command: `cat $GOOGLE_APPLICATION_CREDENTIALS | base64`
+
+
+
+
+ Persistent IO manager using GCS for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for GCS and the backing bucket.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at `\/\`. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of `/my/base/path`, an asset with key
+ `AssetKey(["one", "two", "three"])` would be stored in a file called `three` in a directory
+ with path `/my/base/path/one/two/`.
+
+ Example usage:
+
+ 1. Attach this IO manager to a set of assets.
+ ```python
+ from dagster import Definitions, asset
+ from dagster_gcp.gcs import gcs_pickle_io_manager, gcs_resource
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": gcs_pickle_io_manager.configured(
+ {"gcs_bucket": "my-cool-bucket", "gcs_prefix": "my-cool-prefix"}
+ ),
+ "gcs": gcs_resource.configured({"project": "my-cool-project"}),
+ },
+ )
+ ```
+ 2. Attach this IO manager to your job to make it available to your ops.
+ ```python
+ from dagster import job
+ from dagster_gcp.gcs import gcs_pickle_io_manager, gcs_resource
+
+ @job(
+ resource_defs={
+ "io_manager": gcs_pickle_io_manager.configured(
+ {"gcs_bucket": "my-cool-bucket", "gcs_prefix": "my-cool-prefix"}
+ ),
+ "gcs": gcs_resource.configured({"project": "my-cool-project"}),
+ },
+ )
+ def my_job():
+ ...
+ ```
+
+
+
+
+
+
dagster_gcp.gcs_file_manager ResourceDefinition
+
+
+ FileManager that provides abstract access to GCS.
+
+ Implements the [`FileManager`](../internals.mdx#dagster._core.storage.file_manager.FileManager) API.
+
+
+
+
+
+
+
dagster_gcp.dataproc_resource ResourceDefinition
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-ge.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-ge.mdx
new file mode 100644
index 0000000000000..8100692178ada
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-ge.mdx
@@ -0,0 +1,55 @@
+---
+title: 'great expectations (dagster-ge)'
+title_meta: 'great expectations (dagster-ge) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'great expectations (dagster-ge) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+# Great Expectations (dagster-ge)
+
+
+
+
dagster_ge.ge_validation_op_factory
+
+ Generates ops for interacting with Great Expectations.
+
+ Parameters:
+
+ - - name (str) – the name of the op
+ - datasource_name (str) – the name of your DataSource, see your great_expectations.yml
+ - data_connector_name (str) – the name of the data connector for this datasource. This should
+ - data_asset_name (str) – the name of the data asset that this op will be validating.
+ - suite_name (str) – the name of your expectation suite, see your great_expectations.yml
+ - batch_identifier_fn (dict) – A dicitonary of batch identifiers to uniquely identify this
+ - input_dagster_type ([*DagsterType*](../types.mdx#dagster.DagsterType)) – the Dagster type used to type check the input to the op.
+ - runtime_method_type (str) – how GE should interperet the op input. One of (“batch_data”,
+ extra_kwargs (Optional[dict]) –
+
+ adds extra kwargs to the invocation of ge_data_context’s
+ get_validator method. If not set, input will be:
+
+ >
+
+ ```default
+ {
+ "datasource_name": datasource_name,
+ "data_connector_name": data_connector_name,
+ "data_asset_name": data_asset_name,
+ "runtime_parameters": {
+ "":
+ },
+ "batch_identifiers": batch_identifiers,
+ "expectation_suite_name": suite_name,
+ }
+ ```
+
+
+ Returns: An op that takes in a set of data and yields both an expectation with relevant metadata and
+ an output with all the metadata (for user processing)
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-github.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-github.mdx
new file mode 100644
index 0000000000000..47e4cb0e0281b
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-github.mdx
@@ -0,0 +1,417 @@
+---
+title: 'github (dagster-github)'
+title_meta: 'github (dagster-github) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'github (dagster-github) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# GitHub (dagster-github)
+
+This library provides an integration with GitHub Apps, to support performing various automation
+operations within your github repositories and with the tighter permissions scopes that github apps
+allow for vs using a personal token.
+
+Presently, it provides a thin wrapper on the [github v4 graphql API](https://developer.github.com/v4/).
+
+To use this integration, you’ll first need to create a GitHub App for it.
+
+1. Create App: Follow the instructions in
+ [https://developer.github.com/apps/quickstart-guides/setting-up-your-development-environment/](https://developer.github.com/apps/quickstart-guides/setting-up-your-development-environment/), You will end up with a private key and App ID, which will be used when configuring the
+ `dagster-github` resource. Note you will need to grant your app the relevent permissions
+ for the API requests you want to make, for example to post issues it will need read/write access
+ for the issues repository permission, more info on GitHub application permissions can be found
+ [here](https://developer.github.com/v3/apps/permissions/)
+2. Install App: Follow the instructions in
+ [https://developer.github.com/apps/quickstart-guides/setting-up-your-development-environment/#step-7-install-the-app-on-your-account](https://developer.github.com/apps/quickstart-guides/setting-up-your-development-environment/#step-7-install-the-app-on-your-account)
+3. Find your installation_id: You can pull this from the GitHub app administration page,
+ `https://github.com/apps//installations/`. Note if your app is
+ installed more than once you can also programatically retrieve these IDs.
+Sharing your App ID and Installation ID is fine, but make sure that the Private Key for your app is
+stored securily.
+
+
+
+
+## Posting Issues
+
+Now, you can create issues in GitHub from Dagster with the GitHub resource:
+
+ ```python
+ import os
+
+ from dagster import job, op
+ from dagster_github import GithubResource
+
+
+ @op
+ def github_op(github: GithubResource):
+ github.get_client().create_issue(
+ repo_name='dagster',
+ repo_owner='dagster-io',
+ title='Dagster\'s first github issue',
+ body='this open source thing seems like a pretty good idea',
+ )
+
+ @job(resource_defs={
+ 'github': GithubResource(
+ github_app_id=os.getenv('GITHUB_APP_ID'),
+ github_app_private_rsa_key=os.getenv('GITHUB_PRIVATE_KEY'),
+ github_installation_id=os.getenv('GITHUB_INSTALLATION_ID')
+ )})
+ def github_job():
+ github_op()
+
+ github_job.execute_in_process()
+ ```
+Run the above code, and you’ll see the issue appear in GitHub:
+
+GitHub enterprise users can provide their hostname in the run config. Provide `github_hostname`
+as part of your github config like below.
+
+ ```python
+ GithubResource(
+ github_app_id=os.getenv('GITHUB_APP_ID'),
+ github_app_private_rsa_key=os.getenv('GITHUB_PRIVATE_KEY'),
+ github_installation_id=os.getenv('GITHUB_INSTALLATION_ID'),
+ github_hostname=os.getenv('GITHUB_HOSTNAME'),
+ )
+ ```
+By provisioning `GithubResource` as a Dagster resource, you can post to GitHub from
+within any asset or op execution.
+
+
+
+ A client for interacting with the GitHub API.
+
+ This client handles authentication and provides methods for making requests
+ to the GitHub API using an authenticated session.
+
+
+
client
+
+
+ The HTTP session used for making requests.
+
+ Type: requests.Session
+
+
+
+
+
+
app_id
+
+
+ The GitHub App ID.
+
+ Type: int
+
+
+
+
+
+
app_private_rsa_key
+
+
+ The private RSA key for the GitHub App.
+
+ Type: str
+
+
+
+
+
+
default_installation_id
+
+
+ The default installation ID for the GitHub App.
+
+ Type: Optional[int]
+
+
+
+
+
+
hostname
+
+
+ The GitHub hostname, defaults to None.
+
+ Type: Optional[str]
+
+
+
+
+
+
installation_tokens
+
+
+ A dictionary to store installation tokens.
+
+ Type: Dict[Any, Any]
+
+
+
+
+
+
app_token
+
+
+ A dictionary to store the app token.
+
+ Type: Dict[str, Any]
+
+
+
+
+
+
create_issue
+
+
+ Create a new issue in the specified GitHub repository.
+
+ This method first retrieves the repository ID using the provided repository name
+ and owner, then creates a new issue in that repository with the given title and body.
+
+ Parameters:
+ - repo_name (str) – The name of the repository where the issue will be created.
+ - repo_owner (str) – The owner of the repository where the issue will be created.
+ - title (str) – The title of the issue.
+ - body (str) – The body content of the issue.
+ - installation_id (Optional[int]) – The installation ID to use for authentication.
+
+
+ Returns: The response data from the GitHub API containing the created issue details.Return type: Dict[str, Any]Raises: RuntimeError – If there are errors in the response from the GitHub API.
+
+
+
+
+
+
create_pull_request
+
+
+ Create a new pull request in the specified GitHub repository.
+
+ This method creates a pull request from the head reference (branch) to the base reference (branch)
+ in the specified repositories. It uses the provided title and body for the pull request description.
+
+ Parameters:
+ - base_repo_name (str) – The name of the base repository where the pull request will be created.
+ - base_repo_owner (str) – The owner of the base repository.
+ - base_ref_name (str) – The name of the base reference (branch) to which the changes will be merged.
+ - head_repo_name (str) – The name of the head repository from which the changes will be taken.
+ - head_repo_owner (str) – The owner of the head repository.
+ - head_ref_name (str) – The name of the head reference (branch) from which the changes will be taken.
+ - title (str) – The title of the pull request.
+ - body (Optional[str]) – The body content of the pull request. Defaults to None.
+ - maintainer_can_modify (Optional[bool]) – Whether maintainers can modify the pull request. Defaults to None.
+ - draft (Optional[bool]) – Whether the pull request is a draft. Defaults to None.
+ - installation_id (Optional[int]) – The installation ID to use for authentication.
+
+
+ Returns: The response data from the GitHub API containing the created pull request details.Return type: Dict[str, Any]Raises: RuntimeError – If there are errors in the response from the GitHub API.
+
+
+
+
+
+
create_ref
+
+
+ Create a new reference (branch) in the specified GitHub repository.
+
+ This method first retrieves the repository ID and the source reference (branch or tag)
+ using the provided repository name, owner, and source reference. It then creates a new
+ reference (branch) in that repository with the given target name.
+
+ Parameters:
+ - repo_name (str) – The name of the repository where the reference will be created.
+ - repo_owner (str) – The owner of the repository where the reference will be created.
+ - source (str) – The source reference (branch or tag) from which the new reference will be created.
+ - target (str) – The name of the new reference (branch) to be created.
+ - installation_id (Optional[int]) – The installation ID to use for authentication.
+
+
+ Returns: The response data from the GitHub API containing the created reference details.Return type: Dict[str, Any]Raises: RuntimeError – If there are errors in the response from the GitHub API.
+
+
+
+
+
+
execute
+
+
+ Execute a GraphQL query against the GitHub API.
+
+ This method sends a POST request to the GitHub API with the provided GraphQL query
+ and optional variables. It ensures that the appropriate installation token is included
+ in the request headers.
+
+ Parameters:
+ - query (str) – The GraphQL query string to be executed.
+ - variables (Optional[Dict[str, Any]]) – Optional variables to include in the query.
+ - headers (Optional[Dict[str, Any]]) – Optional headers to include in the request.
+ - installation_id (Optional[int]) – The installation ID to use for authentication.
+
+
+ Returns: The response data from the GitHub API.Return type: Dict[str, Any]Raises:
+ - RuntimeError – If no installation ID is provided and no default installation ID is set.
+ - requests.exceptions.HTTPError – If the request to the GitHub API fails.
+
+
+
+
+
+
+
+
get_installations
+
+
+ Retrieve the list of installations for the authenticated GitHub App.
+
+ This method makes a GET request to the GitHub API to fetch the installations
+ associated with the authenticated GitHub App. It ensures that the app token
+ is valid and includes it in the request headers.
+
+ Parameters: headers (Optional[Dict[str, Any]]) – Optional headers to include in the request.Returns: A dictionary containing the installations data.Return type: Dict[str, Any]Raises: requests.exceptions.HTTPError – If the request to the GitHub API fails.
+
+
+
+ A resource configuration class for GitHub integration.
+
+ This class provides configuration fields for setting up a GitHub Application,
+ including the application ID, private RSA key, installation ID, and hostname.
+
+
+
dagster_github.resources.github_app_id
+
+
+ The GitHub Application ID. For more information, see
+ [https://developer.github.com/apps/](https://developer.github.com/apps/).
+
+ Type: int
+
+
+
+ The private RSA key text for the GitHub Application.
+ For more information, see [https://developer.github.com/apps/](https://developer.github.com/apps/).
+
+ Type: str
+
+
+
+
+
+
dagster_github.resources.github_installation_id
+
+
+ The GitHub Application Installation ID.
+ Defaults to None. For more information, see [https://developer.github.com/apps/](https://developer.github.com/apps/).
+
+ Type: Optional[int]
+
+
+
+
+
+
dagster_github.resources.github_hostname
+
+
+ The GitHub hostname. Defaults to api.github.com.
+ For more information, see [https://developer.github.com/apps/](https://developer.github.com/apps/).
+
+ Type: Optional[str]
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-graphql.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-graphql.mdx
new file mode 100644
index 0000000000000..5886dbf4cd5aa
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-graphql.mdx
@@ -0,0 +1,180 @@
+---
+title: 'graphql (dagster-graphql)'
+title_meta: 'graphql (dagster-graphql) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'graphql (dagster-graphql) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# GraphQL (dagster-graphql)
+
+
+
+## Python Client
+
+
+
class dagster_graphql.DagsterGraphQLClient
+
+
+ Official Dagster Python Client for GraphQL.
+
+ Utilizes the gql library to dispatch queries over HTTP to a remote Dagster GraphQL Server
+
+ As of now, all operations on this client are synchronous.
+
+ Intended usage:
+
+ ```python
+ client = DagsterGraphQLClient("localhost", port_number=3000)
+ status = client.get_run_status(**SOME_RUN_ID**)
+ ```
+ Parameters:
+ - hostname (str) – Hostname for the Dagster GraphQL API, like localhost or
+ - port_number (Optional[int]) – Port number to connect to on the host.
+ - transport (Optional[Transport], optional) – A custom transport to use to connect to the
+ - use_https (bool, optional) – Whether to use https in the URL connection string for the
+ - timeout (int) – Number of seconds before requests should time out. Defaults to 60.
+ - headers (Optional[Dict[str, str]]) – Additional headers to include in the request. To use
+
+
+ Raises: ConnectionError – if the client cannot connect to the host.
+
+
get_run_status
+
+
+ Get the status of a given Pipeline Run.
+
+ Parameters: run_id (str) – run id of the requested pipeline run.Raises:
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("PipelineNotFoundError", message) – if the requested run id is not found
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("PythonError", message) – on internal framework errors
+
+
+ Returns: returns a status Enum describing the state of the requested pipeline runReturn type: [DagsterRunStatus](../internals.mdx#dagster.DagsterRunStatus)
+
+
+
+
+
+
reload_repository_location
+
+
+ Reloads a Dagster Repository Location, which reloads all repositories in that repository location.
+
+ This is useful in a variety of contexts, including refreshing the Dagster UI without restarting
+ the server.
+
+ Parameters: repository_location_name (str) – The name of the repository locationReturns: Object with information about the result of the reload requestReturn type: [ReloadRepositoryLocationInfo](#dagster_graphql.ReloadRepositoryLocationInfo)
+
+
+
+
+
+
shutdown_repository_location
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+
+
+ :::
+
+ Shuts down the server that is serving metadata for the provided repository location.
+
+ This is primarily useful when you want the server to be restarted by the compute environment
+ in which it is running (for example, in Kubernetes, the pod in which the server is running
+ will automatically restart when the server is shut down, and the repository metadata will
+ be reloaded)
+
+ Parameters: repository_location_name (str) – The name of the repository locationReturns: Object with information about the result of the reload requestReturn type: ShutdownRepositoryLocationInfo
+
+
+
+
+
+
submit_job_execution
+
+
+ Submits a job with attached configuration for execution.
+
+ Parameters:
+ - job_name (str) – The job’s name
+ - repository_location_name (Optional[str]) – The name of the repository location where
+ - repository_name (Optional[str]) – The name of the repository where the job is located.
+ - run_config (Optional[Union[[*RunConfig*](../config.mdx#dagster.RunConfig), Mapping[str, Any]]]) – This is the run config to execute the job with.
+ - tags (Optional[Dict[str, Any]]) – A set of tags to add to the job execution.
+
+
+ Raises:
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("InvalidStepError", invalid_step_key) – the job has an invalid step
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("InvalidOutputError", body=error_object) – some solid has an invalid output within the job.
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("RunConflict", message) – a DagsterRunConflict occured during execution.
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("PipelineConfigurationInvalid", invalid_step_key) – the run_config is not in the expected format
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("JobNotFoundError", message) – the requested job does not exist
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("PythonError", message) – an internal framework error occurred
+
+
+ Returns: run id of the submitted pipeline runReturn type: str
+
+
+
+ This class gives information about an InvalidOutputError from submitting a pipeline for execution
+ from GraphQL.
+
+ Parameters:
+ - step_key (str) – key of the step that failed
+ - invalid_output_name (str) – the name of the invalid output from the given step
+
+
+
+
+
+
+
+
class dagster_graphql.ReloadRepositoryLocationInfo
+
+
+ This class gives information about the result of reloading
+ a Dagster repository location with a GraphQL mutation.
+
+ Parameters:
+ - status ([*ReloadRepositoryLocationStatus*](#dagster_graphql.ReloadRepositoryLocationStatus)) – The status of the reload repository location mutation
+ - failure_type – (Optional[str], optional): the failure type if status == ReloadRepositoryLocationStatus.FAILURE.
+ - message (Optional[str], optional) – the failure message/reason if
+
+
+
+
+
+
+
+
class dagster_graphql.ReloadRepositoryLocationStatus
+
+
+ This enum describes the status of a GraphQL mutation to reload a Dagster repository location.
+
+ Parameters: Enum (str) – can be either ReloadRepositoryLocationStatus.SUCCESS
+ or ReloadRepositoryLocationStatus.FAILURE.
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-k8s.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-k8s.mdx
new file mode 100644
index 0000000000000..f371c6932843e
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-k8s.mdx
@@ -0,0 +1,551 @@
+---
+title: 'kubernetes (dagster-k8s)'
+title_meta: 'kubernetes (dagster-k8s) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'kubernetes (dagster-k8s) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Kubernetes (dagster-k8s)
+
+See also the [Kubernetes deployment guide](https://docs.dagster.io/deploying/kubernetes/).
+
+This library contains utilities for running Dagster with Kubernetes. This includes a Python API
+allowing the webserver to launch runs as Kubernetes Jobs, as well as a Helm chart you can use as the basis
+for a Dagster deployment on a Kubernetes cluster.
+
+
+
+
+
+
+
+# APIs
+
+
+
dagster_k8s.K8sRunLauncher RunLauncher
+
+
+
+
+ RunLauncher that starts a Kubernetes Job for each Dagster job run.
+
+ Encapsulates each run in a separate, isolated invocation of `dagster-graphql`.
+
+ You can configure a Dagster instance to use this RunLauncher by adding a section to your
+ `dagster.yaml` like the following:
+
+ ```yaml
+ run_launcher:
+ module: dagster_k8s.launcher
+ class: K8sRunLauncher
+ config:
+ service_account_name: your_service_account
+ job_image: my_project/dagster_image:latest
+ instance_config_map: dagster-instance
+ postgres_password_secret: dagster-postgresql-secret
+ ```
+
+
+
+
+
+
dagster_k8s.k8s_job_executor ExecutorDefinition
+
+
+
+
+ Executor which launches steps as Kubernetes Jobs.
+
+ To use the k8s_job_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster_k8s import k8s_job_executor
+
+ from dagster import job
+
+ @job(executor_def=k8s_job_executor)
+ def k8s_job():
+ pass
+ ```
+ Then you can configure the executor with run config as follows:
+
+ ```YAML
+ execution:
+ config:
+ job_namespace: 'some-namespace'
+ image_pull_policy: ...
+ image_pull_secrets: ...
+ service_account_name: ...
+ env_config_maps: ...
+ env_secrets: ...
+ env_vars: ...
+ job_image: ... # leave out if using userDeployments
+ max_concurrent: ...
+ ```
+ max_concurrent limits the number of pods that will execute concurrently for one run. By default
+ there is no limit- it will maximally parallel as allowed by the DAG. Note that this is not a
+ global limit.
+
+ Configuration set on the Kubernetes Jobs and Pods created by the K8sRunLauncher will also be
+ set on Kubernetes Jobs and Pods created by the k8s_job_executor.
+
+ Configuration set using tags on a @job will only apply to the run level. For configuration
+ to apply at each step it must be set using tags for each @op.
+
+
+
+
+
+
+
+
+## Ops
+
+
+
dagster_k8s.k8s_job_op `=` \
+
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ An op that runs a Kubernetes job using the k8s API.
+
+ Contrast with the k8s_job_executor, which runs each Dagster op in a Dagster job in its
+ own k8s job.
+
+ This op may be useful when:
+ - You need to orchestrate a command that isn’t a Dagster op (or isn’t written in Python)
+ - You want to run the rest of a Dagster job using a specific executor, and only a single
+
+
+ For example:
+
+ ```python
+ from dagster_k8s import k8s_job_op
+
+ from dagster import job
+
+ first_op = k8s_job_op.configured(
+ {
+ "image": "busybox",
+ "command": ["/bin/sh", "-c"],
+ "args": ["echo HELLO"],
+ },
+ name="first_op",
+ )
+ second_op = k8s_job_op.configured(
+ {
+ "image": "busybox",
+ "command": ["/bin/sh", "-c"],
+ "args": ["echo GOODBYE"],
+ },
+ name="second_op",
+ )
+
+ @job
+ def full_job():
+ second_op(first_op())
+ ```
+ You can create your own op with the same implementation by calling the execute_k8s_job function
+ inside your own op.
+
+ The service account that is used to run this job should have the following RBAC permissions:
+
+ ```YAML
+ rules:
+ - apiGroups: ["batch"]
+ resources: ["jobs", "jobs/status"]
+ verbs: ["*"]
+ # The empty arg "" corresponds to the core API group
+ - apiGroups: [""]
+ resources: ["pods", "pods/log", "pods/status"]
+ verbs: ["*"]'
+ ```
+
+
+
+
+
+
dagster_k8s.execute_k8s_job
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This function is a utility for executing a Kubernetes job from within a Dagster op.
+
+ Parameters:
+ - image (str) – The image in which to launch the k8s job.
+ - command (Optional[List[str]]) – The command to run in the container within the launched
+ - args (Optional[List[str]]) – The args for the command for the container. Default: None.
+ - namespace (Optional[str]) – Override the kubernetes namespace in which to run the k8s job.
+ - image_pull_policy (Optional[str]) – Allows the image pull policy to be overridden, e.g. to
+ - image_pull_secrets (Optional[List[Dict[str, str]]]) – Optionally, a list of dicts, each of
+ - service_account_name (Optional[str]) – The name of the Kubernetes service account under which
+ - env_secrets (Optional[List[str]]) – A list of custom Secret names from which to
+ - env_vars (Optional[List[str]]) – A list of environment variables to inject into the Job.
+ - volume_mounts (Optional[List[[*Permissive*](../config.mdx#dagster.Permissive)]]) – A list of volume mounts to include in the job’s
+ - volumes (Optional[List[[*Permissive*](../config.mdx#dagster.Permissive)]]) – A list of volumes to include in the Job’s Pod. Default: `[]`. See:
+ - labels (Optional[Dict[str, str]]) – Additional labels that should be included in the Job’s Pod. See:
+ - resources (Optional[Dict[str, Any]]) – [https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/](https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/)
+ - scheduler_name (Optional[str]) – Use a custom Kubernetes scheduler for launched Pods. See:
+ - load_incluster_config (bool) – Whether the op is running within a k8s cluster. If `True`,
+ - kubeconfig_file (Optional[str]) – The kubeconfig file from which to load config. Defaults to
+ - timeout (Optional[int]) – Raise an exception if the op takes longer than this timeout in
+ - container_config (Optional[Dict[str, Any]]) – Raw k8s config for the k8s pod’s main container
+ - pod_template_spec_metadata (Optional[Dict[str, Any]]) – Raw k8s config for the k8s pod’s
+ - pod_spec_config (Optional[Dict[str, Any]]) – Raw k8s config for the k8s pod’s pod spec
+ - job_metadata (Optional[Dict[str, Any]]) – Raw k8s config for the k8s job’s metadata
+ - job_spec_config (Optional[Dict[str, Any]]) – Raw k8s config for the k8s job’s job spec
+ - k8s_job_name (Optional[str]) – Overrides the name of the k8s job. If not set, will be set
+ - merge_behavior (Optional[K8sConfigMergeBehavior]) – How raw k8s config set on this op should
+ - delete_failed_k8s_jobs (bool) – Whether to immediately delete failed Kubernetes jobs. If False,
+
+
+
+
+
+
+
+
+
+### Python API
+
+The `K8sRunLauncher` allows webserver instances to be configured to launch new runs by starting
+per-run Kubernetes Jobs. To configure the `K8sRunLauncher`, your `dagster.yaml` should
+include a section like:
+
+ ```yaml
+ run_launcher:
+ module: dagster_k8s.launcher
+ class: K8sRunLauncher
+ config:
+ image_pull_secrets:
+ service_account_name: dagster
+ job_image: "my-company.com/image:latest"
+ dagster_home: "/opt/dagster/dagster_home"
+ postgres_password_secret: "dagster-postgresql-secret"
+ image_pull_policy: "IfNotPresent"
+ job_namespace: "dagster"
+ instance_config_map: "dagster-instance"
+ env_config_maps:
+ - "dagster-k8s-job-runner-env"
+ env_secrets:
+ - "dagster-k8s-some-secret"
+ ```
+
+
+
+
+
+
+### Helm chart
+
+For local dev (e.g., on kind or minikube):
+
+ ```shell
+ helm install \
+ --set dagsterWebserver.image.repository="dagster.io/buildkite-test-image" \
+ --set dagsterWebserver.image.tag="py310-latest" \
+ --set job_runner.image.repository="dagster.io/buildkite-test-image" \
+ --set job_runner.image.tag="py310-latest" \
+ --set imagePullPolicy="IfNotPresent" \
+ dagster \
+ helm/dagster/
+ ```
+Upon installation, the Helm chart will provide instructions for port forwarding
+the Dagster webserver and Flower (if configured).
+
+
+
+
+
+
+
+### Running tests
+
+To run the unit tests:
+
+ ```default
+ pytest -m "not integration"
+ ```
+To run the integration tests, you must have [Docker](https://docs.docker.com/install/),
+[kind](https://kind.sigs.k8s.io/docs/user/quick-start#installation),
+and [helm](https://helm.sh/docs/intro/install/) installed.
+
+On macOS:
+
+ ```default
+ brew install kind
+ brew install helm
+ ```
+Docker must be running.
+
+You may experience slow first test runs thanks to image pulls (run `pytest -svv --fulltrace` for
+visibility). Building images and loading them to the kind cluster is slow, and there is
+no visibility into the progress of the load.
+
+NOTE: This process is quite slow, as it requires bootstrapping a local `kind` cluster with
+Docker images and the `dagster-k8s` Helm chart. For faster development, you can either:
+
+1. Keep a warm kind cluster
+2. Use a remote K8s cluster, e.g. via AWS EKS or GCP GKE
+Instructions are below.
+
+
+
+
+#### Faster local development (with kind)
+
+You may find that the kind cluster creation, image loading, and kind cluster creation loop
+is too slow for effective local dev.
+
+You may bypass cluster creation and image loading in the following way. First add the `--no-cleanup`
+flag to your pytest invocation:
+
+ ```shell
+ pytest --no-cleanup -s -vvv -m "not integration"
+ ```
+The tests will run as before, but the kind cluster will be left running after the tests are completed.
+
+For subsequent test runs, you can run:
+
+ ```shell
+ pytest --kind-cluster="cluster-d9971c84d44d47f382a2928c8c161faa" --existing-helm-namespace="dagster-test-95590a" -s -vvv -m "not integration"
+ ```
+This will bypass cluster creation, image loading, and Helm chart installation, for much faster tests.
+
+The kind cluster name and Helm namespace for this command can be found in the logs, or retrieved
+via the respective CLIs, using `kind get clusters` and `kubectl get namespaces`. Note that
+for `kubectl` and `helm` to work correctly with a kind cluster, you should override your
+kubeconfig file location with:
+
+ ```shell
+ kind get kubeconfig --name kind-test > /tmp/kubeconfig
+ export KUBECONFIG=/tmp/kubeconfig
+ ```
+
+
+
+
+
+
+#### Manual kind cluster setup
+
+The test fixtures provided by `dagster-k8s` automate the process described below, but sometimes
+it’s useful to manually configure a kind cluster and load images onto it.
+
+First, ensure you have a Docker image appropriate for your Python version. Run, from the root of
+the repo:
+
+ ```shell
+ ./python_modules/dagster-test/dagster_test/test_project/build.sh 3.7.6 \
+ dagster.io.priv/buildkite-test-image:py310-latest
+ ```
+In the above invocation, the Python majmin version should be appropriate for your desired tests.
+
+Then run the following commands to create the cluster and load the image. Note that there is no
+feedback from the loading process.
+
+ ```shell
+ kind create cluster --name kind-test
+ kind load docker-image --name kind-test dagster.io/dagster-docker-buildkite:py310-latest
+ ```
+If you are deploying the Helm chart with an in-cluster Postgres (rather than an external database),
+and/or with dagster-celery workers (and a RabbitMQ), you’ll also want to have images present for
+rabbitmq and postgresql:
+
+ ```shell
+ docker pull docker.io/bitnami/rabbitmq
+ docker pull docker.io/bitnami/postgresql
+
+ kind load docker-image --name kind-test docker.io/bitnami/rabbitmq:latest
+ kind load docker-image --name kind-test docker.io/bitnami/postgresql:latest
+ ```
+Then you can run pytest as follows:
+
+ ```shell
+ pytest --kind-cluster=kind-test
+ ```
+
+
+
+
+
+
+### Faster local development (with an existing K8s cluster)
+
+If you already have a development K8s cluster available, you can run tests on that cluster vs.
+running locally in `kind`.
+
+For this to work, first build and deploy the test image to a registry available to your cluster.
+For example, with a private ECR repository:
+
+ ```default
+ ./python_modules/dagster-test/dagster_test/test_project/build.sh 3.7.6
+ docker tag dagster-docker-buildkite:latest $AWS_ACCOUNT_ID.dkr.ecr.us-west-2.amazonaws.com/dagster-k8s-tests:2020-04-21T21-04-06
+
+ aws ecr get-login --no-include-email --region us-west-1 | sh
+ docker push $AWS_ACCOUNT_ID.dkr.ecr.us-west-1.amazonaws.com/dagster-k8s-tests:2020-04-21T21-04-06
+ ```
+Then, you can run tests on EKS with:
+
+ ```default
+ export DAGSTER_DOCKER_IMAGE_TAG="2020-04-21T21-04-06"
+ export DAGSTER_DOCKER_REPOSITORY="$AWS_ACCOUNT_ID.dkr.ecr.us-west-2.amazonaws.com"
+ export DAGSTER_DOCKER_IMAGE="dagster-k8s-tests"
+
+ # First run with --no-cleanup to leave Helm chart in place
+ pytest --cluster-provider="kubeconfig" --no-cleanup -s -vvv
+
+ # Subsequent runs against existing Helm chart
+ pytest --cluster-provider="kubeconfig" --existing-helm-namespace="dagster-test-" -s -vvv
+ ```
+
+
+
+### A note about PVCs
+
+Both the Postgres and the RabbitMQ Helm charts will store credentials using Persistent Volume
+Claims, which will outlive test invocations and calls to `helm uninstall`. These must be deleted if
+you want to change credentials. To view your pvcs, run:
+
+ ```default
+ kubectl get pvc
+ ```
+
+
+
+
+
+
+### Testing Redis
+
+The Redis Helm chart installs w/ a randomly-generated password by default; turn this off:
+
+ ```default
+ helm install dagredis stable/redis --set usePassword=false
+ ```
+Then, to connect to your database from outside the cluster execute the following commands:
+
+ ```default
+ kubectl port-forward --namespace default svc/dagredis-master 6379:6379
+ redis-cli -h 127.0.0.1 -p 6379
+ ```
+
+
+
+
+
+## Pipes
+
+
+
class dagster_k8s.PipesK8sClient
+
+
+ A pipes client for launching kubernetes pods.
+
+ By default context is injected via environment variables and messages are parsed out of
+ the pod logs, with other logs forwarded to stdout of the orchestration process.
+
+ The first container within the containers list of the pod spec is expected (or set) to be
+ the container prepared for pipes protocol communication.
+
+ Parameters:
+ - env (Optional[Mapping[str, str]]) – An optional dict of environment variables to pass to the
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - load_incluster_config (Optional[bool]) – Whether this client is expected to be running from inside
+ - kubeconfig_file (Optional[str]) – The value to pass as the config_file argument to
+ - kube_context (Optional[str]) – The value to pass as the context argument to
+ - poll_interval (Optional[float]) – How many seconds to wait between requests when
+
+
+
+
run
+
+
+ Publish a kubernetes pod and wait for it to complete, enriched with the pipes protocol.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The execution context.
+ - image (Optional[str]) – The image to set the first container in the pod spec to use.
+ - command (Optional[Union[str, Sequence[str]]]) – The command to set the first container in the pod spec to use.
+ - namespace (Optional[str]) – Which kubernetes namespace to use, defaults to the current namespace if
+ - env (Optional[Mapping[str,str]]) – A mapping of environment variable names to values to set on the first
+ - base_pod_meta (Optional[Mapping[str, Any]]) – Raw k8s config for the k8s pod’s metadata
+ - base_pod_spec (Optional[Mapping[str, Any]]) – Raw k8s config for the k8s pod’s pod spec
+ - extras (Optional[PipesExtras]) – Extra values to pass along as part of the ext protocol.
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – Override the default ext protocol context injection.
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – Override the default ext protocol message reader.
+ - ignore_containers (Optional[Set]) – Ignore certain containers from waiting for termination. Defaults to
+ - enable_multi_container_logs (bool) – Whether or not to enable multi-container log consumption.
+
+
+ Returns:
+ Wrapper containing results reported by the external
+ process.
+
+ Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_k8s.PipesK8sPodLogsMessageReader
+
+ Message reader that reads messages from kubernetes pod logs.
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-looker.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-looker.mdx
new file mode 100644
index 0000000000000..43b31816fa5a1
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-looker.mdx
@@ -0,0 +1,511 @@
+---
+title: 'looker (dagster-looker)'
+title_meta: 'looker (dagster-looker) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'looker (dagster-looker) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Looker (dagster-looker)
+
+Dagster allows you to represent your Looker project as assets, alongside other your other
+technologies like dbt and Sling. This allows you to see how your Looker assets are connected to
+your other data assets, and how changes to other data assets might impact your Looker project.
+
+
+
+
+## Looker API
+
+Here, we provide interfaces to manage Looker projects using the Looker API.
+
+
+
+
+### Assets (Looker API)
+
+
+
class dagster_looker.LookerResource
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Represents a connection to a Looker instance and provides methods
+ to interact with the Looker API.
+
+
+
build_defs
+
+
+ :::danger[deprecated]
+ This API will be removed in version 1.9.0.
+ Use dagster_looker.load_looker_asset_specs instead.
+
+ :::
+
+ Returns a Definitions object which will load structures from the Looker instance
+ and translate it into assets, using the provided translator.
+
+ Parameters:
+ - request_start_pdt_builds (Optional[Sequence[[*RequestStartPdtBuild*](#dagster_looker.RequestStartPdtBuild)]]) – A list of
+ - dagster_looker_translator (Optional[[*DagsterLookerApiTranslator*](#dagster_looker.DagsterLookerApiTranslator)]) – The translator to
+
+
+ Returns: A Definitions object which will contain return the Looker structures as assets.Return type: [Definitions](../definitions.mdx#dagster.Definitions)
+
+
+
+
+
+
+
+
+
+
class dagster_looker.DagsterLookerApiTranslator
+
+
+
+
get_asset_key
+
+
+ :::danger[deprecated]
+ This API will be removed in version 1.10.
+ Use `DagsterLookerApiTranslator.get_asset_spec().key` instead.
+
+ :::
+
+
+
+
+
+
+
get_asset_spec
+
+
+
+
+
+
+
+
+
+
+
class dagster_looker.LookerStructureData
+
+
+
+
+
+
+
class dagster_looker.LookerStructureType
+
+
+
+
+
+
+
class dagster_looker.RequestStartPdtBuild
+
+
+ A request to start a PDT build. See [https://developers.looker.com/api/explorer/4.0/types/DerivedTable/RequestStartPdtBuild?sdk=py](https://developers.looker.com/api/explorer/4.0/types/DerivedTable/RequestStartPdtBuild?sdk=py)
+ for documentation on all available fields.
+
+ Parameters:
+ - model_name – The model of the PDT to start building.
+ - view_name – The view name of the PDT to start building.
+ - force_rebuild – Force rebuild of required dependent PDTs, even if they are already materialized.
+ - force_full_incremental – Force involved incremental PDTs to fully re-materialize.
+ - workspace – Workspace in which to materialize selected PDT (‘dev’ or default ‘production’).
+ - source – The source of this request.
+
+
+
+
+
+
+
+
class dagster_looker.LookerFilter
+
+
+ Filters the set of Looker objects to fetch.
+
+ Parameters:
+ - dashboard_folders (Optional[List[List[str]]]) – A list of folder paths to fetch dashboards from.
+ - only_fetch_explores_used_in_dashboards (bool) – If True, only explores used in the fetched dashboards
+
+
+
+
+
+
+
+
dagster_looker.load_looker_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Looker structures.
+
+ Parameters:
+ - looker_resource ([*LookerResource*](#dagster_looker.LookerResource)) – The Looker resource to fetch assets from.
+ - dagster_looker_translator (Optional[Union[[*DagsterLookerApiTranslator*](#dagster_looker.DagsterLookerApiTranslator), Type[[*DagsterLookerApiTranslator*](#dagster_looker.DagsterLookerApiTranslator)]]]) – The translator to use to convert Looker structures into [`dagster.AssetSpec`](../assets.mdx#dagster.AssetSpec).
+
+
+ Returns: The set of AssetSpecs representing the Looker structures.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns the AssetsDefinitions of the executable assets for the given the list of refreshable PDTs.
+
+ Parameters:
+ - resource_key (str) – The resource key to use for the Looker resource.
+ - request_start_pdt_builds (Optional[Sequence[[*RequestStartPdtBuild*](#dagster_looker.RequestStartPdtBuild)]]) – A list of requests to start PDT builds.
+ - dagster_looker_translator (Optional[Union[[*DagsterLookerApiTranslator*](#dagster_looker.DagsterLookerApiTranslator), Type[[*DagsterLookerApiTranslator*](#dagster_looker.DagsterLookerApiTranslator)]]]) – The translator to use to convert Looker structures into [`dagster.AssetSpec`](../assets.mdx#dagster.AssetSpec).
+
+
+ Returns: The AssetsDefinitions of the executable assets for the given the list of refreshable PDTs.Return type: [AssetsDefinition](../assets.mdx#dagster.AssetsDefinition)
+
+
+
+
+
+
+
+
+
+
+## lkml (LookML)
+
+Here, we provide interfaces to manage Looker projects defined a set of locally accessible
+LookML files.
+
+
+
+### Assets (lkml)
+
+
+
dagster_looker.build_looker_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Build a list of asset specs from a set of Looker structures defined in a Looker project.
+
+ Parameters:
+ - project_dir (Path) – The path to the Looker project directory.
+ - dagster_looker_translator (Optional[DagsterLookerTranslator]) – Allows customizing how to
+
+
+ Examples:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import external_assets_from_specs
+ from dagster_looker import build_looker_asset_specs
+
+
+ looker_specs = build_looker_asset_specs(project_dir=Path("my_looker_project"))
+ looker_assets = external_assets_from_specs(looker_specs)
+ ```
+
+
+
+
+
+
class dagster_looker.DagsterLookerLkmlTranslator
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Holds a set of methods that derive Dagster asset definition metadata given a representation
+ of a LookML structure (dashboards, explores, views).
+
+ This class is exposed so that methods can be overridden to customize how Dagster asset metadata
+ is derived.
+
+
+
get_asset_key
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).key` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster asset key that represents the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide a custom asset key for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: The Dagster asset key that represents the LookML structure.Return type: [AssetKey](../assets.mdx#dagster.AssetKey)
+
+
+
+
+
+
get_asset_spec
+
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster asset spec that represents the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide a custom asset spec for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: The Dagster asset spec that represents the LookML structure.Return type: [AssetSpec](../assets.mdx#dagster.AssetSpec)
+
+
+
+
+
+
get_deps
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Iterate over `DagsterLookerLkmlTranslator.get_asset_spec(...).deps` to access `AssetDep.asset_key` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster dependencies of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide custom dependencies for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: The Dagster dependencies for the LookML structure.Return type: Sequence[[AssetKey](../assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
get_description
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).description` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster description of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide a custom description for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: The Dagster description for the LookML structure.Return type: Optional[str]
+
+
+
+
+
+
get_group_name
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).group_name` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster group name of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide a custom group name for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: A Dagster group name for the LookML structure.Return type: Optional[str]
+
+
+
+
+
+
get_metadata
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).metadata` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster metadata of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide custom metadata for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns:
+ A dictionary representing the Dagster metadata for the
+ LookML structure.
+
+ Return type: Optional[Mapping[str, Any]]
+
+
+
+
+
+
get_owners
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).owners` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster owners of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide custom owners for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: A sequence of Dagster owners for the LookML structure.Return type: Optional[Sequence[str]]
+
+
+
+
+
+
get_tags
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).tags` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster tags of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide custom tags for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns:
+ A dictionary representing the Dagster tags for the
+ LookML structure.
+
+ Return type: Optional[Mapping[str, str]]
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-mlflow.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-mlflow.mdx
new file mode 100644
index 0000000000000..d7d247e8e6be0
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-mlflow.mdx
@@ -0,0 +1,87 @@
+---
+title: 'mlflow (dagster-mlflow)'
+title_meta: 'mlflow (dagster-mlflow) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'mlflow (dagster-mlflow) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# MLflow (dagster-mlflow)
+
+
+
dagster_mlflow.mlflow_tracking ResourceDefinition
+
+
+
+
+ This resource initializes an MLflow run that’s used for all steps within a Dagster run.
+
+ This resource provides access to all of mlflow’s methods as well as the mlflow tracking client’s
+ methods.
+
+ Usage:
+
+ 1. Add the mlflow resource to any ops in which you want to invoke mlflow tracking APIs.
+ 2. Add the end_mlflow_on_run_finished hook to your job to end the MLflow run
+ when the Dagster run is finished.
+ Examples:
+
+ ```python
+ from dagster_mlflow import end_mlflow_on_run_finished, mlflow_tracking
+
+ @op(required_resource_keys={"mlflow"})
+ def mlflow_op(context):
+ mlflow.log_params(some_params)
+ mlflow.tracking.MlflowClient().create_registered_model(some_model_name)
+
+ @end_mlflow_on_run_finished
+ @job(resource_defs={"mlflow": mlflow_tracking})
+ def mlf_example():
+ mlflow_op()
+
+ # example using an mlflow instance with s3 storage
+ mlf_example.execute_in_process(run_config={
+ "resources": {
+ "mlflow": {
+ "config": {
+ "experiment_name": my_experiment,
+ "mlflow_tracking_uri": "http://localhost:5000",
+
+ # if want to run a nested run, provide parent_run_id
+ "parent_run_id": an_existing_mlflow_run_id,
+
+ # if you want to resume a run or avoid creating a new run in the resource init,
+ # provide mlflow_run_id
+ "mlflow_run_id": an_existing_mlflow_run_id,
+
+ # env variables to pass to mlflow
+ "env": {
+ "MLFLOW_S3_ENDPOINT_URL": my_s3_endpoint,
+ "AWS_ACCESS_KEY_ID": my_aws_key_id,
+ "AWS_SECRET_ACCESS_KEY": my_secret,
+ },
+
+ # env variables you want to log as mlflow tags
+ "env_to_tag": ["DOCKER_IMAGE_TAG"],
+
+ # key-value tags to add to your experiment
+ "extra_tags": {"super": "experiment"},
+ }
+ }
+ }
+ })
+ ```
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-msteams.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-msteams.mdx
new file mode 100644
index 0000000000000..f346553f84e61
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-msteams.mdx
@@ -0,0 +1,243 @@
+---
+title: 'microsoft teams (dagster-msteams)'
+title_meta: 'microsoft teams (dagster-msteams) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'microsoft teams (dagster-msteams) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+ This resource is for connecting to Microsoft Teams.
+
+ Provides a dagster_msteams.TeamsClient which can be used to
+ interface with the MS Teams API.
+
+ By configuring this resource, you can post messages to MS Teams from any Dagster op,
+ asset, schedule, or sensor:
+
+ Examples:
+
+ ```python
+ import os
+
+ from dagster import op, job, Definitions, EnvVar
+ from dagster_msteams import Card, MSTeamsResource
+
+
+ @op
+ def teams_op(msteams: MSTeamsResource):
+ card = Card()
+ card.add_attachment(text_message="Hello There !!")
+ msteams.get_client().post_message(payload=card.payload)
+
+
+ @job
+ def teams_job():
+ teams_op()
+
+ defs = Definitions(
+ jobs=[teams_job],
+ resources={
+ "msteams": MSTeamsResource(
+ hook_url=EnvVar("TEAMS_WEBHOOK_URL")
+ )
+ }
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+## Sensors
+
+
+
dagster_msteams.teams_on_failure HookDefinition
+
+
+ Create a hook on step failure events that will message the given MS Teams webhook URL.
+
+ Parameters:
+ - message_fn (Optional(Callable[[[*HookContext*](../hooks.mdx#dagster.HookContext)], str])) – Function which takes in the
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your webserver instance. Specify this
+ - webserver_base_url – (Optional[str]): The base url of your webserver instance. Specify this
+
+
+ Examples:
+
+ ```python
+ @teams_on_failure(webserver_base_url="http://localhost:3000")
+ @job(...)
+ def my_job():
+ pass
+ ```
+ ```python
+ def my_message_fn(context: HookContext) -> str:
+ return f"Op {context.op.name} failed!"
+
+ @op
+ def a_op(context):
+ pass
+
+ @job(...)
+ def my_job():
+ a_op.with_hooks(hook_defs={teams_on_failure("#foo", my_message_fn)})
+ ```
+
+
+
+
+
+
dagster_msteams.teams_on_success HookDefinition
+
+
+ Create a hook on step success events that will message the given MS Teams webhook URL.
+
+ Parameters:
+ - message_fn (Optional(Callable[[[*HookContext*](../hooks.mdx#dagster.HookContext)], str])) – Function which takes in the
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your webserver instance. Specify this
+
+
+ Examples:
+
+ ```python
+ @teams_on_success(webserver_base_url="http://localhost:3000")
+ @job(...)
+ def my_job():
+ pass
+ ```
+ ```python
+ def my_message_fn(context: HookContext) -> str:
+ return f"Op {context.op.name} failed!"
+
+ @op
+ def a_op(context):
+ pass
+
+ @job(...)
+ def my_job():
+ a_op.with_hooks(hook_defs={teams_on_success("#foo", my_message_fn)})
+ ```
+
+
+
+
+
+
dagster_msteams.make_teams_on_run_failure_sensor
+
+
+ Create a sensor on run failures that will message the given MS Teams webhook URL.
+
+ Parameters:
+ - hook_url (str) – MS Teams incoming webhook URL.
+ - message_fn (Optional(Callable[[[*RunFailureSensorContext*](../schedules-sensors.mdx#dagster.RunFailureSensorContext)], str])) – Function which
+ - http_proxy – (Optional[str]): Proxy for requests using http protocol.
+ - https_proxy – (Optional[str]): Proxy for requests using https protocol.
+ - timeout – (Optional[float]): Connection timeout in seconds. Defaults to 60.
+ - verify – (Optional[bool]): Whether to verify the servers TLS certificate.
+ - name – (Optional[str]): The name of the sensor. Defaults to “teams_on_run_failure”.
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your webserver instance. Specify this to allow
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](../jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](../graphs.mdx#dagster.GraphDefinition), UnresolvedAssetJobDefinition, [*RepositorySelector*](../schedules-sensors.mdx#dagster.RepositorySelector), [*JobSelector*](../schedules-sensors.mdx#dagster.JobSelector)]]]) – Jobs in the current repository that will be monitored by this sensor. Defaults to None,
+ - monitor_all_code_locations (bool) – If set to True, the sensor will monitor all runs in the
+ - webserver_base_url – (Optional[str]): The base url of your webserver instance. Specify this to allow
+ - monitor_all_repositories (bool) – deprecatedmonitor_all_code_locations instead.) If set to True, the sensor will monitor all runs in the
+
+
+ Examples:
+
+ ```python
+ teams_on_run_failure = make_teams_on_run_failure_sensor(
+ hook_url=os.getenv("TEAMS_WEBHOOK_URL")
+ )
+
+ @repository
+ def my_repo():
+ return [my_job + teams_on_run_failure]
+ ```
+ ```python
+ def my_message_fn(context: RunFailureSensorContext) -> str:
+ return "Job {job_name} failed! Error: {error}".format(
+ job_name=context.dagster_run.job_name,
+ error=context.failure_event.message,
+ )
+
+ teams_on_run_failure = make_teams_on_run_failure_sensor(
+ hook_url=os.getenv("TEAMS_WEBHOOK_URL"),
+ message_fn=my_message_fn,
+ webserver_base_url="http://localhost:3000",
+ )
+ ```
+
+
+
+ This resource is for connecting to Microsoft Teams.
+
+ The resource object is a dagster_msteams.TeamsClient.
+
+ By configuring this resource, you can post messages to MS Teams from any Dagster solid:
+
+ Examples:
+
+ ```python
+ import os
+
+ from dagster import op, job
+ from dagster_msteams import Card, msteams_resource
+
+
+ @op(required_resource_keys={"msteams"})
+ def teams_op(context):
+ card = Card()
+ card.add_attachment(text_message="Hello There !!")
+ context.resources.msteams.post_message(payload=card.payload)
+
+
+ @job(resource_defs={"msteams": msteams_resource})
+ def teams_job():
+ teams_op()
+
+
+ teams_job.execute_in_process(
+ {"resources": {"msteams": {"config": {"hook_url": os.getenv("TEAMS_WEBHOOK_URL")}}}}
+ )
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-mysql.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-mysql.mdx
new file mode 100644
index 0000000000000..6a03098bda46f
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-mysql.mdx
@@ -0,0 +1,112 @@
+---
+title: 'mysql (dagster-mysql)'
+title_meta: 'mysql (dagster-mysql) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'mysql (dagster-mysql) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# MySQL (dagster-mysql)
+
+
+
class dagster_mysql.MySQLEventLogStorage
+
+
+ MySQL-backed event log storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ dagster.yaml
+
+ ```YAML
+
+ event_log_storage:
+ module: dagster_mysql.event_log
+ class: MySQLEventLogStorage
+ config:
+ mysql_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { db_name }
+ port: { port }
+
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
+
class dagster_mysql.MySQLRunStorage
+
+
+ MySQL-backed run storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ dagster.yaml
+
+ ```YAML
+
+ run_storage:
+ module: dagster_mysql.run_storage
+ class: MySQLRunStorage
+ config:
+ mysql_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { database }
+ port: { port }
+
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
+
class dagster_mysql.MySQLScheduleStorage
+
+
+ MySQL-backed run storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ dagster.yaml
+
+ ```YAML
+
+ schedule_storage:
+ module: dagster_mysql.schedule_storage
+ class: MySQLScheduleStorage
+ config:
+ mysql_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { db_name }
+ port: { port }
+
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-openai.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-openai.mdx
new file mode 100644
index 0000000000000..18e537b8ef7ca
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-openai.mdx
@@ -0,0 +1,284 @@
+---
+title: 'openai (dagster-openai)'
+title_meta: 'openai (dagster-openai) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'openai (dagster-openai) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# OpenAI (dagster-openai)
+
+The dagster_openai library provides utilities for using OpenAI with Dagster.
+A good place to start with dagster_openai is [the guide](https://docs.dagster.io/integrations/libraries/openai/).
+
+
+
dagster_openai.with_usage_metadata
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This wrapper can be used on any endpoint of the
+ openai library \
+ to log the OpenAI API usage metadata in the asset metadata.
+
+ Examples:
+
+ ```python
+ from dagster import (
+ AssetExecutionContext,
+ AssetKey,
+ AssetSelection,
+ AssetSpec,
+ Definitions,
+ EnvVar,
+ MaterializeResult,
+ asset,
+ define_asset_job,
+ multi_asset,
+ )
+ from dagster_openai import OpenAIResource, with_usage_metadata
+
+
+ @asset(compute_kind="OpenAI")
+ def openai_asset(context: AssetExecutionContext, openai: OpenAIResource):
+ with openai.get_client(context) as client:
+ client.fine_tuning.jobs.create = with_usage_metadata(
+ context=context, output_name="some_output_name", func=client.fine_tuning.jobs.create
+ )
+ client.fine_tuning.jobs.create(model="gpt-3.5-turbo", training_file="some_training_file")
+
+
+ openai_asset_job = define_asset_job(name="openai_asset_job", selection="openai_asset")
+
+
+ @multi_asset(
+ specs=[
+ AssetSpec("my_asset1"),
+ AssetSpec("my_asset2"),
+ ]
+ )
+ def openai_multi_asset(context: AssetExecutionContext, openai: OpenAIResource):
+ with openai.get_client(context, asset_key=AssetKey("my_asset1")) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo", messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+
+ # The materialization of `my_asset1` will include both OpenAI usage metadata
+ # and the metadata added when calling `MaterializeResult`.
+ return (
+ MaterializeResult(asset_key="my_asset1", metadata={"foo": "bar"}),
+ MaterializeResult(asset_key="my_asset2", metadata={"baz": "qux"}),
+ )
+
+
+ openai_multi_asset_job = define_asset_job(
+ name="openai_multi_asset_job", selection=AssetSelection.assets(openai_multi_asset)
+ )
+
+
+ defs = Definitions(
+ assets=[openai_asset, openai_multi_asset],
+ jobs=[openai_asset_job, openai_multi_asset_job],
+ resources={
+ "openai": OpenAIResource(api_key=EnvVar("OPENAI_API_KEY")),
+ },
+ )
+ ```
+
+
+
+
+
+
class dagster_openai.OpenAIResource
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This resource is wrapper over the
+ [openai library](https://github.com/openai/openai-python).
+
+ By configuring this OpenAI resource, you can interact with OpenAI API
+ and log its usage metadata in the asset metadata.
+
+ Examples:
+
+ ```python
+ import os
+
+ from dagster import AssetExecutionContext, Definitions, EnvVar, asset, define_asset_job
+ from dagster_openai import OpenAIResource
+
+
+ @asset(compute_kind="OpenAI")
+ def openai_asset(context: AssetExecutionContext, openai: OpenAIResource):
+ with openai.get_client(context) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo",
+ messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+
+ openai_asset_job = define_asset_job(name="openai_asset_job", selection="openai_asset")
+
+ defs = Definitions(
+ assets=[openai_asset],
+ jobs=[openai_asset_job],
+ resources={
+ "openai": OpenAIResource(api_key=EnvVar("OPENAI_API_KEY")),
+ },
+ )
+ ```
+
+
get_client
+
+
+ Yields an `openai.Client` for interacting with the OpenAI API.
+
+ By default, in an asset context, the client comes with wrapped endpoints
+ for three API resources, Completions, Embeddings and Chat,
+ allowing to log the API usage metadata in the asset metadata.
+
+ Note that the endpoints are not and cannot be wrapped
+ to automatically capture the API usage metadata in an op context.
+
+ Parameters: context – The `context` object for computing the op or asset in which `get_client` is called.
+ Examples:
+
+ ```python
+ from dagster import (
+ AssetExecutionContext,
+ Definitions,
+ EnvVar,
+ GraphDefinition,
+ OpExecutionContext,
+ asset,
+ define_asset_job,
+ op,
+ )
+ from dagster_openai import OpenAIResource
+
+
+ @op
+ def openai_op(context: OpExecutionContext, openai: OpenAIResource):
+ with openai.get_client(context) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo", messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+
+
+ openai_op_job = GraphDefinition(name="openai_op_job", node_defs=[openai_op]).to_job()
+
+
+ @asset(compute_kind="OpenAI")
+ def openai_asset(context: AssetExecutionContext, openai: OpenAIResource):
+ with openai.get_client(context) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo", messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+
+
+ openai_asset_job = define_asset_job(name="openai_asset_job", selection="openai_asset")
+
+ defs = Definitions(
+ assets=[openai_asset],
+ jobs=[openai_asset_job, openai_op_job],
+ resources={
+ "openai": OpenAIResource(api_key=EnvVar("OPENAI_API_KEY")),
+ },
+ )
+ ```
+
+
+
+
+
+
get_client_for_asset
+
+
+ Yields an `openai.Client` for interacting with the OpenAI.
+
+ When using this method, the OpenAI API usage metadata is automatically
+ logged in the asset materializations associated with the provided `asset_key`.
+
+ By default, the client comes with wrapped endpoints
+ for three API resources, Completions, Embeddings and Chat,
+ allowing to log the API usage metadata in the asset metadata.
+
+ This method can only be called when working with assets,
+ i.e. the provided `context` must be of type `AssetExecutionContext`.
+
+ Parameters:
+ - context – The `context` object for computing the asset in which `get_client` is called.
+ - asset_key – the `asset_key` of the asset for which a materialization should include the metadata.
+
+
+ Examples:
+
+ ```python
+ from dagster import (
+ AssetExecutionContext,
+ AssetKey,
+ AssetSpec,
+ Definitions,
+ EnvVar,
+ MaterializeResult,
+ asset,
+ define_asset_job,
+ multi_asset,
+ )
+ from dagster_openai import OpenAIResource
+
+
+ @asset(compute_kind="OpenAI")
+ def openai_asset(context: AssetExecutionContext, openai: OpenAIResource):
+ with openai.get_client_for_asset(context, context.asset_key) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo", messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+
+
+ openai_asset_job = define_asset_job(name="openai_asset_job", selection="openai_asset")
+
+
+ @multi_asset(specs=[AssetSpec("my_asset1"), AssetSpec("my_asset2")], compute_kind="OpenAI")
+ def openai_multi_asset(context: AssetExecutionContext, openai_resource: OpenAIResource):
+ with openai_resource.get_client_for_asset(context, asset_key=AssetKey("my_asset1")) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo", messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+ return (
+ MaterializeResult(asset_key="my_asset1", metadata={"some_key": "some_value1"}),
+ MaterializeResult(asset_key="my_asset2", metadata={"some_key": "some_value2"}),
+ )
+
+
+ openai_multi_asset_job = define_asset_job(
+ name="openai_multi_asset_job", selection="openai_multi_asset"
+ )
+
+ defs = Definitions(
+ assets=[openai_asset, openai_multi_asset],
+ jobs=[openai_asset_job, openai_multi_asset_job],
+ resources={
+ "openai": OpenAIResource(api_key=EnvVar("OPENAI_API_KEY")),
+ },
+ )
+ ```
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-pagerduty.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-pagerduty.mdx
new file mode 100644
index 0000000000000..335bc87a41c7f
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-pagerduty.mdx
@@ -0,0 +1,95 @@
+---
+title: 'pagerduty (dagster-pagerduty)'
+title_meta: 'pagerduty (dagster-pagerduty) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'pagerduty (dagster-pagerduty) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# PagerDuty (dagster-pagerduty)
+
+This library provides an integration with PagerDuty, to support creating alerts from your Dagster
+code.
+
+Presently, it provides a thin wrapper on the [Events API V2](https://v2.developer.pagerduty.com/docs/events-api-v2).
+
+
+
+
+
+
+
+# Getting Started
+
+You can install this library with:
+
+ ```default
+ pip install dagster-pagerduty
+ ```
+To use this integration, you’ll first need to create an Events API V2 PagerDuty integration on a PagerDuty service. There are instructions
+[here](https://support.pagerduty.com/docs/services-and-integrations#section-events-api-v2) for
+creating a new PagerDuty service & integration.
+
+Once your Events API V2 integration is set up, you’ll find an Integration Key (also referred to as a
+“Routing Key”) on the Integrations tab for your service. This key is used to authorize events
+created from the PagerDuty events API.
+
+Once your service/integration is created, you can provision a PagerDuty resource and issue PagerDuty
+alerts from within your ops.
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-pandas.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-pandas.mdx
new file mode 100644
index 0000000000000..040507ed2c8f8
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-pandas.mdx
@@ -0,0 +1,114 @@
+---
+title: 'pandas (dagster-pandas)'
+title_meta: 'pandas (dagster-pandas) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'pandas (dagster-pandas) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Pandas (dagster-pandas)
+
+The dagster_pandas library provides utilities for using pandas with Dagster and for implementing
+validation on pandas DataFrames. A good place to start with dagster_pandas is the [validation
+guide](https://docs.dagster.io/integrations/libraries/pandas/).
+
+
+
+ Constructs a custom pandas dataframe dagster type.
+
+ Parameters:
+ - name (str) – Name of the dagster pandas type.
+ - description (Optional[str]) – A markdown-formatted string, displayed in tooling.
+ - columns (Optional[List[[*PandasColumn*](#dagster_pandas.PandasColumn)]]) – A list of `PandasColumn` objects
+ - metadata_fn (Optional[Callable[[], Union[Dict[str, Union[str, float, int, Dict, [*MetadataValue*](../metadata.mdx#dagster.MetadataValue)]]) – A callable which takes your dataframe and returns a dict with string label keys and
+ - dataframe_constraints (Optional[List[DataFrameConstraint]]) – A list of objects that inherit from
+ - loader (Optional[[*DagsterTypeLoader*](../types.mdx#dagster.DagsterTypeLoader)]) – An instance of a class that
+
+
+
+
+
+
+
+
class dagster_pandas.RowCountConstraint
+
+
+ A dataframe constraint that validates the expected count of rows.
+
+ Parameters:
+ - num_allowed_rows (int) – The number of allowed rows in your dataframe.
+ - error_tolerance (Optional[int]) – The acceptable threshold if you are not completely certain. Defaults to 0.
+
+
+
+
+
+
+
+
class dagster_pandas.StrictColumnsConstraint
+
+
+ A dataframe constraint that validates column existence and ordering.
+
+ Parameters:
+ - strict_column_list (List[str]) – The exact list of columns that your dataframe must have.
+ - enforce_ordering (Optional[bool]) – If true, will enforce that the ordering of column names must match.
+
+
+
+
+
+
+
+
class dagster_pandas.PandasColumn
+
+
+ The main API for expressing column level schemas and constraints for your custom dataframe
+ types.
+
+ Parameters:
+ - name (str) – Name of the column. This must match up with the column name in the dataframe you
+ - is_required (Optional[bool]) – Flag indicating the optional/required presence of the column.
+ - constraints (Optional[List[Constraint]]) – List of constraint objects that indicate the
+
+
+
+
+
+
+
+
dagster_pandas.DataFrame `=` \
+
+
+ Define a type in dagster. These can be used in the inputs and outputs of ops.
+
+ Parameters:
+ - type_check_fn (Callable[[[*TypeCheckContext*](../execution.mdx#dagster.TypeCheckContext), Any], [Union[bool, [*TypeCheck*](../ops.mdx#dagster.TypeCheck)]]]) – The function that defines the type check. It takes the value flowing
+ - key (Optional[str]) –
+
+ The unique key to identify types programmatically.
+ The key property always has a value. If you omit key to the argument
+ to the init function, it instead receives the value of `name`. If
+ neither `key` nor `name` is provided, a `CheckError` is thrown.
+
+ In the case of a generic type such as `List` or `Optional`, this is
+ generated programmatically based on the type parameters.
+
+ - name (Optional[str]) – A unique name given by a user. If `key` is `None`, `key`
+ - description (Optional[str]) – A markdown-formatted string, displayed in tooling.
+ - loader (Optional[[*DagsterTypeLoader*](../types.mdx#dagster.DagsterTypeLoader)]) – An instance of a class that
+ - required_resource_keys (Optional[Set[str]]) – Resource keys required by the `type_check_fn`.
+ - is_builtin (bool) – Defaults to False. This is used by tools to display or
+ - kind (DagsterTypeKind) – Defaults to None. This is used to determine the kind of runtime type
+ - typing_type – Defaults to None. A valid python typing type (e.g. Optional[List[int]]) for the
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-pandera.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-pandera.mdx
new file mode 100644
index 0000000000000..b4ff01bbad00a
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-pandera.mdx
@@ -0,0 +1,49 @@
+---
+title: 'pandera (dagster-pandera)'
+title_meta: 'pandera (dagster-pandera) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'pandera (dagster-pandera) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Pandera (dagster-pandera)
+
+The dagster_pandera library allows Dagster users to use dataframe validation library [Pandera](https://github.com/pandera-dev/pandera) for the validation of Pandas dataframes. See [the guide](https://docs.dagster.io/integrations/libraries/pandera/) for details.
+
+
+
dagster_pandera.pandera_schema_to_dagster_type
+
+
+ Convert a Pandera dataframe schema to a DagsterType.
+
+ The generated Dagster type will be given an automatically generated name. The schema’s title
+ property, name property, or class name (in that order) will be used. If neither title or
+ name is defined, a name of the form DagsterPanderaDataframe\ is generated.
+
+ Additional metadata is also extracted from the Pandera schema and attached to the returned
+ DagsterType as a metadata dictionary. The extracted metadata includes:
+
+ - Descriptions on the schema and constituent columns and checks.
+ - Data types for each column.
+ - String representations of all column-wise checks.
+ - String representations of all row-wise (i.e. “wide”) checks.
+
+
+ The returned DagsterType type will call the Pandera schema’s validate() method in its type
+ check function. Validation is done in lazy mode, i.e. pandera will attempt to validate all
+ values in the dataframe, rather than stopping on the first error.
+
+ If validation fails, the returned TypeCheck object will contain two pieces of metadata:
+
+ - num_failures total number of validation errors.
+ - failure_sample a table containing up to the first 10 validation errors.
+
+
+ Parameters: schema (Union[pa.DataFrameSchema, Type[pa.DataFrameModel]])Returns: Dagster Type constructed from the Pandera schema.Return type: [DagsterType](../types.mdx#dagster.DagsterType)
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-papertrail.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-papertrail.mdx
new file mode 100644
index 0000000000000..a0b6559e67613
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-papertrail.mdx
@@ -0,0 +1,37 @@
+---
+title: 'papertrail (dagster-papertrail)'
+title_meta: 'papertrail (dagster-papertrail) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'papertrail (dagster-papertrail) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Papertrail (dagster-papertrail)
+
+This library provides an integration with [Papertrail](https://papertrailapp.com) for logging.
+
+You can easily set up your Dagster job to log to Papertrail. You’ll need an active Papertrail
+account, and have your papertrail URL and port handy.
+
+
+
+ Core class for defining loggers.
+
+ Loggers are job-scoped logging handlers, which will be automatically invoked whenever
+ dagster messages are logged from within a job.
+
+ Parameters:
+ - logger_fn (Callable[[[*InitLoggerContext*](../loggers.mdx#dagster.InitLoggerContext)], logging.Logger]) – User-provided function to
+ - config_schema (Optional[[*ConfigSchema*](../config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of this logger.
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-pipes.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-pipes.mdx
new file mode 100644
index 0000000000000..e81c8d56333b3
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-pipes.mdx
@@ -0,0 +1,1014 @@
+---
+title: 'pipes (dagster-pipes)'
+title_meta: 'pipes (dagster-pipes) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'pipes (dagster-pipes) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Pipes (dagster-pipes)
+
+The `dagster-pipes` library is intended for inclusion in an external process that integrates with Dagster using the [Pipes](https://docs.dagster.io/guides/build/external-pipelines/) protocol. This could be in an environment like Databricks, Kubernetes, or Docker. Using this library, you can write code in the external process that streams metadata back to Dagster.
+
+For a detailed look at the Pipes process, including how to customize it, refer to the [Dagster Pipes details and customization guide](https://docs.dagster.io/guides/build/external-pipelines/dagster-pipes-details-and-customization).
+
+Looking to set up a Pipes client in Dagster? Refer to the [Dagster Pipes API reference](https://docs.dagster.io/api/python-api/libraries/dagster-pipes).
+
+Note: This library isn’t included with `dagster` and must be [installed separately](https://pypi.org/project/dagster-pipes/).
+
+
+
+
+## Context
+
+
+
dagster_pipes.open_dagster_pipes
+
+
+ Initialize the Dagster Pipes context.
+
+ This function should be called near the entry point of a pipes process. It will load injected
+ context information from Dagster and spin up the machinery for streaming messages back to
+ Dagster.
+
+ If the process was not launched by Dagster, this function will emit a warning and return a
+ MagicMock object. This should make all operations on the context no-ops and prevent your code
+ from crashing.
+
+ Parameters:
+ - context_loader (Optional[[*PipesContextLoader*](#dagster_pipes.PipesContextLoader)]) – The context loader to use. Defaults to
+ - message_writer (Optional[[*PipesMessageWriter*](#dagster_pipes.PipesMessageWriter)]) – The message writer to use. Defaults to
+ - params_loader (Optional[[*PipesParamsLoader*](#dagster_pipes.PipesParamsLoader)]) – The params loader to use. Defaults to
+
+
+ Returns: The initialized context.Return type: [PipesContext](#dagster_pipes.PipesContext)
+
+
+
+
+
+
class dagster_pipes.PipesContext
+
+
+ The context for a Dagster Pipes process.
+
+ This class is analogous to [`OpExecutionContext`](../execution.mdx#dagster.OpExecutionContext) on the Dagster side of the Pipes
+ connection. It provides access to information such as the asset key(s) and partition key(s) in
+ scope for the current step. It also provides methods for logging and emitting results that will
+ be streamed back to Dagster.
+
+ This class should not be directly instantiated by the user. Instead it should be initialized by
+ calling [`open_dagster_pipes()`](#dagster_pipes.open_dagster_pipes), which will return the singleton instance of this class.
+ After open_dagster_pipes() has been called, the singleton instance can also be retrieved by
+ calling [`PipesContext.get()`](#dagster_pipes.PipesContext.get).
+
+
+
close
+
+ Close the pipes connection. This will flush all buffered messages to the orchestration
+ process and cause any further attempt to write a message to raise an error. This method is
+ idempotent– subsequent calls after the first have no effect.
+
+
+
+
+
classmethod get
+
+ Get the singleton instance of the context. Raises an error if the context has not been initialized.
+
+
+
+
+
get_extra
+
+
+ Get the value of an extra provided by the user. Raises an error if the extra is not defined.
+
+ Parameters: key (str) – The key of the extra.Returns: The value of the extra.Return type: Any
+
+
+
+
+
+
classmethod is_initialized
+
+ bool: Whether the context has been initialized.
+
+
+
+
+
log_external_stream
+
+
+
+
+
+
+
report_asset_check
+
+
+ Report to Dagster that an asset check has been performed. Streams a payload containing
+ check result information back to Dagster. If no assets or associated checks are in scope, raises an error.
+
+ Parameters:
+ - check_name (str) – The name of the check.
+ - passed (bool) – Whether the check passed.
+ - severity (PipesAssetCheckSeverity) – The severity of the check. Defaults to “ERROR”.
+ - metadata (Optional[Mapping[str, Union[PipesMetadataRawValue, PipesMetadataValue]]]) – Metadata for the check. Defaults to None.
+ - asset_key (Optional[str]) – The asset key for the check. If only a single asset is in
+
+
+
+
+
+
+
+
report_asset_materialization
+
+
+ Report to Dagster that an asset has been materialized. Streams a payload containing
+ materialization information back to Dagster. If no assets are in scope, raises an error.
+
+ Parameters:
+ - metadata (Optional[Mapping[str, Union[PipesMetadataRawValue, PipesMetadataValue]]]) – Metadata for the materialized asset. Defaults to None.
+ - data_version (Optional[str]) – The data version for the materialized asset.
+ - asset_key (Optional[str]) – The asset key for the materialized asset. If only a
+
+
+
+
+
+
+
+
report_custom_message
+
+
+ Send a JSON serializable payload back to the orchestration process. Can be retrieved there
+ using get_custom_messages.
+
+ Parameters: payload (Any) – JSON serializable data.
+
+
+
+
+
+
classmethod set
+
+ Set the singleton instance of the context.
+
+
+
+
+
property asset_key
+
+
+ The AssetKey for the currently scoped asset. Raises an error if 0 or multiple assets
+ are in scope.
+
+ Type: str
+
+
+
+
+
+
property asset_keys
+
+
+ The AssetKeys for the currently scoped assets. Raises an error if no
+ assets are in scope.
+
+ Type: Sequence[str]
+
+
+
+
+
+
property code_version
+
+
+ The code version for the currently scoped asset. Raises an error if 0 or
+ multiple assets are in scope.
+
+ Type: Optional[str]
+
+
+
+
+
+
property code_version_by_asset_key
+
+
+ Mapping of asset key to code version for the currently
+ scoped assets. Raises an error if no assets are in scope.
+
+ Type: Mapping[str, Optional[str]]
+
+
+
+
+
+
property extras
+
+
+ Key-value map for all extras provided by the user.
+
+ Type: Mapping[str, Any]
+
+
+
+
+
+
property is_asset_step
+
+
+ Whether the current step targets assets.
+
+ Type: bool
+
+
+
+
+
+
property is_closed
+
+
+ Whether the context has been closed.
+
+ Type: bool
+
+
+
+
+
+
property is_partition_step
+
+
+ Whether the current step is scoped to one or more partitions.
+
+ Type: bool
+
+
+
+
+
+
property job_name
+
+
+ The job name for the currently executing run. Returns None if the run is
+ not derived from a job.
+
+ Type: Optional[str]
+
+
+
+
+
+
property log
+
+
+ A logger that streams log messages back to Dagster.
+
+ Type: logging.Logger
+
+
+
+
+
+
property partition_key
+
+
+ The partition key for the currently scoped partition. Raises an error if 0 or
+ multiple partitions are in scope.
+
+ Type: str
+
+
+
+
+
+
property partition_key_range
+
+
+ The partition key range for the currently scoped partition or
+ partitions. Raises an error if no partitions are in scope.
+
+ Type: PipesPartitionKeyRange
+
+
+
+
+
+
property partition_time_window
+
+
+ The partition time window for the currently scoped partition
+ or partitions. Returns None if partitions in scope are not temporal. Raises an error if no
+ partitions are in scope.
+
+ Type: Optional[PipesTimeWindow]
+
+
+
+
+
+
property provenance
+
+
+ The provenance for the currently scoped asset. Raises an
+ error if 0 or multiple assets are in scope.
+
+ Type: Optional[PipesDataProvenance]
+
+
+
+
+
+
property provenance_by_asset_key
+
+
+ Mapping of asset key to provenance for the
+ currently scoped assets. Raises an error if no assets are in scope.
+
+ Type: Mapping[str, Optional[PipesDataProvenance]]
+
+
+
+
+
+
property retry_number
+
+
+ The retry number for the currently executing run.
+
+ Type: int
+
+
+
+
+
+
property run_id
+
+
+ The run ID for the currently executing pipeline run.
+
+ Type: str
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Advanced
+
+Most Pipes users won’t need to use the APIs in the following sections unless they are customizing the Pipes protocol.
+
+Refer to the [Dagster Pipes details and customization guide](https://docs.dagster.io/guides/build/external-pipelines/dagster-pipes-details-and-customization) for more information.
+
+
+
+
+### Context loaders
+
+Context loaders load the context payload from the location specified in the bootstrap payload.
+
+
+
class dagster_pipes.PipesContextLoader
+
+
+
+
abstract load_context
+
+
+ A @contextmanager that loads context data injected by the orchestration process.
+
+ This method should read and yield the context data from the location specified by the passed in
+ PipesParams.
+
+ Parameters: params (PipesParams) – The params provided by the context injector in the orchestration
+ process.Yields: PipesContextData – The context data.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesDefaultContextLoader
+
+
+ Context loader that loads context data from either a file or directly from the provided params.
+
+ The location of the context data is configured by the params received by the loader. If the params
+ include a key path, then the context data will be loaded from a file at the specified path. If
+ the params instead include a key data, then the corresponding value should be a dict
+ representing the context data.
+
+
+
load_context
+
+
+ A @contextmanager that loads context data injected by the orchestration process.
+
+ This method should read and yield the context data from the location specified by the passed in
+ PipesParams.
+
+ Parameters: params (PipesParams) – The params provided by the context injector in the orchestration
+ process.Yields: PipesContextData – The context data.
+
+
+
+
+
+
DIRECT_KEY `=` 'data'
+
+
+
+
+
+
+
FILE_PATH_KEY `=` 'path'
+
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesDbfsContextLoader
+
+
+ Context loader that reads context from a JSON file on DBFS.
+
+
+
load_context
+
+
+ A @contextmanager that loads context data injected by the orchestration process.
+
+ This method should read and yield the context data from the location specified by the passed in
+ PipesParams.
+
+ Parameters: params (PipesParams) – The params provided by the context injector in the orchestration
+ process.Yields: PipesContextData – The context data.
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+### Params loaders
+
+Params loaders load the bootstrap payload from some globally accessible key-value store.
+
+
+
class dagster_pipes.PipesParamsLoader
+
+
+ Object that loads params passed from the orchestration process by the context injector and
+ message reader. These params are used to respectively bootstrap the
+ [`PipesContextLoader`](#dagster_pipes.PipesContextLoader) and [`PipesMessageWriter`](#dagster_pipes.PipesMessageWriter).
+
+
+
abstract is_dagster_pipes_process
+
+ Whether or not this process has been provided with provided with information to create
+ a PipesContext or should instead return a mock.
+
+
+
+
+
abstract load_context_params
+
+ PipesParams: Load params passed by the orchestration-side context injector.
+
+
+
+
+
abstract load_messages_params
+
+ PipesParams: Load params passed by the orchestration-side message reader.
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesEnvVarParamsLoader
+
+ Params loader that extracts params from environment variables.
+
+
+
+
+
class dagster_pipes.PipesCliArgsParamsLoader
+
+
+ Params loader that extracts params from known CLI arguments.
+
+
+
is_dagster_pipes_process
+
+ Whether or not this process has been provided with provided with information to create
+ a PipesContext or should instead return a mock.
+
+
+
+
+
load_context_params
+
+ PipesParams: Load params passed by the orchestration-side context injector.
+
+
+
+
+
load_messages_params
+
+ PipesParams: Load params passed by the orchestration-side message reader.
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesMappingParamsLoader
+
+
+ Params loader that extracts params from a Mapping provided at init time.
+
+
+
is_dagster_pipes_process
+
+ Whether or not this process has been provided with provided with information to create
+ a PipesContext or should instead return a mock.
+
+
+
+
+
load_context_params
+
+ PipesParams: Load params passed by the orchestration-side context injector.
+
+
+
+
+
load_messages_params
+
+ PipesParams: Load params passed by the orchestration-side message reader.
+
+
+
+
+
+
+
+
+
+
+
+
+
+### Message writers
+
+Message writers write messages to the location specified in the bootstrap payload.
+
+
+
class dagster_pipes.PipesMessageWriter
+
+
+
+
get_opened_extras
+
+
+ Return arbitary reader-specific information to be passed back to the orchestration
+ process under the extras key of the initialization payload.
+
+ Returns: A dict of arbitrary data to be passed back to the orchestration process.Return type: PipesExtras
+
+
+
+
+
+
final get_opened_payload
+
+
+ Return a payload containing information about the external process to be passed back to
+ the orchestration process. This should contain information that cannot be known before
+ the external process is launched.
+
+ This method should not be overridden by users. Instead, users should
+ override get_opened_extras to inject custom data.
+
+
+
+
+
+
+
abstract open
+
+
+ A @contextmanager that initializes a channel for writing messages back to Dagster.
+
+ This method should takes the params passed by the orchestration-side
+ `PipesMessageReader` and use them to construct and yield a
+ [`PipesMessageWriterChannel`](#dagster_pipes.PipesMessageWriterChannel).
+
+ Parameters: params (PipesParams) – The params provided by the message reader in the orchestration
+ process.Yields: PipesMessageWriterChannel – Channel for writing messagse back to Dagster.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesDefaultMessageWriter
+
+
+ Message writer that writes messages to either a file or the stdout or stderr stream.
+
+ The write location is configured by the params received by the writer. If the params include a
+ key path, then messages will be written to a file at the specified path. If the params instead
+ include a key stdio, then messages then the corresponding value must specify either stderr
+ or stdout, and messages will be written to the selected stream.
+
+
+
open
+
+
+ A @contextmanager that initializes a channel for writing messages back to Dagster.
+
+ This method should takes the params passed by the orchestration-side
+ `PipesMessageReader` and use them to construct and yield a
+ [`PipesMessageWriterChannel`](#dagster_pipes.PipesMessageWriterChannel).
+
+ Parameters: params (PipesParams) – The params provided by the message reader in the orchestration
+ process.Yields: PipesMessageWriterChannel – Channel for writing messagse back to Dagster.
+
+
+
+ Construct and yield a [`PipesBlobStoreMessageWriterChannel`](#dagster_pipes.PipesBlobStoreMessageWriterChannel).
+
+ Parameters: params (PipesParams) – The params provided by the message reader in the orchestration
+ process.Yields: PipesBlobStoreMessageWriterChannel – Channel that periodically uploads message chunks to
+ a blob store.
+
+
+ Message writer channel that periodically uploads message chunks to some blob store endpoint.
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesS3MessageWriter
+
+
+ Message writer that writes messages by periodically writing message chunks to an S3 bucket.
+
+ Parameters:
+ - client (Any) – A boto3.client(“s3”) object.
+ - interval (float) – interval in seconds between upload chunk uploads
+
+
+
+
make_channel
+
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesDbfsMessageWriter
+
+
+ Message writer that writes messages by periodically writing message chunks to a directory on DBFS.
+
+
+
get_opened_extras
+
+
+ Return arbitary reader-specific information to be passed back to the orchestration
+ process under the extras key of the initialization payload.
+
+ Returns: A dict of arbitrary data to be passed back to the orchestration process.Return type: PipesExtras
+
+
+
+
+
+
make_channel
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+### Message writer channels
+
+Message writer channels are objects that write messages back to the Dagster orchestration process.
+
+
+
class dagster_pipes.PipesMessageWriterChannel
+
+
+ Object that writes messages back to the Dagster orchestration process.
+
+
+
abstract write_message
+
+
+ Write a message to the orchestration process.
+
+ Parameters: message (PipesMessage) – The message to write.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesBlobStoreMessageWriterChannel
+
+
+ Message writer channel that periodically uploads message chunks to some blob store endpoint.
+
+
+
buffered_upload_loop
+
+
+
+
+
+
+
flush_messages
+
+
+
+
+
+
+
abstract upload_messages_chunk
+
+
+
+
+
+
+
write_message
+
+
+ Write a message to the orchestration process.
+
+ Parameters: message (PipesMessage) – The message to write.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesBufferedFilesystemMessageWriterChannel
+
+
+ Message writer channel that periodically writes message chunks to an endpoint mounted on the filesystem.
+
+ Parameters: interval (float) – interval in seconds between chunk uploads
+
+
upload_messages_chunk
+
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesFileMessageWriterChannel
+
+
+ Message writer channel that writes one message per line to a file.
+
+
+
write_message
+
+
+ Write a message to the orchestration process.
+
+ Parameters: message (PipesMessage) – The message to write.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesStreamMessageWriterChannel
+
+
+ Message writer channel that writes one message per line to a TextIO stream.
+
+
+
write_message
+
+
+ Write a message to the orchestration process.
+
+ Parameters: message (PipesMessage) – The message to write.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesS3MessageWriterChannel
+
+
+ Message writer channel for writing messages by periodically writing message chunks to an S3 bucket.
+
+ Parameters:
+ - client (Any) – A boto3.client(“s3”) object.
+ - bucket (str) – The name of the S3 bucket to write to.
+ - key_prefix (Optional[str]) – An optional prefix to use for the keys of written blobs.
+ - interval (float) – interval in seconds between upload chunk uploads
+
+
+
+
upload_messages_chunk
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+### Utilities
+
+
+
dagster_pipes.encode_env_var
+
+
+ Encode value by serializing to JSON, compressing with zlib, and finally encoding with base64.
+ base64_encode(compress(to_json(value))) in function notation.
+
+ Parameters: value (Any) – The value to encode. Must be JSON-serializable.Returns: The encoded value.Return type: str
+
+
+
+
+
+
dagster_pipes.decode_env_var
+
+
+ Decode a value by decoding from base64, decompressing with zlib, and finally deserializing from
+ JSON. from_json(decompress(base64_decode(value))) in function notation.
+
+ Parameters: value (Any) – The value to decode.Returns: The decoded value.Return type: Any
+
+
+
+
+
+
class dagster_pipes.DagsterPipesError
+
+
+
+
+
+
+
class dagster_pipes.DagsterPipesWarning
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-polars.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-polars.mdx
new file mode 100644
index 0000000000000..8fb15b4d00b05
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-polars.mdx
@@ -0,0 +1,411 @@
+---
+title: 'polars (dagster-polars)'
+title_meta: 'polars (dagster-polars) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'polars (dagster-polars) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Polars (dagster-polars)
+
+This library provides Dagster integration with [Polars](https://pola.rs).
+It allows using Polars eager or lazy DataFrames as inputs and outputs with Dagster’s @asset and @op.
+Type annotations are used to control whether to load an eager or lazy DataFrame. Lazy DataFrames can be sinked as output.
+Multiple serialization formats (Parquet, Delta Lake, BigQuery) and filesystems (local, S3, GCS, …) are supported.
+
+Comprehensive list of dagster-polars behavior for supported type annotations can be found in [Type Annotations](#types)`Type Annotations` section.
+
+
+
+
+
+
+
+# Installation
+
+ ```default
+ pip install dagster-polars
+ ```
+Some IOManagers (like [`PolarsDeltaIOManager`](#dagster_polars.PolarsDeltaIOManager)) may require additional dependencies, which are provided with extras like dagster-polars[delta].
+Please check the documentation for each IOManager for more details.
+
+
+
+
+
+
+
+# Quickstart
+
+Common filesystem-based IOManagers features highlights, using [`PolarsParquetIOManager`](#dagster_polars.PolarsParquetIOManager) as an example (see [`BasePolarsUPathIOManager`](#dagster_polars.BasePolarsUPathIOManager) for the full list of features provided by dagster-polars):
+
+Type annotations are not required. By default an eager pl.DataFrame will be loaded.
+
+ ```python
+ from dagster import asset
+ import polars as pl
+
+ @asset(io_manager_key="polars_parquet_io_manager")
+ def upstream():
+ return DataFrame({"foo": [1, 2, 3]})
+
+ @asset(io_manager_key="polars_parquet_io_manager")
+ def downstream(upstream) -> pl.LazyFrame:
+ assert isinstance(upstream, pl.DataFrame)
+ return upstream.lazy() # LazyFrame will be sinked
+ ```
+Lazy pl.LazyFrame can be scanned by annotating the input with pl.LazyFrame, and returning a pl.LazyFrame will sink it:
+
+ ```python
+ @asset(io_manager_key="polars_parquet_io_manager")
+ def downstream(upstream: pl.LazyFrame) -> pl.LazyFrame:
+ assert isinstance(upstream, pl.LazyFrame)
+ return upstream
+ ```
+The same logic applies to partitioned assets:
+
+ ```python
+ @asset
+ def downstream(partitioned_upstream: Dict[str, pl.LazyFrame]):
+ assert isinstance(partitioned_upstream, dict)
+ assert isinstance(partitioned_upstream["my_partition"], pl.LazyFrame)
+ ```
+Optional inputs and outputs are supported:
+
+ ```python
+ @asset
+ def upstream() -> Optional[pl.DataFrame]:
+ if has_data:
+ return DataFrame({"foo": [1, 2, 3]}) # type check will pass
+ else:
+ return None # type check will pass and `dagster_polars` will skip writing the output completely
+
+ @asset
+ def downstream(upstream: Optional[pl.LazyFrame]): # upstream will be None if it doesn't exist in storage
+ ...
+ ```
+By default all the IOManagers store separate partitions as physically separated locations, such as:
+
+ - /my/asset/key/partition_0.extension
+ - /my/asset/key/partition_1.extension
+
+
+This mode is useful for e.g. snapshotting.
+
+Some IOManagers (like [`PolarsDeltaIOManager`](#dagster_polars.PolarsDeltaIOManager)) support reading and writing partitions in storage-native format in the same location.
+This mode can be typically enabled by setting “partition_by” metadata value. For example, [`PolarsDeltaIOManager`](#dagster_polars.PolarsDeltaIOManager) would store different partitions in the same /my/asset/key.delta directory, which will be properly partitioned.
+
+This mode should be preferred for true partitioning.
+
+
+
+
+
+
+
+
+# Type Annotations
+
+Type aliases like DataFrameWithPartitions are provided by `dagster_polars.types` for convenience.
+
+# Supported type annotations and dagster-polars behavior
+
+| Type annotation | Type Alias | Behavior |
+| :------------------------ | :--------- | :-------------------------------------------------------------------------- |
+| DataFrame | | read/write aDataFrame |
+| LazyFrame | | read/sink aLazyFrame |
+| Optional[DataFrame] | | read/write aDataFrame. Do nothing if no data is found in storage or the output isNone |
+| Optional[LazyFrame] | | read aLazyFrame. Do nothing if no data is found in storage |
+| Dict[str, DataFrame] | DataFrameWithPartitions | read multipleDataFrame`s as `Dict[str, DataFrame]. Raises an error for missing partitions, unless“allow_missing_partitions”input metadata is set toTrue |
+| Dict[str, LazyFrame] | LazyFramePartitions | read multipleLazyFrame`s as `Dict[str, LazyFrame]. Raises an error for missing partitions, unless“allow_missing_partitions”input metadata is set toTrue |
+
+
+Generic builtins (like tuple[…] instead of Tuple[…]) are supported for Python >= 3.9.
+
+
+
+ Base class for dagster-polars IOManagers.
+
+ Doesn’t define a specific storage format.
+
+ To implement a specific storage format (parquet, csv, etc), inherit from this class and implement the write_df_to_path, sink_df_to_path and scan_df_from_path methods.
+
+ Features:
+ - All the features of [`UPathIOManager`](../io-managers.mdx#dagster.UPathIOManager) - works with local and remote filesystems (like S3), supports loading multiple partitions with respect to [`PartitionMapping`](../partitions.mdx#dagster.PartitionMapping), and more
+ - loads the correct type - polars.DataFrame, polars.LazyFrame, or other types defined in `dagster_polars.types` - based on the input type annotation (or dagster.DagsterType’s typing_type)
+ - can sink lazy pl.LazyFrame DataFrames
+ - handles Nones with Optional types by skipping loading missing inputs or saving None outputs
+ - logs various metadata about the DataFrame - size, schema, sample, stats, …
+ - the “columns” input metadata value can be used to select a subset of columns to load
+
+
+
+
+
+ Implements reading and writing Polars DataFrames from/to [BigQuery](https://cloud.google.com/bigquery)).
+
+ Features:
+ - All `DBIOManager` features
+ - Supports writing partitioned tables (“partition_expr” input metadata key must be specified).
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster import Definitions, EnvVar
+ from dagster_polars import PolarsBigQueryIOManager
+
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pl.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": PolarsBigQueryIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+ You can tell Dagster in which dataset to create tables by setting the “dataset” configuration value.
+ If you do not provide a dataset as configuration to the I/O manager, Dagster will determine a dataset based
+ on the assets and ops using the I/O Manager. For assets, the dataset will be determined from the asset key,
+ as shown in the above example. The final prefix before the asset name will be used as the dataset. For example,
+ if the asset “my_table” had the key prefix [“gcp”, “bigquery”, “my_dataset”], the dataset “my_dataset” will be
+ used. For ops, the dataset can be specified by including a “schema” entry in output metadata. If “schema” is
+ not provided via config or on the asset/op, “public” will be used for the dataset.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_dataset"})}
+ )
+ def make_my_table() -> pl.DataFrame:
+ # the returned value will be stored at my_dataset.my_table
+ ...
+ ```
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pl.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the “gcp_credentials” configuration.
+ Dagster will store this key in a temporary file and set GOOGLE_APPLICATION_CREDENTIALS to point to the file.
+ After the run completes, the file will be deleted, and GOOGLE_APPLICATION_CREDENTIALS will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded key with this shell command: cat $GOOGLE_APPLICATION_CREDENTIALS | base64
+
+ The “write_disposition” metadata key can be used to set the write_disposition parameter
+ of bigquery.JobConfig. For example, set it to “WRITE_APPEND” to append to an existing table intead of
+ overwriting it.
+
+ Install dagster-polars[gcp] to use this IOManager.
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-postgres.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-postgres.mdx
new file mode 100644
index 0000000000000..70782ce214e11
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-postgres.mdx
@@ -0,0 +1,169 @@
+---
+title: 'postgresql (dagster-postgres)'
+title_meta: 'postgresql (dagster-postgres) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'postgresql (dagster-postgres) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# PostgreSQL (dagster-postgres)
+
+
+
dagster_postgres.PostgresEventLogStorage `=` \
+
+
+
+
+ Postgres-backed event log storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ To use Postgres for all of the components of your instance storage, you can add the following
+ block to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ storage:
+ postgres:
+ postgres_db:
+ username: my_username
+ password: my_password
+ hostname: my_hostname
+ db_name: my_database
+ port: 5432
+ ```
+ If you are configuring the different storage components separately and are specifically
+ configuring your event log storage to use Postgres, you can add a block such as the following
+ to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ event_log_storage:
+ module: dagster_postgres.event_log
+ class: PostgresEventLogStorage
+ config:
+ postgres_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { db_name }
+ port: { port }
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
+
dagster_postgres.PostgresRunStorage `=` \
+
+
+
+
+ Postgres-backed run storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ To use Postgres for all of the components of your instance storage, you can add the following
+ block to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ storage:
+ postgres:
+ postgres_db:
+ username: my_username
+ password: my_password
+ hostname: my_hostname
+ db_name: my_database
+ port: 5432
+ ```
+ If you are configuring the different storage components separately and are specifically
+ configuring your run storage to use Postgres, you can add a block such as the following
+ to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ run_storage:
+ module: dagster_postgres.run_storage
+ class: PostgresRunStorage
+ config:
+ postgres_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { db_name }
+ port: { port }
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
+
dagster_postgres.PostgresScheduleStorage `=` \
+
+
+
+
+ Postgres-backed run storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ To use Postgres for all of the components of your instance storage, you can add the following
+ block to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ storage:
+ postgres:
+ postgres_db:
+ username: my_username
+ password: my_password
+ hostname: my_hostname
+ db_name: my_database
+ port: 5432
+ ```
+ If you are configuring the different storage components separately and are specifically
+ configuring your schedule storage to use Postgres, you can add a block such as the following
+ to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ schedule_storage:
+ module: dagster_postgres.schedule_storage
+ class: PostgresScheduleStorage
+ config:
+ postgres_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { db_name }
+ port: { port }
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-powerbi.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-powerbi.mdx
new file mode 100644
index 0000000000000..5f96cda0423d2
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-powerbi.mdx
@@ -0,0 +1,146 @@
+---
+title: 'powerbi (dagster-powerbi)'
+title_meta: 'powerbi (dagster-powerbi) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'powerbi (dagster-powerbi) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# PowerBI (dagster-powerbi)
+
+Dagster allows you to represent your PowerBI Workspaces as assets, alongside other your other
+technologies like dbt and Sling. This allows you to see how your PowerBI assets are connected to
+your other data assets, and how changes to other data assets might impact your PowerBI Workspaces.
+
+
+
+
+## PowerBI API
+
+Here, we provide interfaces to manage PowerBI Workspaces using the PowerBI API.
+
+
+
+### Assets (PowerBI)
+
+
+
class dagster_powerbi.PowerBIServicePrincipal
+
+ Authenticates with PowerBI using a service principal.
+
+
+
+
+
class dagster_powerbi.PowerBIToken
+
+ Authenticates with PowerBI directly using an API access token.
+
+
+
+
+
class dagster_powerbi.PowerBIWorkspace
+
+
+ Represents a workspace in PowerBI and provides utilities
+ to interact with the PowerBI API.
+
+
+
build_defs
+
+
+ :::danger[deprecated]
+ This API will be removed in version 1.9.0.
+ Use dagster_powerbi.load_powerbi_asset_specs instead.
+
+ :::
+
+ Returns a Definitions object which will load Power BI content from
+ the workspace and translate it into assets, using the provided translator.
+
+ Parameters:
+ - context (Optional[DefinitionsLoadContext]) – The context to use when loading the definitions.
+ - dagster_powerbi_translator (Type[[*DagsterPowerBITranslator*](#dagster_powerbi.DagsterPowerBITranslator)]) – The translator to use
+ - enable_refresh_semantic_models (bool) – Whether to enable refreshing semantic models
+
+
+ Returns: A Definitions object which will build and return the Power BI content.Return type: [Definitions](../definitions.mdx#dagster.Definitions)
+
+
+
+
+
+
poll_refresh
+
+ Polls the refresh status of a PowerBI dataset until it completes or fails.
+
+
+
+
+
trigger_and_poll_refresh
+
+ Triggers a refresh of a PowerBI dataset and polls until it completes or fails.
+
+
+
+
+
trigger_refresh
+
+ Triggers a refresh of a PowerBI dataset.
+
+
+
+
+
+
+
+
+
class dagster_powerbi.DagsterPowerBITranslator
+
+ Translator class which converts raw response data from the PowerBI API into AssetSpecs.
+ Subclass this class to implement custom logic for each type of PowerBI content.
+
+
+
+
+
dagster_powerbi.load_powerbi_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Power BI content in the workspace.
+
+ Parameters:
+ - workspace ([*PowerBIWorkspace*](#dagster_powerbi.PowerBIWorkspace)) – The Power BI workspace to load assets from.
+ - dagster_powerbi_translator (Optional[Union[[*DagsterPowerBITranslator*](#dagster_powerbi.DagsterPowerBITranslator), Type[[*DagsterPowerBITranslator*](#dagster_powerbi.DagsterPowerBITranslator)]]]) – The translator to use to convert Power BI content into [`dagster.AssetSpec`](../assets.mdx#dagster.AssetSpec).
+ - use_workspace_scan (bool) – Whether to scan the entire workspace using admin APIs
+
+
+ Returns: The set of assets representing the Power BI content in the workspace.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Builds an asset definition for refreshing a PowerBI semantic model.
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-prometheus.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-prometheus.mdx
new file mode 100644
index 0000000000000..9e6acf11b3522
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-prometheus.mdx
@@ -0,0 +1,66 @@
+---
+title: 'prometheus (dagster-prometheus)'
+title_meta: 'prometheus (dagster-prometheus) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'prometheus (dagster-prometheus) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-pyspark.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-pyspark.mdx
new file mode 100644
index 0000000000000..3a8f991c012aa
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-pyspark.mdx
@@ -0,0 +1,79 @@
+---
+title: 'pyspark (dagster-pyspark)'
+title_meta: 'pyspark (dagster-pyspark) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'pyspark (dagster-pyspark) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-shell.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-shell.mdx
new file mode 100644
index 0000000000000..8cc6ed4722244
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-shell.mdx
@@ -0,0 +1,250 @@
+---
+title: 'shell (dagster-shell)'
+title_meta: 'shell (dagster-shell) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'shell (dagster-shell) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Shell (dagster-shell)
+
+The Dagster shell library provides utilities and op factories for executing inline shell scripts or script files.
+
+
+
+
+
+
+# APIs
+
+
+
dagster_shell.create_shell_command_op
+
+
+ :::danger[deprecated]
+ This API will be removed in version 0.25.
+ Use PipesSubprocessClient instead..
+
+ :::
+
+ DEPRECATED: Use PipesSubprocessClient instead.
+
+ This function is a factory that constructs ops to execute a shell command.
+
+ Note that you can only use `shell_command_op` if you know the command you’d like to execute
+ at job construction time. If you’d like to construct shell commands dynamically during
+ job execution and pass them between ops, you should use `shell_op` instead.
+
+ The resulting op can take a single `start` argument that is a
+ [Nothing dependency](https://legacy-docs.dagster.io/concepts/ops-jobs-graphs/graphs#defining-nothing-dependencies)
+ to allow you to run ops before the shell op.
+
+ Examples:
+
+ ```python
+ from dagster import graph
+ from dagster_shell import create_shell_command_op
+
+
+ @graph
+ def my_graph():
+ a = create_shell_command_op('echo "hello, world!"', name="a")
+ a()
+ ```
+ ```python
+ @op
+ def run_before_shell_op():
+ do_some_work()
+
+ @graph
+ def my_graph():
+ my_echo_op = create_shell_command_op("echo hello world!", name="echo_op")
+ my_echo_op(start=run_before_shell_op())
+ ```
+ Parameters:
+ - shell_command (str) – The shell command that the constructed op will execute.
+ - name (str) – The name of the constructed op.
+ - description (Optional[str]) – Human-readable description of this op.
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by this op.
+ - tags (Optional[Dict[str, Any]]) – Arbitrary metadata for the op. Frameworks may
+
+
+ Raises: [Failure](../ops.mdx#dagster.Failure)Failure – Raised when the shell command returns a non-zero exit code.Returns: Returns the constructed op definition.Return type: [OpDefinition](../ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
dagster_shell.create_shell_script_op
+
+
+ :::danger[deprecated]
+ This API will be removed in version 0.25.
+ Use PipesSubprocessClient instead..
+
+ :::
+
+ DEPRECATED: Use PipesSubprocessClient instead.
+
+ This function is a factory which constructs an op that will execute a shell command read
+ from a script file.
+
+ Any kwargs passed to this function will be passed along to the underlying [`@op`](../ops.mdx#dagster.op) decorator. However, note that overriding `config` or `output_defs` is not
+ supported.
+
+ You might consider using [`@graph`](../graphs.mdx#dagster.graph) to wrap this op
+ in the cases where you’d like to configure the shell op with different config fields.
+
+ If no `ins` are passed then the resulting op can take a single `start` argument that is a
+ [Nothing dependency](https://legacy-docs.dagster.io/concepts/ops-jobs-graphs/graphs#defining-nothing-dependencies)
+ to allow you to run ops before the shell op.
+
+ Examples:
+
+ ```python
+ from dagster import file_relative_path, graph
+ from dagster_shell import create_shell_script_op
+
+
+ @graph
+ def my_graph():
+ a = create_shell_script_op(file_relative_path(__file__, "hello_world.sh"), name="a")
+ a()
+ ```
+ ```python
+ @op
+ def run_before_shell_op():
+ do_some_work()
+
+ @graph
+ def my_graph():
+ my_echo_op = create_shell_script_op(file_relative_path(__file__, "hello_world.sh"), name="echo_op")
+ my_echo_op(start=run_before_shell_op())
+ ```
+ Parameters:
+ - shell_script_path (str) – The script file to execute.
+ - name (Optional[str]) – The name of this op. Defaults to “create_shell_script_op”.
+ - ins (Optional[Mapping[str, [*In*](../ops.mdx#dagster.In)]]) – Ins for the op. Defaults to
+
+
+ Raises: [Failure](../ops.mdx#dagster.Failure)Failure – Raised when the shell command returns a non-zero exit code.Returns: Returns the constructed op definition.Return type: [OpDefinition](../ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
dagster_shell.shell_op
+
+
+ This op executes a shell command it receives as input.
+ This op is suitable for uses where the command to execute is generated dynamically by
+ upstream ops. If you know the command to execute at job construction time,
+ consider `shell_command_op` instead.
+
+ Parameters:
+ - shell_command – The shell command to be executed
+ - config (ShellOpConfig) – A ShellOpConfig object specifying configuration options
+
+
+ Examples:
+
+ ```python
+ @op
+ def create_shell_command():
+ return "echo hello world!"
+
+ @graph
+ def echo_graph():
+ shell_op(create_shell_command())
+ ```
+
+
+
+
+
+
dagster_shell.execute_shell_command
+
+
+ This function is a utility for executing shell commands from within a Dagster op (or from Python in general).
+ It can be used to execute shell commands on either op input data, or any data generated within a generic python op.
+
+ Internally, it executes a shell script specified by the argument `shell_command`. The script will be written
+ to a temporary file first and invoked via `subprocess.Popen(['bash', shell_script_path], ...)`.
+
+ In the Popen invocation, `stdout=PIPE, stderr=STDOUT` is used, and the combined stdout/stderr
+ output is retrieved.
+
+ Examples:
+
+ ```python
+ from dagster import OpExecutionContext, op
+ from dagster_shell import execute_shell_command
+
+
+ @op
+ def my_shell_op(context: OpExecutionContext, data: str):
+ temp_file = "/tmp/data.txt"
+ with open(temp_file, "w", encoding="utf-8") as temp_file_writer:
+ temp_file_writer.write(data)
+ execute_shell_command(f"cat {temp_file}", output_logging="STREAM", log=context.log)
+ ```
+ Parameters:
+ - shell_command (str) – The shell command to execute
+ - output_logging (str) – The logging mode to use. Supports STREAM, BUFFER, and NONE.
+ - log (Union[logging.Logger, [*DagsterLogManager*](../loggers.mdx#dagster.DagsterLogManager)]) – Any logger which responds to .info()
+ - cwd (str, optional) – Working directory for the shell command to use. Defaults to the
+ - env (Dict[str, str], optional) – Environment dictionary to pass to `subprocess.Popen`.
+ - log_shell_command (bool, optional) – Whether to log the shell command before executing it.
+
+
+ Returns: A tuple where the first element is the combined stdout/stderr output of running the shell
+ command and the second element is the return code.Return type: Tuple[str, int]
+
+
+
+
+
+
dagster_shell.execute_shell_script
+
+
+ Execute a shell script file specified by the argument `shell_script_path`. The script will be
+ invoked via `subprocess.Popen(['bash', shell_script_path], ...)`.
+
+ In the Popen invocation, `stdout=PIPE, stderr=STDOUT` is used, and the combined stdout/stderr
+ output is retrieved.
+
+ Examples:
+
+ ```python
+ from dagster import OpExecutionContext, op
+ from dagster_shell import execute_shell_script
+
+
+ @op
+ def my_shell_op(context: OpExecutionContext, data: str):
+ temp_file = "/tmp/echo_data.sh"
+ with open(temp_file, "w", encoding="utf-8") as temp_file_writer:
+ temp_file_writer.write(f"echo {data}")
+ execute_shell_script(temp_file, output_logging="STREAM", log=context.log)
+ ```
+ Parameters:
+ - shell_script_path (str) – The shell script to execute.
+ - output_logging (str) – The logging mode to use. Supports STREAM, BUFFER, and NONE.
+ - log (Union[logging.Logger, [*DagsterLogManager*](../loggers.mdx#dagster.DagsterLogManager)]) – Any logger which responds to .info()
+ - cwd (str, optional) – Working directory for the shell command to use. Defaults to the
+ - env (Dict[str, str], optional) – Environment dictionary to pass to `subprocess.Popen`.
+ - log_shell_command (bool, optional) – Whether to log the shell command before executing it.
+
+
+ Raises: Exception – When an invalid output_logging is selected. Unreachable from op-based
+ invocation since the config system will check output_logging against the config
+ enum.Returns: A tuple where the first element is the combined stdout/stderr output of running the shell
+ command and the second element is the return code.Return type: Tuple[str, int]
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-sigma.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-sigma.mdx
new file mode 100644
index 0000000000000..463c6e413ea44
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-sigma.mdx
@@ -0,0 +1,148 @@
+---
+title: 'sigma (dagster-sigma)'
+title_meta: 'sigma (dagster-sigma) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'sigma (dagster-sigma) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Sigma (dagster-sigma)
+
+Dagster allows you to represent the workbooks and datasets in your Sigma project as assets alongside other
+technologies including dbt and Sling. This allows you to visualize relationships between your Sigma assets
+and their dependencies.
+
+Related documentation pages: [Using Dagster with Sigma](https://docs.dagster.io/integrations/libraries/sigma/).
+
+
+
+
+## Sigma API
+
+Here, we provide interfaces to manage Sigma projects using the Sigma API.
+
+
+
+### Assets (Sigma API)
+
+
+
class dagster_sigma.SigmaOrganization
+
+
+ Represents a workspace in Sigma and provides utilities
+ to interact with the Sigma API.
+
+
+
build_defs
+
+
+ :::danger[deprecated]
+ This API will be removed in version 1.9.0.
+ Use dagster_sigma.load_sigma_asset_specs instead.
+
+ :::
+
+ Returns a Definitions object representing the Sigma content in the organization.
+
+ Parameters: dagster_sigma_translator (Type[[*DagsterSigmaTranslator*](#dagster_sigma.DagsterSigmaTranslator)]) – The translator to use
+ to convert Sigma content into AssetSpecs. Defaults to DagsterSigmaTranslator.Returns: The set of assets representing the Sigma content in the organization.Return type: [Definitions](../definitions.mdx#dagster.Definitions)
+
+
+
+
+
+
+
+
+
+
class dagster_sigma.SigmaBaseUrl
+
+
+ Enumeration of Sigma API base URLs for different cloud providers.
+
+ [https://help.sigmacomputing.com/reference/get-started-sigma-api#identify-your-api-request-url](https://help.sigmacomputing.com/reference/get-started-sigma-api#identify-your-api-request-url)
+
+
+
+
+
+
+
class dagster_sigma.DagsterSigmaTranslator
+
+ Translator class which converts raw response data from the Sigma API into AssetSpecs.
+ Subclass this class to provide custom translation logic.
+
+
+
+
+
class dagster_sigma.SigmaDataset
+
+
+ Represents a Sigma dataset, a centralized data definition which can
+ contain aggregations or other manipulations.
+
+ [https://help.sigmacomputing.com/docs/datasets](https://help.sigmacomputing.com/docs/datasets)
+
+
+
+
+
+
+
class dagster_sigma.SigmaWorkbook
+
+
+ Represents a Sigma workbook, a collection of visualizations and queries
+ for data exploration and analysis.
+
+ [https://help.sigmacomputing.com/docs/workbooks](https://help.sigmacomputing.com/docs/workbooks)
+
+
+
+
+
+
+
class dagster_sigma.SigmaFilter
+
+
+ Filters the set of Sigma objects to fetch.
+
+ Parameters:
+ - workbook_folders (Optional[Sequence[Sequence[str]]]) – A list of folder paths to fetch workbooks from.
+ - workbooks (Optional[Sequence[Sequence[str]]]) – A list of fully qualified workbook paths to fetch.
+ - include_unused_datasets (bool) – Whether to include datasets that are not used in any workbooks.
+
+
+
+
+
+
+
+
dagster_sigma.load_sigma_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Sigma content in the organization.
+
+ Parameters:
+ - organization ([*SigmaOrganization*](#dagster_sigma.SigmaOrganization)) – The Sigma organization to fetch assets from.
+ - dagster_sigma_translator (Optional[Union[[*DagsterSigmaTranslator*](#dagster_sigma.DagsterSigmaTranslator), Type[DagsterSigmaTranslatorr]]]) – The translator to use to convert Sigma content into [`dagster.AssetSpec`](../assets.mdx#dagster.AssetSpec).
+ - sigma_filter (Optional[[*SigmaFilter*](#dagster_sigma.SigmaFilter)]) – Filters the set of Sigma objects to fetch.
+ - fetch_column_data (bool) – Whether to fetch column data for datasets, which can be slow.
+ - fetch_lineage_data (bool) – Whether to fetch any lineage data for workbooks and datasets.
+ - snapshot_path (Optional[Union[str, Path]]) – Path to a snapshot file to load Sigma data from,
+
+
+ Returns: The set of assets representing the Sigma content in the organization.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-slack.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-slack.mdx
new file mode 100644
index 0000000000000..92204e44fa2c6
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-slack.mdx
@@ -0,0 +1,234 @@
+---
+title: 'slack (dagster-slack)'
+title_meta: 'slack (dagster-slack) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'slack (dagster-slack) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Slack (dagster-slack)
+
+
+
+This library provides an integration with Slack, to support posting messages in your company’s Slack workspace.
+
+
+
+Presently, it provides a thin wrapper on the Slack client API [chat.postMessage](https://api.slack.com/methods/chat.postMessage).
+
+
+
+To use this integration, you’ll first need to create a Slack App for it.
+
+1. Create App: Go to [https://api.slack.com/apps](https://api.slack.com/apps) and click “Create New App”:
+
+2. Install App: After creating an app, on the left-hand side of the app configuration, click “Bot Users”, and then create a bot user. Then, click “Install App” on the left hand side, and finally “Install App to Workspace”.
+3. Bot Token: Once finished, this will create a new bot token for your bot/workspace:
+
+Copy this bot token and put it somewhere safe; see [Safely Storing Credentials](https://api.slack.com/docs/oauth-safety) for more on this topic.
+
+
+
dagster_slack.SlackResource ResourceDefinition
+
+
+ This resource is for connecting to Slack.
+
+ By configuring this Slack resource, you can post messages to Slack from any Dagster op, asset, schedule or sensor.
+
+ Examples:
+
+ ```python
+ import os
+
+ from dagster import EnvVar, job, op
+ from dagster_slack import SlackResource
+
+
+ @op
+ def slack_op(slack: SlackResource):
+ slack.get_client().chat_postMessage(channel='#noise', text=':wave: hey there!')
+
+ @job
+ def slack_job():
+ slack_op()
+
+ defs = Definitions(
+ jobs=[slack_job],
+ resources={
+ "slack": SlackResource(token=EnvVar("MY_SLACK_TOKEN")),
+ },
+ )
+ ```
+
+
+
+
+
+
dagster_slack.make_slack_on_run_failure_sensor
+
+
+ Create a sensor on job failures that will message the given Slack channel.
+
+ Parameters:
+ - channel (str) – The channel to send the message to (e.g. “#my_channel”)
+ - slack_token (str) – The slack token.
+ - text_fn (Optional(Callable[[[*RunFailureSensorContext*](../schedules-sensors.mdx#dagster.RunFailureSensorContext)], str])) – Function which
+ - blocks_fn (Callable[[[*RunFailureSensorContext*](../schedules-sensors.mdx#dagster.RunFailureSensorContext)], List[Dict]]) – Function which takes in
+ - name – (Optional[str]): The name of the sensor. Defaults to “slack_on_run_failure”.
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your Dagit instance. Specify this to allow
+ - minimum_interval_seconds – (Optional[int]): The minimum number of seconds that will elapse
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](../jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](../graphs.mdx#dagster.GraphDefinition), [*RepositorySelector*](../schedules-sensors.mdx#dagster.RepositorySelector), [*JobSelector*](../schedules-sensors.mdx#dagster.JobSelector), CodeLocationSensor]]]) – The jobs in the
+ - job_selection (Optional[List[Union[[*JobDefinition*](../jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](../graphs.mdx#dagster.GraphDefinition), [*RepositorySelector*](../schedules-sensors.mdx#dagster.RepositorySelector), [*JobSelector*](../schedules-sensors.mdx#dagster.JobSelector), CodeLocationSensor]]]) – deprecatedmonitored_jobs instead.) (deprecated in favor of monitored_jobs)
+ - monitor_all_code_locations (bool) – If set to True, the sensor will monitor all runs in the
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - webserver_base_url – (Optional[str]): The base url of your webserver instance. Specify this to allow
+ - monitor_all_repositories (bool) – deprecatedmonitor_all_code_locations instead.) If set to True, the sensor will monitor all runs in the
+
+
+ Examples:
+
+ ```python
+ slack_on_run_failure = make_slack_on_run_failure_sensor(
+ "#my_channel",
+ os.getenv("MY_SLACK_TOKEN")
+ )
+
+ @repository
+ def my_repo():
+ return [my_job + slack_on_run_failure]
+ ```
+ ```python
+ def my_message_fn(context: RunFailureSensorContext) -> str:
+ return (
+ f"Job {context.dagster_run.job_name} failed!"
+ f"Error: {context.failure_event.message}"
+ )
+
+ slack_on_run_failure = make_slack_on_run_failure_sensor(
+ channel="#my_channel",
+ slack_token=os.getenv("MY_SLACK_TOKEN"),
+ text_fn=my_message_fn,
+ webserver_base_url="http://mycoolsite.com",
+ )
+ ```
+
+
+
+
+
+
dagster_slack.slack_on_failure HookDefinition
+
+
+ Create a hook on step failure events that will message the given Slack channel.
+
+ Parameters:
+ - channel (str) – The channel to send the message to (e.g. “#my_channel”)
+ - message_fn (Optional(Callable[[[*HookContext*](../hooks.mdx#dagster.HookContext)], str])) – Function which takes in the HookContext
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your webserver instance. Specify this to allow
+ - webserver_base_url – (Optional[str]): The base url of your webserver instance. Specify this to allow
+
+
+ Examples:
+
+ ```python
+ @slack_on_failure("#foo", webserver_base_url="http://localhost:3000")
+ @job(...)
+ def my_job():
+ pass
+ ```
+ ```python
+ def my_message_fn(context: HookContext) -> str:
+ return f"Op {context.op} failed!"
+
+ @op
+ def an_op(context):
+ pass
+
+ @job(...)
+ def my_job():
+ an_op.with_hooks(hook_defs={slack_on_failure("#foo", my_message_fn)})
+ ```
+
+
+
+
+
+
dagster_slack.slack_on_success HookDefinition
+
+
+ Create a hook on step success events that will message the given Slack channel.
+
+ Parameters:
+ - channel (str) – The channel to send the message to (e.g. “#my_channel”)
+ - message_fn (Optional(Callable[[[*HookContext*](../hooks.mdx#dagster.HookContext)], str])) – Function which takes in the HookContext
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your webserver instance. Specify this to allow
+ - webserver_base_url – (Optional[str]): The base url of your webserver instance. Specify this to allow
+
+
+ Examples:
+
+ ```python
+ @slack_on_success("#foo", webserver_base_url="http://localhost:3000")
+ @job(...)
+ def my_job():
+ pass
+ ```
+ ```python
+ def my_message_fn(context: HookContext) -> str:
+ return f"Op {context.op} worked!"
+
+ @op
+ def an_op(context):
+ pass
+
+ @job(...)
+ def my_job():
+ an_op.with_hooks(hook_defs={slack_on_success("#foo", my_message_fn)})
+ ```
+
+
+
+
+
+
+## Legacy
+
+
+
dagster_slack.slack_resource ResourceDefinition
+
+
+ This resource is for connecting to Slack.
+
+ The resource object is a slack_sdk.WebClient.
+
+ By configuring this Slack resource, you can post messages to Slack from any Dagster op, asset, schedule or sensor.
+
+ Examples:
+
+ ```python
+ import os
+
+ from dagster import job, op
+ from dagster_slack import slack_resource
+
+
+ @op(required_resource_keys={'slack'})
+ def slack_op(context):
+ context.resources.slack.chat_postMessage(channel='#noise', text=':wave: hey there!')
+
+ @job(resource_defs={'slack': slack_resource})
+ def slack_job():
+ slack_op()
+
+ slack_job.execute_in_process(
+ run_config={'resources': {'slack': {'config': {'token': os.getenv('SLACK_TOKEN')}}}}
+ )
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-sling.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-sling.mdx
new file mode 100644
index 0000000000000..0b3043d75d243
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-sling.mdx
@@ -0,0 +1,454 @@
+---
+title: 'dlt (dagster-dlt)'
+title_meta: 'dlt (dagster-dlt) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dlt (dagster-dlt) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# dlt (dagster-dlt)
+
+This library provides a Dagster integration with [Sling](https://slingdata.io).
+
+For more information on getting started, see the [Sling & Dagster](https://docs.dagster.io/integrations/libraries/sling) documentation.
+
+
+
+
+## Assets
+
+
+
@dagster_sling.sling_assets
+
+
+ Create a definition for how to materialize a set of Sling replication streams as Dagster assets, as
+ described by a Sling replication config. This will create on Asset for every Sling target stream.
+
+ A Sling Replication config is a configuration that maps sources to destinations. For the full
+ spec and descriptions, see [Sling’s Documentation](https://docs.slingdata.io/sling-cli/run/configuration).
+
+ Parameters:
+ - replication_config (Union[Mapping[str, Any], str, Path]) – A path to a Sling replication config, or a dictionary
+ - dagster_sling_translator – (DagsterSlingTranslator): Allows customization of how to map a Sling stream to a Dagster
+ - (Optional[str] (name) – The name of the op.
+ - partitions_def (Optional[[*PartitionsDefinition*](../partitions.mdx#dagster.PartitionsDefinition)]) – The partitions definition for this asset.
+ - backfill_policy (Optional[[*BackfillPolicy*](../partitions.mdx#dagster.BackfillPolicy)]) – The backfill policy for this asset.
+ - op_tags (Optional[Mapping[str, Any]]) – The tags for the underlying op.
+
+
+ Examples:
+
+ Running a sync by providing a path to a Sling Replication config:
+
+ ```python
+ from dagster_sling import sling_assets, SlingResource, SlingConnectionResource
+
+ sling_resource = SlingResource(
+ connections=[
+ SlingConnectionResource(
+ name="MY_POSTGRES", type="postgres", connection_string=EnvVar("POSTGRES_URL")
+ ),
+ SlingConnectionResource(
+ name="MY_DUCKDB",
+ type="duckdb",
+ connection_string="duckdb:///var/tmp/duckdb.db",
+ ),
+ ]
+ )
+
+ config_path = "/path/to/replication.yaml"
+ @sling_assets(replication_config=config_path)
+ def my_assets(context, sling: SlingResource):
+ yield from sling.replicate(context=context)
+ ```
+
+
+
+
+
+
class dagster_sling.DagsterSlingTranslator
+
+
+
+
get_asset_key
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).key` instead..
+
+ :::
+
+ A function that takes a stream definition from a Sling replication config and returns a
+ Dagster AssetKey.
+
+ The stream definition is a dictionary key/value pair where the key is the stream name and
+ the value is a dictionary representing the Sling Replication Stream Config.
+
+ For example:
+
+ ```python
+ stream_definition = {"public.users":
+ {'sql': 'select all_user_id, name from public."all_Users"',
+ 'object': 'public.all_users'}
+ }
+ ```
+ By default, this returns the class’s target_prefix parameter concatenated with the stream name.
+ A stream named “public.accounts” will create an AssetKey named “target_public_accounts”.
+
+ Override this function to customize how to map a Sling stream to a Dagster AssetKey.
+
+ Alternatively, you can provide metadata in your Sling replication config to specify the
+ Dagster AssetKey for a stream as follows:
+
+ ```yaml
+ public.users:
+ meta:
+ dagster:
+ asset_key: "mydb_users"
+ ```
+ Parameters: stream_definition (Mapping[str, Any]) – A dictionary representing the stream definitionReturns: The Dagster AssetKey for the replication stream.Return type: [AssetKey](../assets.mdx#dagster.AssetKey)
+ Examples:
+
+ Using a custom mapping for streams:
+
+ ```python
+ class CustomSlingTranslator(DagsterSlingTranslator):
+ def get_asset_spec(self, stream_definition: Mapping[str, Any]) -> AssetKey:
+ default_spec = super().get_asset_spec(stream_definition)
+ map = {"stream1": "asset1", "stream2": "asset2"}
+ return default_spec.replace_attributes(key=AssetKey(map[stream_definition["name"]]))
+ ```
+
+
+
+
+
+
get_asset_spec
+
+
+ A function that takes a stream definition from a Sling replication config and returns a
+ Dagster AssetSpec.
+
+ The stream definition is a dictionary key/value pair where the key is the stream name and
+ the value is a dictionary representing the Sling Replication Stream Config.
+
+
+
+
+
+
+
get_auto_materialize_policy
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).auto_materialize_policy` instead..
+
+ :::
+
+ Defines the auto-materialize policy for a given stream definition.
+
+ This method checks the provided stream definition for a specific configuration
+ indicating an auto-materialize policy. If the configuration is found, it returns
+ an eager auto-materialize policy. Otherwise, it returns None.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: An eager auto-materialize policy if the configuration
+ is found, otherwise None.Return type: Optional[AutoMaterializePolicy]
+
+
+
+
+
+
get_deps_asset_key
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Iterate over `DagsterSlingTranslator.get_asset_spec(...).deps` to access `AssetDep.asset_key` instead..
+
+ :::
+
+ A function that takes a stream definition from a Sling replication config and returns a
+ Dagster AssetKey for each dependency of the replication stream.
+
+ By default, this returns the stream name. For example, a stream named “public.accounts”
+ will create an AssetKey named “target_public_accounts” and a dependency named “public_accounts”.
+
+ Override this function to customize how to map a Sling stream to a Dagster dependency.
+ Alternatively, you can provide metadata in your Sling replication config to specify the
+ Dagster AssetKey for a stream as follows:
+
+ ```yaml
+ public.users:
+ meta:
+ dagster:
+ deps: "sourcedb_users"
+ ```
+ Parameters: stream_definition (Mapping[str, Any]) – A dictionary representing the stream definitionReturns: A list of Dagster AssetKey for each dependency of the replication stream.Return type: Iterable[[AssetKey](../assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
get_description
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).description` instead..
+
+ :::
+
+ Retrieves the description for a given stream definition.
+
+ This method checks the provided stream definition for a description. It first looks
+ for an “sql” key in the configuration and returns its value if found. If not, it looks
+ for a description in the metadata under the “dagster” key.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: The description of the stream if found, otherwise None.Return type: Optional[str]
+
+
+
+
+
+
get_freshness_policy
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).freshness_policy` instead..
+
+ :::
+
+ Retrieves the freshness policy for a given stream definition.
+
+ This method checks the provided stream definition for a specific configuration
+ indicating a freshness policy. If the configuration is found, it constructs and
+ returns a FreshnessPolicy object based on the provided parameters. Otherwise,
+ it returns None.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: A FreshnessPolicy object if the configuration is found,
+ otherwise None.Return type: Optional[FreshnessPolicy]
+
+
+
+
+
+
get_group_name
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).group_name` instead..
+
+ :::
+
+ Retrieves the group name for a given stream definition.
+
+ This method checks the provided stream definition for a group name in the metadata
+ under the “dagster” key.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: The group name if found, otherwise None.Return type: Optional[str]
+
+
+
+
+
+
get_kinds
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).kinds` instead..
+
+ :::
+
+ Retrieves the kinds for a given stream definition.
+
+ This method returns “sling” by default. This method can be overridden to provide custom kinds.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: A set containing kinds for the stream’s assets.Return type: Set[str]
+
+
+
+
+
+
get_metadata
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).metadata` instead..
+
+ :::
+
+ Retrieves the metadata for a given stream definition.
+
+ This method extracts the configuration from the provided stream definition and returns
+ it as a JSON metadata value.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: A dictionary containing the stream configuration as JSON metadata.Return type: Mapping[str, Any]
+
+
+
+
+
+
get_tags
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).tags` instead..
+
+ :::
+
+ Retrieves the tags for a given stream definition.
+
+ This method returns an empty dictionary, indicating that no tags are associated with
+ the stream definition by default. This method can be overridden to provide custom tags.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: An empty dictionary.Return type: Mapping[str, Any]
+
+
+
+
+
+
sanitize_stream_name
+
+
+ A function that takes a stream name from a Sling replication config and returns a
+ sanitized name for the stream.
+
+ By default, this removes any non-alphanumeric characters from the stream name and replaces
+ them with underscores, while removing any double quotes.
+
+ Parameters: stream_name (str) – The name of the stream.
+ Examples:
+
+ Using a custom stream name sanitizer:
+
+ ```python
+ class CustomSlingTranslator(DagsterSlingTranslator):
+ def sanitize_stream_name(self, stream_name: str) -> str:
+ return stream_name.replace(".", "")
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Resources
+
+
+
class dagster_sling.SlingResource
+
+
+ Resource for interacting with the Sling package. This resource can be used to run Sling replications.
+
+ Parameters: connections (List[[*SlingConnectionResource*](#dagster_sling.SlingConnectionResource)]) – A list of connections to use for the replication.
+ Examples:
+
+ ```python
+ from dagster_etl.sling import SlingResource, SlingConnectionResource
+
+ sling_resource = SlingResource(
+ connections=[
+ SlingConnectionResource(
+ name="MY_POSTGRES",
+ type="postgres",
+ connection_string=EnvVar("POSTGRES_CONNECTION_STRING"),
+ ),
+ SlingConnectionResource(
+ name="MY_SNOWFLAKE",
+ type="snowflake",
+ host=EnvVar("SNOWFLAKE_HOST"),
+ user=EnvVar("SNOWFLAKE_USER"),
+ database=EnvVar("SNOWFLAKE_DATABASE"),
+ password=EnvVar("SNOWFLAKE_PASSWORD"),
+ role=EnvVar("SNOWFLAKE_ROLE"),
+ ),
+ ]
+ )
+ ```
+
+
+
+
+
+
class dagster_sling.SlingConnectionResource
+
+
+ A representation of a connection to a database or file to be used by Sling. This resource can be used as a source or a target for a Sling syncs.
+
+ Reference the Sling docs for more information on possible connection types and parameters: [https://docs.slingdata.io/connections](https://docs.slingdata.io/connections)
+
+ The name of the connection is passed to Sling and must match the name of the connection provided in the replication configuration: [https://docs.slingdata.io/sling-cli/run/configuration/replication](https://docs.slingdata.io/sling-cli/run/configuration/replication)
+ You may provide either a connection string or keyword arguments for the connection.
+
+ Examples:
+
+ Creating a Sling Connection for a file, such as CSV or JSON:
+
+ ```python
+ source = SlingConnectionResource(name="MY_FILE", type="file")
+ ```
+ Create a Sling Connection for a Postgres database, using a connection string:
+
+ ```python
+ postgres_conn = SlingConnectionResource(name="MY_POSTGRES", type="postgres", connection_string=EnvVar("POSTGRES_CONNECTION_STRING"))
+ mysql_conn = SlingConnectionResource(name="MY_MYSQL", type="mysql", connection_string="mysql://user:password@host:port/schema")
+ ```
+ Create a Sling Connection for a Postgres or Snowflake database, using keyword arguments:
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-snowflake-pandas.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-snowflake-pandas.mdx
new file mode 100644
index 0000000000000..af8c2faca8d0f
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-snowflake-pandas.mdx
@@ -0,0 +1,238 @@
+---
+title: 'snowflake with pandas (dagster-snowflake-pandas)'
+title_meta: 'snowflake with pandas (dagster-snowflake-pandas) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'snowflake with pandas (dagster-snowflake-pandas) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Snowflake with Pandas (dagster-snowflake-pandas)
+
+This library provides an integration with the [Snowflake](https://www.snowflake.com/) data
+warehouse and Pandas data processing library.
+
+To use this library, you should first ensure that you have an appropriate [Snowflake user](https://docs.snowflake.net/manuals/user-guide/admin-user-management.html) configured to access
+your data warehouse.
+
+Related Guides:
+
+ - [Using Dagster with Snowflake guide](https://docs.dagster.io/integrations/snowflake/)
+ - [Snowflake I/O manager reference](https://docs.dagster.io/integrations/libraries/snowflake/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes Pandas DataFrames to Snowflake. When
+ using the SnowflakePandasIOManager, any inputs and outputs without type annotations will be loaded
+ as Pandas DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_snowflake_pandas import SnowflakePandasIOManager
+ from dagster import asset, Definitions, EnvVar
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": SnowflakePandasIOManager(database="MY_DATABASE", account=EnvVar("SNOWFLAKE_ACCOUNT"), ...)
+ }
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={
+ "io_manager" SnowflakePandasIOManager(database="my_database", schema="my_schema", ...)
+ }
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_snowflake_pandas.SnowflakePandasTypeHandler
+
+
+ Plugin for the Snowflake I/O Manager that can store and load Pandas DataFrames as Snowflake tables.
+
+ Examples:
+
+ ```python
+ from dagster_snowflake import SnowflakeIOManager
+ from dagster_snowflake_pandas import SnowflakePandasTypeHandler
+ from dagster_snowflake_pyspark import SnowflakePySparkTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MySnowflakeIOManager(SnowflakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [SnowflakePandasTypeHandler(), SnowflakePySparkTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MySnowflakeIOManager(database="MY_DATABASE", account=EnvVar("SNOWFLAKE_ACCOUNT"), ...)
+ }
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes Pandas DataFrames to Snowflake. When
+ using the snowflake_pandas_io_manager, any inputs and outputs without type annotations will be loaded
+ as Pandas DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_snowflake_pandas import snowflake_pandas_io_manager
+ from dagster import asset, Definitions
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": snowflake_pandas_io_manager.configured({
+ "database": "my_database",
+ "account" : {"env": "SNOWFLAKE_ACCOUNT"}
+ ...
+ })
+ }
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={"io_manager" snowflake_pandas_io_manager.configured(
+ {"database": "my_database", "schema": "my_schema", ...} # will be used as the schema
+ )}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-snowflake-pyspark.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-snowflake-pyspark.mdx
new file mode 100644
index 0000000000000..66199ad40a73a
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-snowflake-pyspark.mdx
@@ -0,0 +1,251 @@
+---
+title: 'snowflake with pyspark (dagster-snowflake-pyspark)'
+title_meta: 'snowflake with pyspark (dagster-snowflake-pyspark) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'snowflake with pyspark (dagster-snowflake-pyspark) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Snowflake with PySpark (dagster-snowflake-pyspark)
+
+This library provides an integration with the [Snowflake](https://www.snowflake.com/) data
+warehouse and PySpark data processing library.
+
+To use this library, you should first ensure that you have an appropriate [Snowflake user](https://docs.snowflake.net/manuals/user-guide/admin-user-management.html) configured to access
+your data warehouse.
+
+Related Guides:
+
+ - [Using Dagster with Snowflake guide](https://docs.dagster.io/integrations/libraries/snowflake/)
+ - [Snowflake I/O manager reference](https://docs.dagster.io/integrations/libraries/snowflake/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to Snowflake. When
+ using the SnowflakePySparkIOManager, any inputs and outputs without type annotations will be loaded
+ as PySpark DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_snowflake_pyspark import SnowflakePySparkIOManager
+ from pyspark.sql import DataFrame
+ from dagster import Definitions, EnvVar
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": SnowflakePySparkIOManager(
+ database="my_database",
+ warehouse="my_warehouse", # required for SnowflakePySparkIOManager
+ account=EnvVar("SNOWFLAKE_ACCOUNT"),
+ password=EnvVar("SNOWFLAKE_PASSWORD"),
+ ...
+ )
+ }
+ )
+ ```
+ Note that the warehouse configuration value is required when using the SnowflakePySparkIOManager
+
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={
+ "io_manager" SnowflakePySparkIOManager(database="my_database", schema="my_schema", ...)
+ }
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: DataFrame) -> DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_snowflake_pyspark.SnowflakePySparkTypeHandler
+
+
+ Plugin for the Snowflake I/O Manager that can store and load PySpark DataFrames as Snowflake tables.
+
+ Examples:
+
+ ```python
+ from dagster_snowflake import SnowflakeIOManager
+ from dagster_snowflake_pandas import SnowflakePandasTypeHandler
+ from dagster_snowflake_pyspark import SnowflakePySparkTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MySnowflakeIOManager(SnowflakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [SnowflakePandasTypeHandler(), SnowflakePySparkTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MySnowflakeIOManager(database="MY_DATABASE", account=EnvVar("SNOWFLAKE_ACCOUNT"), warehouse="my_warehouse", ...)
+ }
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to Snowflake. When
+ using the snowflake_pyspark_io_manager, any inputs and outputs without type annotations will be loaded
+ as PySpark DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_snowflake_pyspark import snowflake_pyspark_io_manager
+ from pyspark.sql import DataFrame
+ from dagster import Definitions
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": snowflake_pyspark_io_manager.configured({
+ "database": "my_database",
+ "warehouse": "my_warehouse", # required for snowflake_pyspark_io_manager
+ "account" : {"env": "SNOWFLAKE_ACCOUNT"},
+ "password": {"env": "SNOWFLAKE_PASSWORD"},
+ ...
+ })
+ }
+ )
+ ```
+ Note that the warehouse configuration value is required when using the snowflake_pyspark_io_manager
+
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={"io_manager" snowflake_pyspark_io_manager.configured(
+ {"database": "my_database", "schema": "my_schema", ...} # will be used as the schema
+ )}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: DataFrame) -> DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-snowflake.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-snowflake.mdx
new file mode 100644
index 0000000000000..146e5e22c4aab
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-snowflake.mdx
@@ -0,0 +1,522 @@
+---
+title: 'snowflake (dagster-snowflake)'
+title_meta: 'snowflake (dagster-snowflake) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'snowflake (dagster-snowflake) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Snowflake (dagster-snowflake)
+
+This library provides an integration with the [Snowflake](https://www.snowflake.com/) data
+warehouse.
+
+To use this library, you should first ensure that you have an appropriate [Snowflake user](https://docs.snowflake.net/manuals/user-guide/admin-user-management.html) configured to access
+your data warehouse.
+
+Related Guides:
+
+ - [Using Dagster with Snowflake](https://docs.dagster.io/integrations/libraries/snowflake/)
+ - [Snowflake I/O manager reference](https://docs.dagster.io/integrations/libraries/snowflake/reference)
+ - [Transitioning data pipelines from development to production](https://docs.dagster.io/guides/deploy/dev-to-prod)
+ - [Testing against production with Dagster+ Branch Deployments](https://docs.dagster.io/dagster-plus/features/ci-cd/branch-deployments/)
+
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to Snowflake.
+
+ Examples:
+
+ ```python
+ from dagster_snowflake import SnowflakeIOManager
+ from dagster_snowflake_pandas import SnowflakePandasTypeHandler
+ from dagster_snowflake_pyspark import SnowflakePySparkTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MySnowflakeIOManager(SnowflakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [SnowflakePandasTypeHandler(), SnowflakePySparkTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MySnowflakeIOManager(database="my_database", account=EnvVar("SNOWFLAKE_ACCOUNT"), ...)
+ }
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={
+ "io_manager" MySnowflakeIOManager(database="my_database", schema="my_schema", ...)
+ }
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata `columns` to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+ A resource for connecting to the Snowflake data warehouse.
+
+ If connector configuration is not set, SnowflakeResource.get_connection() will return a
+ [snowflake.connector.Connection](https://docs.snowflake.com/en/developer-guide/python-connector/python-connector-api#object-connection)
+ object. If connector=”sqlalchemy” configuration is set, then SnowflakeResource.get_connection() will
+ return a [SQLAlchemy Connection](https://docs.sqlalchemy.org/en/20/core/connections.html#sqlalchemy.engine.Connection)
+ or a [SQLAlchemy raw connection](https://docs.sqlalchemy.org/en/20/core/connections.html#sqlalchemy.engine.Engine.raw_connection).
+
+ A simple example of loading data into Snowflake and subsequently querying that data is shown below:
+
+ Examples:
+
+ ```python
+ from dagster import job, op
+ from dagster_snowflake import SnowflakeResource
+
+ @op
+ def get_one(snowflake_resource: SnowflakeResource):
+ with snowflake_resource.get_connection() as conn:
+ # conn is a snowflake.connector.Connection object
+ conn.cursor().execute("SELECT 1")
+
+ @job
+ def my_snowflake_job():
+ get_one()
+
+ my_snowflake_job.execute_in_process(
+ resources={
+ 'snowflake_resource': SnowflakeResource(
+ account=EnvVar("SNOWFLAKE_ACCOUNT"),
+ user=EnvVar("SNOWFLAKE_USER"),
+ password=EnvVar("SNOWFLAKE_PASSWORD")
+ database="MY_DATABASE",
+ schema="MY_SCHEMA",
+ warehouse="MY_WAREHOUSE"
+ )
+ }
+ )
+ ```
+
+
+
+
+
+
class dagster_snowflake.SnowflakeConnection
+
+
+ A connection to Snowflake that can execute queries. In general this class should not be
+ directly instantiated, but rather used as a resource in an op or asset via the
+ [`snowflake_resource()`](#dagster_snowflake.snowflake_resource).
+
+ Note that the SnowflakeConnection is only used by the snowflake_resource. The Pythonic SnowflakeResource does
+ not use this SnowflakeConnection class.
+
+
+
execute_queries
+
+
+ Execute multiple queries in Snowflake.
+
+ Parameters:
+ - sql_queries (str) – List of queries to be executed in series
+ - parameters (Optional[Union[Sequence[Any], Mapping[Any, Any]]]) – Parameters to be passed to every query. See the
+ - fetch_results (bool) – If True, will return the results of the queries as a list. Defaults to False. If True
+ - use_pandas_result (bool) – If True, will return the results of the queries as a list of a Pandas DataFrames.
+
+
+ Returns: The results of the queries as a list if fetch_results or use_pandas_result is True,
+ otherwise returns None
+ Examples:
+
+ ```python
+ @op
+ def create_fresh_database(snowflake: SnowflakeResource):
+ queries = ["DROP DATABASE IF EXISTS MY_DATABASE", "CREATE DATABASE MY_DATABASE"]
+ snowflake.execute_queries(
+ sql_queries=queries
+ )
+ ```
+
+
+
+
+
+
execute_query
+
+
+ Execute a query in Snowflake.
+
+ Parameters:
+ - sql (str) – the query to be executed
+ - parameters (Optional[Union[Sequence[Any], Mapping[Any, Any]]]) – Parameters to be passed to the query. See the
+ - fetch_results (bool) – If True, will return the result of the query. Defaults to False. If True
+ - use_pandas_result (bool) – If True, will return the result of the query as a Pandas DataFrame.
+
+
+ Returns: The result of the query if fetch_results or use_pandas_result is True, otherwise returns None
+ Examples:
+
+ ```python
+ @op
+ def drop_database(snowflake: SnowflakeResource):
+ snowflake.execute_query(
+ "DROP DATABASE IF EXISTS MY_DATABASE"
+ )
+ ```
+
+
+
+
+
+
get_connection
+
+
+ Gets a connection to Snowflake as a context manager.
+
+ If using the execute_query, execute_queries, or load_table_from_local_parquet methods,
+ you do not need to create a connection using this context manager.
+
+ Parameters: raw_conn (bool) – If using the sqlalchemy connector, you can set raw_conn to True to create a raw
+ connection. Defaults to True.
+ Examples:
+
+ ```python
+ @op(
+ required_resource_keys={"snowflake"}
+ )
+ def get_query_status(query_id):
+ with context.resources.snowflake.get_connection() as conn:
+ # conn is a Snowflake Connection object or a SQLAlchemy Connection if
+ # sqlalchemy is specified as the connector in the Snowflake Resource config
+
+ return conn.get_query_status(query_id)
+ ```
+
+
+
+
+
+
load_table_from_local_parquet
+
+
+ Stores the content of a parquet file to a Snowflake table.
+
+ Parameters:
+ - src (str) – the name of the file to store in Snowflake
+ - table (str) – the name of the table to store the data. If the table does not exist, it will
+
+
+ Examples:
+
+ ```python
+ import pandas as pd
+ import pyarrow as pa
+ import pyarrow.parquet as pq
+
+ @op
+ def write_parquet_file(snowflake: SnowflakeResource):
+ df = pd.DataFrame({"one": [1, 2, 3], "ten": [11, 12, 13]})
+ table = pa.Table.from_pandas(df)
+ pq.write_table(table, "example.parquet')
+ snowflake.load_table_from_local_parquet(
+ src="example.parquet",
+ table="MY_TABLE"
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Data Freshness
+
+
+
dagster_snowflake.fetch_last_updated_timestamps
+
+
+ Fetch the last updated times of a list of tables in Snowflake.
+
+ If the underlying query to fetch the last updated time returns no results, a ValueError will be raised.
+
+ Parameters:
+ - snowflake_connection (Union[SqlDbConnection, [*SnowflakeConnection*](#dagster_snowflake.SnowflakeConnection)]) – A connection to Snowflake.
+ - schema (str) – The schema of the tables to fetch the last updated time for.
+ - tables (Sequence[str]) – A list of table names to fetch the last updated time for.
+ - database (Optional[str]) – The database of the table. Only required if the connection
+ - ignore_missing_tables (Optional[bool]) – If True, tables not found in Snowflake
+
+
+ Returns: A dictionary of table names to their last updated time in UTC.Return type: Mapping[str, datetime]
+
+
+
+
+
+
+
+
+
+
+## Ops
+
+
+
dagster_snowflake.snowflake_op_for_query
+
+
+ This function is an op factory that constructs an op to execute a snowflake query.
+
+ Note that you can only use snowflake_op_for_query if you know the query you’d like to
+ execute at graph construction time. If you’d like to execute queries dynamically during
+ job execution, you should manually execute those queries in your custom op using the
+ snowflake resource.
+
+ Parameters:
+ - sql (str) – The sql query that will execute against the provided snowflake resource.
+ - parameters (dict) – The parameters for the sql query.
+
+
+ Returns: Returns the constructed op definition.Return type: [OpDefinition](../ops.mdx#dagster.OpDefinition)
+
+
+
+ Builds an IO manager definition that reads inputs from and writes outputs to Snowflake.
+
+ Parameters:
+ - type_handlers (Sequence[DbTypeHandler]) – Each handler defines how to translate between
+ - default_load_type (Type) – When an input has no type annotation, load it as this type.
+
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_snowflake import build_snowflake_io_manager
+ from dagster_snowflake_pandas import SnowflakePandasTypeHandler
+ from dagster_snowflake_pyspark import SnowflakePySparkTypeHandler
+ from dagster import Definitions
+
+ @asset(
+ key_prefix=["my_prefix"]
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_second_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ snowflake_io_manager = build_snowflake_io_manager([SnowflakePandasTypeHandler(), SnowflakePySparkTypeHandler()])
+
+ defs = Definitions(
+ assets=[my_table, my_second_table],
+ resources={
+ "io_manager": snowflake_io_manager.configured({
+ "database": "my_database",
+ "account" : {"env": "SNOWFLAKE_ACCOUNT"}
+ ...
+ })
+ }
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={"io_manager" snowflake_io_manager.configured(
+ {"database": "my_database", "schema": "my_schema", ...} # will be used as the schema
+ )}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata `columns` to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+ A resource for connecting to the Snowflake data warehouse. The returned resource object is an
+ instance of [`SnowflakeConnection`](#dagster_snowflake.SnowflakeConnection).
+
+ A simple example of loading data into Snowflake and subsequently querying that data is shown below:
+
+ Examples:
+
+ ```python
+ from dagster import job, op
+ from dagster_snowflake import snowflake_resource
+
+ @op(required_resource_keys={'snowflake'})
+ def get_one(context):
+ context.resources.snowflake.execute_query('SELECT 1')
+
+ @job(resource_defs={'snowflake': snowflake_resource})
+ def my_snowflake_job():
+ get_one()
+
+ my_snowflake_job.execute_in_process(
+ run_config={
+ 'resources': {
+ 'snowflake': {
+ 'config': {
+ 'account': {'env': 'SNOWFLAKE_ACCOUNT'},
+ 'user': {'env': 'SNOWFLAKE_USER'},
+ 'password': {'env': 'SNOWFLAKE_PASSWORD'},
+ 'database': {'env': 'SNOWFLAKE_DATABASE'},
+ 'schema': {'env': 'SNOWFLAKE_SCHEMA'},
+ 'warehouse': {'env': 'SNOWFLAKE_WAREHOUSE'},
+ }
+ }
+ }
+ }
+ )
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-spark.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-spark.mdx
new file mode 100644
index 0000000000000..f27607b42761b
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-spark.mdx
@@ -0,0 +1,62 @@
+---
+title: 'spark (dagster-spark)'
+title_meta: 'spark (dagster-spark) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'spark (dagster-spark) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Spark (dagster-spark)
+
+
+
class dagster_spark.SparkOpError
+
+
+
+
+
+
+
dagster_spark.define_spark_config
+
+
+ Spark configuration.
+
+ See the Spark documentation for reference:
+ [https://spark.apache.org/docs/latest/submitting-applications.html](https://spark.apache.org/docs/latest/submitting-applications.html)
+
+
+
+
+
+
+
dagster_spark.create_spark_op
+
+
+
+
+
+
+
dagster_spark.construct_spark_shell_command
+
+ Constructs the spark-submit command for a Spark job.
+
+
+
+
+
+## Legacy
+
+
+
dagster_spark.spark_resource ResourceDefinition
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-ssh.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-ssh.mdx
new file mode 100644
index 0000000000000..84ef25f121454
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-ssh.mdx
@@ -0,0 +1,25 @@
+---
+title: 'ssh / sftp (dagster-ssh)'
+title_meta: 'ssh / sftp (dagster-ssh) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'ssh / sftp (dagster-ssh) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# SSH / SFTP (dagster-ssh)
+
+This library provides an integration with SSH and SFTP.
+
+
+
dagster_ssh.ssh_resource ResourceDefinition
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-tableau.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-tableau.mdx
new file mode 100644
index 0000000000000..b4e0a6afaa658
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-tableau.mdx
@@ -0,0 +1,137 @@
+---
+title: 'tableau (dagster-tableau)'
+title_meta: 'tableau (dagster-tableau) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'tableau (dagster-tableau) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Tableau (dagster-tableau)
+
+Dagster allows you to represent your Tableau workspace as assets, alongside other your other
+technologies like dbt and Sling. This allows you to see how your Tableau assets are connected to
+your other data assets, and how changes to other data assets might impact your Tableau workspace.
+
+
+
+
+## Tableau API
+
+Here, we provide interfaces to manage Tableau projects using the Tableau API.
+
+
+
+### Assets (Tableau API)
+
+
+
class dagster_tableau.TableauCloudWorkspace
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Represents a workspace in Tableau Cloud and provides utilities
+ to interact with Tableau APIs.
+
+
+
+
+
+
+
class dagster_tableau.TableauServerWorkspace
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Represents a workspace in Tableau Server and provides utilities
+ to interact with Tableau APIs.
+
+
+
+
+
+
+
class dagster_tableau.DagsterTableauTranslator
+
+ Translator class which converts raw response data from the Tableau API into AssetSpecs.
+ Subclass this class to implement custom logic for each type of Tableau content.
+
+
+
+
+
dagster_tableau.load_tableau_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Tableau content in the workspace.
+
+ Parameters:
+ - workspace (Union[[*TableauCloudWorkspace*](#dagster_tableau.TableauCloudWorkspace), [*TableauServerWorkspace*](#dagster_tableau.TableauServerWorkspace)]) – The Tableau workspace to fetch assets from.
+ - dagster_tableau_translator (Optional[Union[[*DagsterTableauTranslator*](#dagster_tableau.DagsterTableauTranslator), Type[[*DagsterTableauTranslator*](#dagster_tableau.DagsterTableauTranslator)]]]) – The translator to use to convert Tableau content into [`dagster.AssetSpec`](../assets.mdx#dagster.AssetSpec).
+
+
+ Returns: The set of assets representing the Tableau content in the workspace.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns the AssetsDefinition of the materializable assets in the Tableau workspace.
+
+ Parameters:
+ - resource_key (str) – The resource key to use for the Tableau resource.
+ - specs (Sequence[[*AssetSpec*](../assets.mdx#dagster.AssetSpec)]) – The asset specs of the executable assets in the Tableau workspace.
+ - refreshable_workbook_ids (Optional[Sequence[str]]) –
+
+ A list of workbook IDs. The workbooks provided must
+ have extracts as data sources and be refreshable in Tableau.
+
+ When materializing your Tableau assets, the workbooks provided are refreshed,
+ refreshing their sheets and dashboards before pulling their data in Dagster.
+
+
+
+ Returns: The AssetsDefinition of the executable assets in the Tableau workspace.Return type: [AssetsDefinition](../assets.mdx#dagster.AssetsDefinition)
+
+
+
+ Parses a list of Tableau AssetSpecs provided as input and return two lists of AssetSpecs,
+ one for the Tableau external assets and another one for the Tableau materializable assets.
+
+ In Tableau, data sources are considered external assets,
+ while sheets and dashboards are considered materializable assets.
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-twilio.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-twilio.mdx
new file mode 100644
index 0000000000000..62f3759256415
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-twilio.mdx
@@ -0,0 +1,42 @@
+---
+title: 'twilio (dagster-twilio)'
+title_meta: 'twilio (dagster-twilio) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'twilio (dagster-twilio) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Twilio (dagster-twilio)
+
+This library provides an integration with Twilio.
+
+
+
dagster_twilio.TwilioResource ResourceDefinition
+
+
+
+
+ This resource is for connecting to Twilio.
+
+
+
+
+
+
+
+## Legacy
+
+
+
dagster_twilio.twilio_resource ResourceDefinition
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagster-wandb.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagster-wandb.mdx
new file mode 100644
index 0000000000000..2b8e9422c6338
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagster-wandb.mdx
@@ -0,0 +1,289 @@
+---
+title: 'weights & biases (dagster-wandb)'
+title_meta: 'weights & biases (dagster-wandb) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'weights & biases (dagster-wandb) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Weights & Biases (dagster-wandb)
+
+This library provides a Dagster integration with [Weights & Biases](https://wandb.ai/).
+
+Use Dagster and Weights & Biases (W&B) to orchestrate your MLOps pipelines and maintain ML assets.
+
+The integration with W&B makes it easy within Dagster to:
+
+ - use and create [W&B Artifacts](https://docs.wandb.ai/guides/artifacts).
+ - use and create Registered Models in the [W&B Model Registry](https://docs.wandb.ai/guides/models).
+ - run training jobs on dedicated compute using [W&B Launch](https://docs.wandb.ai/guides/launch).
+ - use the [wandb](https://github.com/wandb/wandb) client in ops and assets.
+
+
+
+
+
+## Useful links
+
+For a complete set of documentation, see [Dagster integration](https://docs.wandb.ai/guides/integrations/dagster) on the W&B website.
+
+For full-code examples, see [examples/with_wandb](https://github.com/dagster-io/dagster/tree/master/examples/with_wandb) in the Dagster’s Github repo.
+
+
+
+
+
+
+
+## Resource
+
+
+
dagster_wandb.wandb_resource ResourceDefinition
+
+
+
+
+ Dagster resource used to communicate with the W&B API. It’s useful when you want to use the
+ wandb client within your ops and assets. It’s a required resources if you are using the W&B IO
+ Manager.
+
+ It automatically authenticates using the provided API key.
+
+ For a complete set of documentation, see [Dagster integration](https://docs.wandb.ai/guides/integrations/dagster).
+
+ To configure this resource, we recommend using the [configured](https://legacy-docs.dagster.io/concepts/configuration/configured) method.
+
+ Example:
+
+ ```python
+ from dagster import job
+ from dagster_wandb import wandb_resource
+
+ my_wandb_resource = wandb_resource.configured({"api_key": {"env": "WANDB_API_KEY"}})
+
+ @job(resource_defs={"wandb_resource": my_wandb_resource})
+ def my_wandb_job():
+ ...
+ ```
+
+
+ Represents an execution error of the W&B Artifacts IO Manager.
+
+
+
+
+
+
+
+
+## Ops
+
+
+
dagster_wandb.run_launch_agent
+
+
+ It starts a Launch Agent and runs it as a long running process until stopped manually.
+
+ Agents are processes that poll launch queues and execute the jobs (or dispatch them to external
+ services to be executed) in order.
+
+ Example:
+
+ ```YAML
+ # config.yaml
+
+ resources:
+ wandb_config:
+ config:
+ entity: my_entity
+ project: my_project
+ ops:
+ run_launch_agent:
+ config:
+ max_jobs: -1
+ queues:
+ - my_dagster_queue
+ ```
+ ```python
+ from dagster_wandb.launch.ops import run_launch_agent
+ from dagster_wandb.resources import wandb_resource
+
+ from dagster import job, make_values_resource
+
+
+ @job(
+ resource_defs={
+ "wandb_config": make_values_resource(
+ entity=str,
+ project=str,
+ ),
+ "wandb_resource": wandb_resource.configured(
+ {"api_key": {"env": "WANDB_API_KEY"}}
+ ),
+ },
+ )
+ def run_launch_agent_example():
+ run_launch_agent()
+ ```
+
+
+
+
+
+
dagster_wandb.run_launch_job
+
+
+ Executes a Launch job.
+
+ A Launch job is assigned to a queue in order to be executed. You can create a queue or use the
+ default one. Make sure you have an active agent listening to that queue. You can run an agent
+ inside your Dagster instance but can also consider using a deployable agent in Kubernetes.
+
+ Example:
+
+ ```YAML
+ # config.yaml
+
+ resources:
+ wandb_config:
+ config:
+ entity: my_entity
+ project: my_project
+ ops:
+ my_launched_job:
+ config:
+ entry_point:
+ - python
+ - train.py
+ queue: my_dagster_queue
+ uri: https://github.com/wandb/example-dagster-integration-with-launch
+ ```
+ ```python
+ from dagster_wandb.launch.ops import run_launch_job
+ from dagster_wandb.resources import wandb_resource
+
+ from dagster import job, make_values_resource
+
+
+ @job(
+ resource_defs={
+ "wandb_config": make_values_resource(
+ entity=str,
+ project=str,
+ ),
+ "wandb_resource": wandb_resource.configured(
+ {"api_key": {"env": "WANDB_API_KEY"}}
+ ),
+ },
+ )
+ def run_launch_job_example():
+ run_launch_job.alias("my_launched_job")() # we rename the job with an alias
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/libraries/dagstermill.mdx b/docs/docs-beta/docs/api/python-api/libraries/dagstermill.mdx
new file mode 100644
index 0000000000000..e65c0cdbdbd27
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/libraries/dagstermill.mdx
@@ -0,0 +1,269 @@
+---
+title: 'dagstermill'
+title_meta: 'dagstermill API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dagstermill Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Dagstermill
+
+This library provides an integration with papermill to allow you to run Jupyter notebooks with Dagster.
+
+Related Guides:
+
+ - [Using Jupyter notebooks with Papermill and Dagster](https://docs.dagster.io/integrations/libraries/jupyter/)
+
+
+
+
dagstermill.define_dagstermill_asset
+
+
+ Creates a Dagster asset for a Jupyter notebook.
+
+ Parameters:
+ - name (str) – The name for the asset
+ - notebook_path (str) – Path to the backing notebook
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the
+ - ins (Optional[Mapping[str, [*AssetIn*](../assets.mdx#dagster.AssetIn)]]) – A dictionary that maps input names to information
+ - deps (Optional[Sequence[Union[[*AssetsDefinition*](../assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](../assets.mdx#dagster.SourceAsset), [*AssetKey*](../assets.mdx#dagster.AssetKey), str]]]) – The assets
+ - config_schema (Optional[[*ConfigSchema*](../config.mdx#dagster.ConfigSchema)) – The configuration schema for the asset’s underlying
+ - metadata (Optional[Dict[str, Any]]) – A dict of metadata entries for the asset.
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the notebook.
+ - description (Optional[str]) – Description of the asset to display in the Dagster UI.
+ - partitions_def (Optional[[*PartitionsDefinition*](../partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset.
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided,
+ - resource_defs (Optional[Mapping[str, [*ResourceDefinition*](../resources.mdx#dagster.ResourceDefinition)]]) – (Experimental) A mapping of resource keys to resource definitions. These resources
+ - io_manager_key (Optional[str]) – A string key for the IO manager used to store the output notebook.
+ - retry_policy (Optional[[*RetryPolicy*](../ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that computes the asset.
+ - save_notebook_on_failure (bool) – If True and the notebook fails during execution, the failed notebook will be
+ - asset_tags (Optional[Dict[str, Any]]) – A dictionary of tags to apply to the asset.
+ - non_argument_deps (Optional[Union[Set[[*AssetKey*](../assets.mdx#dagster.AssetKey)], Set[str]]]) – Deprecated, use deps instead. Set of asset keys that are
+
+
+ Examples:
+
+ ```python
+ from dagstermill import define_dagstermill_asset
+ from dagster import asset, AssetIn, AssetKey
+ from sklearn import datasets
+ import pandas as pd
+ import numpy as np
+
+ @asset
+ def iris_dataset():
+ sk_iris = datasets.load_iris()
+ return pd.DataFrame(
+ data=np.c_[sk_iris["data"], sk_iris["target"]],
+ columns=sk_iris["feature_names"] + ["target"],
+ )
+
+ iris_kmeans_notebook = define_dagstermill_asset(
+ name="iris_kmeans_notebook",
+ notebook_path="/path/to/iris_kmeans.ipynb",
+ ins={
+ "iris": AssetIn(key=AssetKey("iris_dataset"))
+ }
+ )
+ ```
+
+
+
+
+
+
dagstermill.define_dagstermill_op
+
+
+ Wrap a Jupyter notebook in a op.
+
+ Parameters:
+ - name (str) – The name of the op.
+ - notebook_path (str) – Path to the backing notebook.
+ - ins (Optional[Mapping[str, [*In*](../ops.mdx#dagster.In)]]) – The op’s inputs.
+ - outs (Optional[Mapping[str, [*Out*](../ops.mdx#dagster.Out)]]) – The op’s outputs. Your notebook should
+ - required_resource_keys (Optional[Set[str]]) – The string names of any required resources.
+ - output_notebook_name – (Optional[str]): If set, will be used as the name of an injected output
+ - asset_key_prefix (Optional[Union[List[str], str]]) – If set, will be used to prefix the
+ - description (Optional[str]) – If set, description used for op.
+ - tags (Optional[Dict[str, str]]) – If set, additional tags used to annotate op.
+ - io_manager_key (Optional[str]) – If using output_notebook_name, you can additionally provide
+ - save_notebook_on_failure (bool) – If True and the notebook fails during execution, the failed notebook will be
+
+
+ Returns: [`OpDefinition`](../ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
class dagstermill.ConfigurableLocalOutputNotebookIOManager
+
+ Built-in IO Manager for handling output notebook.
+
+
+
+
+
dagstermill.get_context
+
+
+ Get a dagstermill execution context for interactive exploration and development.
+
+ Parameters:
+ - op_config (Optional[Any]) – If specified, this value will be made available on the
+ - resource_defs (Optional[Mapping[str, [*ResourceDefinition*](../resources.mdx#dagster.ResourceDefinition)]]) – Specifies resources to provide to context.
+ - logger_defs (Optional[Mapping[str, [*LoggerDefinition*](../loggers.mdx#dagster.LoggerDefinition)]]) – Specifies loggers to provide to context.
+ - run_config (Optional[dict]) – The config dict with which to construct
+
+
+ Returns: [`DagstermillExecutionContext`](#dagstermill.DagstermillExecutionContext)
+
+
+
+
+
+
dagstermill.yield_event
+
+
+ Yield a dagster event directly from notebook code.
+
+ When called interactively or in development, returns its input.
+
+ Parameters: dagster_event (Union[[`dagster.AssetMaterialization`](../ops.mdx#dagster.AssetMaterialization), [`dagster.ExpectationResult`](../ops.mdx#dagster.ExpectationResult), [`dagster.TypeCheck`](../ops.mdx#dagster.TypeCheck), [`dagster.Failure`](../ops.mdx#dagster.Failure), [`dagster.RetryRequested`](../ops.mdx#dagster.RetryRequested)]) – An event to yield back to Dagster.
+
+
+
+
+
+
dagstermill.yield_result
+
+
+ Yield a result directly from notebook code.
+
+ When called interactively or in development, returns its input.
+
+ Parameters:
+ - value (Any) – The value to yield.
+ - output_name (Optional[str]) – The name of the result to yield (default: `'result'`).
+
+
+
+
+
+
+
+
class dagstermill.DagstermillExecutionContext
+
+
+ Dagstermill-specific execution context.
+
+ Do not initialize directly: use [`dagstermill.get_context()`](#dagstermill.get_context).
+
+
+
property job_def
+
+
+ The job definition for the context.
+
+ This will be a dagstermill-specific shim.
+
+ Type: [`dagster.JobDefinition`](../jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
property job_name
+
+
+ The name of the executing job.
+
+ Type: str
+
+
+
+
+
+
property logging_tags
+
+
+ The logging tags for the context.
+
+ Type: dict
+
+
+
+
+
+
property op_config
+
+
+ A dynamically-created type whose properties allow access to
+ op-specific config.
+
+ Type: collections.namedtuple
+
+
+
+
+
+
property op_def
+
+
+ The op definition for the context.
+
+ In interactive contexts, this may be a dagstermill-specific shim, depending whether an
+ op definition was passed to `dagstermill.get_context`.
+
+ Type: [`dagster.OpDefinition`](../ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
property run
+
+
+ The job run for the context.
+
+ Type: [`dagster.DagsterRun`](../internals.mdx#dagster.DagsterRun)
+
+
+
+
+
+
property run_config
+
+
+ The run_config for the context.
+
+ Type: dict
+
+
+
+
+
+
property run_id
+
+
+ The run_id for the context.
+
+ Type: str
+
+
+
+
+
+
+
+
+
+
class dagstermill.DagstermillError
+
+ Base class for errors raised by dagstermill.
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/loggers.mdx b/docs/docs-beta/docs/api/python-api/loggers.mdx
new file mode 100644
index 0000000000000..6f3e79040bf68
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/loggers.mdx
@@ -0,0 +1,244 @@
+---
+title: 'loggers'
+title_meta: 'loggers API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'loggers Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Loggers
+
+
+
+
+## Built-in loggers
+
+
+
dagster._loggers.colored_console_logger
+
+
+ Core class for defining loggers.
+
+ Loggers are job-scoped logging handlers, which will be automatically invoked whenever
+ dagster messages are logged from within a job.
+
+ Parameters:
+ - logger_fn (Callable[[[*InitLoggerContext*](#dagster.InitLoggerContext)], logging.Logger]) – User-provided function to
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of this logger.
+
+
+
+
+
+
+
+
dagster._loggers.json_console_logger
+
+
+ Core class for defining loggers.
+
+ Loggers are job-scoped logging handlers, which will be automatically invoked whenever
+ dagster messages are logged from within a job.
+
+ Parameters:
+ - logger_fn (Callable[[[*InitLoggerContext*](#dagster.InitLoggerContext)], logging.Logger]) – User-provided function to
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of this logger.
+
+
+
+
+
+
+
+
+
+
+
+
+## Logging from an @op
+
+
+
class dagster.DagsterLogManager
+
+
+ Centralized dispatch for logging from user code.
+
+ Handles the construction of uniform structured log messages and passes them through to the
+ underlying loggers/handlers.
+
+ An instance of the log manager is made available to ops as `context.log`. Users should not
+ initialize instances of the log manager directly. To configure custom loggers, set the
+ `logger_defs` argument in an @job decorator or when calling the to_job() method on a
+ [`GraphDefinition`](graphs.mdx#dagster.GraphDefinition).
+
+ The log manager inherits standard convenience methods like those exposed by the Python standard
+ library `python:logging` module (i.e., within the body of an op,
+ `context.log.\{debug, info, warning, warn, error, critical, fatal}`).
+
+ The underlying integer API can also be called directly using, e.g.
+ `context.log.log(5, msg)`, and the log manager will delegate to the `log` method
+ defined on each of the loggers it manages.
+
+ User-defined custom log levels are not supported, and calls to, e.g.,
+ `context.log.trace` or `context.log.notice` will result in hard exceptions at runtime.
+
+
+
+
+
+
+
+
+
+
+## Defining custom loggers
+
+
+
@dagster.logger
+
+
+ Define a logger.
+
+ The decorated function should accept an [`InitLoggerContext`](#dagster.InitLoggerContext) and return an instance of
+ `python:logging.Logger`. This function will become the `logger_fn` of an underlying
+ [`LoggerDefinition`](#dagster.LoggerDefinition).
+
+ Parameters:
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of the logger.
+
+
+
+
+
+
+
+
class dagster.LoggerDefinition
+
+
+ Core class for defining loggers.
+
+ Loggers are job-scoped logging handlers, which will be automatically invoked whenever
+ dagster messages are logged from within a job.
+
+ Parameters:
+ - logger_fn (Callable[[[*InitLoggerContext*](#dagster.InitLoggerContext)], logging.Logger]) – User-provided function to
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of this logger.
+
+
+
+
property config_schema
+
+
+ The schema for the logger’s config. Configuration data available in init_context.logger_config.
+
+ Type: Any
+
+
+
+
+
+
property description
+
+
+ A human-readable description of the logger.
+
+ Type: Optional[str]
+
+
+
+
+
+
property logger_fn
+
+
+ The function that will be invoked to
+ instantiate the logger.
+
+ Type: Callable[[[InitLoggerContext](#dagster.InitLoggerContext)], logging.Logger]
+
+
+
+
+
+
+
+
+
+
class dagster.InitLoggerContext
+
+
+ The context object available as the argument to the initialization function of a [`dagster.LoggerDefinition`](#dagster.LoggerDefinition).
+
+ Users should not instantiate this object directly. To construct an
+ InitLoggerContext for testing purposes, use `dagster.
+ build_init_logger_context()`.
+
+ Example:
+
+ ```python
+ from dagster import logger, InitLoggerContext
+
+ @logger
+ def hello_world(init_context: InitLoggerContext):
+ ...
+ ```
+
+
property logger_config
+
+ The configuration data provided by the run config. The
+ schema for this data is defined by `config_schema` on the [`LoggerDefinition`](#dagster.LoggerDefinition).
+
+
+
+
+
property logger_def
+
+ The logger definition for the logger being constructed.
+
+
+
+
+
property run_id
+
+ The ID for this run of the job.
+
+
+
+
+
+
+
+
+
dagster.build_init_logger_context
+
+
+ Builds logger initialization context from provided parameters.
+
+ This function can be used to provide the context argument to the invocation of a logger
+ definition.
+
+ Note that you may only specify one of pipeline_def and job_def.
+
+ Parameters:
+ - logger_config (Any) – The config to provide during initialization of logger.
+ - job_def (Optional[[*JobDefinition*](jobs.mdx#dagster.JobDefinition)]) – The job definition that the logger will be used with.
+
+
+ Examples:
+
+ ```python
+ context = build_init_logger_context()
+ logger_to_init(context)
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/metadata.mdx b/docs/docs-beta/docs/api/python-api/metadata.mdx
new file mode 100644
index 0000000000000..e358a3442f4d6
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/metadata.mdx
@@ -0,0 +1,1239 @@
+---
+title: 'metadata'
+title_meta: 'metadata API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'metadata Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Metadata
+
+Dagster uses metadata to communicate arbitrary user-specified metadata about structured
+events.
+
+Refer to the [Metadata](https://docs.dagster.io/guides/build/assets/metadata-and-tags/) documentation for more information.
+
+
+
class dagster.MetadataValue
+
+
+ Utility class to wrap metadata values passed into Dagster events so that they can be
+ displayed in the Dagster UI and other tooling.
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "my_text_label": "hello",
+ "dashboard_url": MetadataValue.url("http://mycoolsite.com/my_dashboard"),
+ "num_rows": 0,
+ },
+ )
+ ```
+
+
static asset
+
+
+ Static constructor for a metadata value referencing a Dagster asset, by key.
+
+ For example:
+
+ ```python
+ @op
+ def validate_table(context, df):
+ yield AssetMaterialization(
+ asset_key=AssetKey("my_table"),
+ metadata={
+ "Related asset": MetadataValue.asset(AssetKey('my_other_table')),
+ },
+ )
+ ```
+ Parameters: asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – The asset key referencing the asset.
+
+
+
+
+
+
static bool
+
+
+ Static constructor for a metadata value wrapping a bool as
+ `BoolMetadataValuye`. Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "num rows > 1000": MetadataValue.bool(len(df) > 1000),
+ },
+ )
+ ```
+ Parameters: value (bool) – The bool value for a metadata entry.
+
+
+
+
+
+
static column_lineage
+
+
+ Static constructor for a metadata value wrapping a column lineage as
+ [`TableColumnLineageMetadataValue`](#dagster.TableColumnLineageMetadataValue). Can be used as the value type
+ for the metadata parameter for supported events.
+
+ Parameters: lineage ([*TableColumnLineage*](#dagster.TableColumnLineage)) – The column lineage for a metadata entry.
+
+
+
+
+
+
static dagster_run
+
+
+ Static constructor for a metadata value wrapping a reference to a Dagster run.
+
+ Parameters: run_id (str) – The ID of the run.
+
+
+
+
+
+
static float
+
+
+ Static constructor for a metadata value wrapping a float as
+ [`FloatMetadataValue`](#dagster.FloatMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "size (bytes)": MetadataValue.float(calculate_bytes(df)),
+ }
+ )
+ ```
+ Parameters: value (float) – The float value for a metadata entry.
+
+
+
+
+
+
static int
+
+
+ Static constructor for a metadata value wrapping an int as
+ [`IntMetadataValue`](#dagster.IntMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "number of rows": MetadataValue.int(len(df)),
+ },
+ )
+ ```
+ Parameters: value (int) – The int value for a metadata entry.
+
+
+
+
+
+
static job
+
+
+ Static constructor for a metadata value referencing a Dagster job, by name.
+
+ For example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset"
+ metadata={
+ "Producing job": MetadataValue.job('my_other_job'),
+ },
+ )
+ ```
+ Parameters:
+ - job_name (str) – The name of the job.
+ - location_name (Optional[str]) – The code location name for the job.
+ - repository_name (Optional[str]) – The repository name of the job, if different from the
+
+
+
+
+
+
+
+
static json
+
+
+ Static constructor for a metadata value wrapping a json-serializable list or dict
+ as [`JsonMetadataValue`](#dagster.JsonMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context):
+ yield ExpectationResult(
+ success=not missing_things,
+ label="is_present",
+ metadata={
+ "about my dataset": MetadataValue.json({"missing_columns": missing_things})
+ },
+ )
+ ```
+ Parameters: data (Union[Sequence[Any], Mapping[str, Any]]) – The JSON data for a metadata entry.
+
+
+
+
+
+
static md
+
+
+ Static constructor for a metadata value wrapping markdown data as
+ [`MarkdownMetadataValue`](#dagster.MarkdownMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, md_str):
+ yield AssetMaterialization(
+ asset_key="info",
+ metadata={
+ 'Details': MetadataValue.md(md_str)
+ },
+ )
+ ```
+ Parameters: md_str (str) – The markdown for a metadata entry.
+
+
+
+
+
+
static notebook
+
+
+ Static constructor for a metadata value wrapping a notebook path as
+ [`NotebookMetadataValue`](#dagster.NotebookMetadataValue).
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "notebook_path": MetadataValue.notebook("path/to/notebook.ipynb"),
+ }
+ )
+ ```
+ Parameters: path (str) – The path to a notebook for a metadata entry.
+
+
+
+
+
+
static null
+
+ Static constructor for a metadata value representing null. Can be used as the value type
+ for the metadata parameter for supported events.
+
+
+
+
+
static path
+
+
+ Static constructor for a metadata value wrapping a path as
+ [`PathMetadataValue`](#dagster.PathMetadataValue).
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "filepath": MetadataValue.path("path/to/file"),
+ }
+ )
+ ```
+ Parameters: path (str) – The path for a metadata entry.
+
+
+
+
+
+
static python_artifact
+
+
+ Static constructor for a metadata value wrapping a python artifact as
+ [`PythonArtifactMetadataValue`](#dagster.PythonArtifactMetadataValue). Can be used as the value type for the
+ metadata parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "class": MetadataValue.python_artifact(MyClass),
+ "function": MetadataValue.python_artifact(my_function),
+ }
+ )
+ ```
+ Parameters: value (Callable) – The python class or function for a metadata entry.
+
+
+
+
+
+
static table
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Static constructor for a metadata value wrapping arbitrary tabular data as
+ [`TableMetadataValue`](#dagster.TableMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context):
+ yield ExpectationResult(
+ success=not has_errors,
+ label="is_valid",
+ metadata={
+ "errors": MetadataValue.table(
+ records=[
+ TableRecord(code="invalid-data-type", row=2, col="name"),
+ ],
+ schema=TableSchema(
+ columns=[
+ TableColumn(name="code", type="string"),
+ TableColumn(name="row", type="int"),
+ TableColumn(name="col", type="string"),
+ ]
+ )
+ ),
+ },
+ )
+ ```
+
+
+
+
+
+
static table_schema
+
+
+ Static constructor for a metadata value wrapping a table schema as
+ [`TableSchemaMetadataValue`](#dagster.TableSchemaMetadataValue). Can be used as the value type
+ for the metadata parameter for supported events.
+
+ Example:
+
+ ```python
+ schema = TableSchema(
+ columns = [
+ TableColumn(name="id", type="int"),
+ TableColumn(name="status", type="bool"),
+ ]
+ )
+
+ DagsterType(
+ type_check_fn=some_validation_fn,
+ name='MyTable',
+ metadata={
+ 'my_table_schema': MetadataValue.table_schema(schema),
+ }
+ )
+ ```
+ Parameters: schema ([*TableSchema*](#dagster.TableSchema)) – The table schema for a metadata entry.
+
+
+
+
+
+
static text
+
+
+ Static constructor for a metadata value wrapping text as
+ [`TextMetadataValue`](#dagster.TextMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "my_text_label": MetadataValue.text("hello")
+ },
+ )
+ ```
+ Parameters: text (str) – The text string for a metadata entry.
+
+
+
+
+
+
static timestamp
+
+
+ Static constructor for a metadata value wrapping a UNIX timestamp as a
+ [`TimestampMetadataValue`](#dagster.TimestampMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Parameters: value (Union[float, datetime]) – The unix timestamp value for a metadata entry. If a
+ datetime is provided, the timestamp will be extracted. datetimes without timezones
+ are not accepted, because their timestamps can be ambiguous.
+
+
+
+
+
+
static url
+
+
+ Static constructor for a metadata value wrapping a URL as
+ [`UrlMetadataValue`](#dagster.UrlMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context):
+ yield AssetMaterialization(
+ asset_key="my_dashboard",
+ metadata={
+ "dashboard_url": MetadataValue.url("http://mycoolsite.com/my_dashboard"),
+ }
+ )
+ ```
+ Parameters: url (str) – The URL for a metadata entry.
+
+
+
+
+
+
abstract property value
+
+ The wrapped value.
+
+
+
+
+
+
+
+
+
class dagster.MetadataEntry
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Please use a dict with `MetadataValue` values instead..
+
+ :::
+
+ A structure for describing metadata for Dagster events.
+
+ Note: This class is no longer usable in any Dagster API, and will be completely removed in 2.0.
+
+ Lists of objects of this type can be passed as arguments to Dagster events and will be displayed
+ in the Dagster UI and other tooling.
+
+ Should be yielded from within an IO manager to append metadata for a given input/output event.
+ For other event types, passing a dict with MetadataValue values to the metadata argument
+ is preferred.
+
+ Parameters:
+ - label (str) – Short display label for this metadata entry.
+ - description (Optional[str]) – A human-readable description of this metadata entry.
+ - value ([*MetadataValue*](#dagster.MetadataValue)) – Typed metadata entry data. The different types allow
+
+
+
+
+
+
+
+
+
+## Metadata types
+
+All metadata types inherit from MetadataValue. The following types are defined:
+
+
+
class dagster.DagsterAssetMetadataValue
+
+
+ Representation of a dagster asset.
+
+ Parameters: asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – The dagster asset key
+
+
+ Container class for markdown metadata entry data.
+
+ Parameters: md_str (Optional[str]) – The markdown as a string.
+
+
property value
+
+
+ The wrapped markdown as a string.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.PathMetadataValue
+
+
+ Container class for path metadata entry data.
+
+ Parameters: path (Optional[str]) – The path as a string or conforming to os.PathLike.
+
+
property value
+
+
+ The wrapped path.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.NotebookMetadataValue
+
+
+ Container class for notebook metadata entry data.
+
+ Parameters: path (Optional[str]) – The path to the notebook as a string or conforming to os.PathLike.
+
+
property value
+
+
+ The wrapped path to the notebook as a string.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.PythonArtifactMetadataValue
+
+
+ Container class for python artifact metadata entry data.
+
+ Parameters:
+ - module (str) – The module where the python artifact can be found
+ - name (str) – The name of the python artifact
+
+
+
+
+ Representation of the lineage of column inputs to column outputs of arbitrary tabular data.
+
+ Parameters: column_lineage ([*TableColumnLineage*](#dagster.TableColumnLineage)) – The lineage of column inputs to column outputs
+ for the table.
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Container class for table metadata entry data.
+
+ Parameters:
+ - records ([*TableRecord*](#dagster.TableRecord)) – The data as a list of records (i.e. rows).
+ - schema (Optional[[*TableSchema*](#dagster.TableSchema)]) – A schema for the table.
+
+
+ Example:
+
+ ```python
+ from dagster import TableMetadataValue, TableRecord
+
+ TableMetadataValue(
+ schema=None,
+ records=[
+ TableRecord({"column1": 5, "column2": "x"}),
+ TableRecord({"column1": 7, "column2": "y"}),
+ ]
+ )
+ ```
+
+
static infer_column_type
+
+ str: Infer the [`TableSchema`](#dagster.TableSchema) column type that will be used for a value.
+
+
+ Representation of a schema for arbitrary tabular data.
+
+ Parameters: schema ([*TableSchema*](#dagster.TableSchema)) – The dictionary containing the schema representation.
+
+
+ Container class for text metadata entry data.
+
+ Parameters: text (Optional[str]) – The text data.
+
+
property value
+
+
+ The wrapped text data.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.TimestampMetadataValue
+
+
+ Container class for metadata value that’s a unix timestamp.
+
+ Parameters: value (float) – Seconds since the unix epoch.
+
+
+
+
+
+
class dagster.UrlMetadataValue
+
+
+ Container class for URL metadata entry data.
+
+ Parameters: url (Optional[str]) – The URL as a string.
+
+
property value
+
+
+ The wrapped URL.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.CodeReferencesMetadataValue
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Metadata value type which represents source locations (locally or otherwise)
+ of the asset in question. For example, the file path and line number where the
+ asset is defined.
+
+
+
sources
+
+
+ A list of code references for the asset, such as file locations or
+ references to source control.
+
+ Type: List[Union[LocalFileCodeReference, SourceControlCodeReference]]
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Tables
+
+These APIs provide the ability to express column schemas (TableSchema), rows/records (TableRecord), and column lineage (TableColumnLineage) in Dagster as metadata.
+
+
+
class dagster.TableRecord
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Represents one record in a table. Field keys are arbitrary strings– field values must be
+ strings, integers, floats, or bools.
+
+
+
+
+
+
+
class dagster.TableSchema
+
+
+ Representation of a schema for tabular data.
+
+ Schema is composed of two parts:
+
+ - A required list of columns (TableColumn). Each column specifies a
+ - An optional list of table-level constraints (TableConstraints). A
+
+
+ ```python
+ # example schema
+ TableSchema(
+ constraints = TableConstraints(
+ other = [
+ "foo > bar",
+ ],
+ ),
+ columns = [
+ TableColumn(
+ name = "foo",
+ type = "string",
+ description = "Foo description",
+ constraints = TableColumnConstraints(
+ nullable = False,
+ other = [
+ "starts with the letter 'a'",
+ ],
+ ),
+ ),
+ TableColumn(
+ name = "bar",
+ type = "string",
+ ),
+ TableColumn(
+ name = "baz",
+ type = "custom_type",
+ constraints = TableColumnConstraints(
+ unique = True,
+ )
+ ),
+ ],
+ )
+ ```
+ Parameters:
+ - columns (List[[*TableColumn*](#dagster.TableColumn)]) – The columns of the table.
+ - constraints (Optional[[*TableConstraints*](#dagster.TableConstraints)]) – The constraints of the table.
+
+
+
+
static from_name_type_dict
+
+ Constructs a TableSchema from a dictionary whose keys are column names and values are the
+ names of data types of those columns.
+
+
+
+
+
+
+
+
+
class dagster.TableConstraints
+
+
+ Descriptor for “table-level” constraints. Presently only one property,
+ other is supported. This contains strings describing arbitrary
+ table-level constraints. A table-level constraint is a constraint defined
+ in terms of multiple columns (e.g. col_A > col_B) or in terms of rows.
+
+ Parameters: other (List[str]) – Descriptions of arbitrary table-level constraints.
+
+
+
+
+
+
class dagster.TableColumn
+
+
+ Descriptor for a table column. The only property that must be specified
+ by the user is name. If no type is specified, string is assumed. If
+ no constraints are specified, the column is assumed to be nullable
+ (i.e. required = False) and have no other constraints beyond the data type.
+
+ Parameters:
+ - name (List[str]) – Descriptions of arbitrary table-level constraints.
+ - type (Optional[str]) – The type of the column. Can be an arbitrary
+ - description (Optional[str]) – Description of this column. Defaults to None.
+ - constraints (Optional[[*TableColumnConstraints*](#dagster.TableColumnConstraints)]) – Column-level constraints.
+ - tags (Optional[Mapping[str, str]]) – Tags for filtering or organizing columns.
+
+
+
+
+
+
+
+
class dagster.TableColumnConstraints
+
+
+ Descriptor for a table column’s constraints. Nullability and uniqueness are specified with
+ boolean properties. All other constraints are described using arbitrary strings under the
+ other property.
+
+ Parameters:
+ - nullable (Optional[bool]) – If true, this column can hold null values.
+ - unique (Optional[bool]) – If true, all values in this column must be unique.
+ - other (List[str]) – Descriptions of arbitrary column-level constraints
+
+
+
+
+
+
+
+
class dagster.TableColumnLineage
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Represents the lineage of column outputs to column inputs for a tabular asset.
+
+ Parameters: deps_by_column (Mapping[str, Sequence[[*TableColumnDep*](#dagster.TableColumnDep)]]) – A mapping from column names to
+ the columns that the column depends on.
+ Examples:
+
+ Defining column lineage at materialization time, where the resulting asset has two columns,
+ `new_column_foo` and `new_column_qux`. The first column, `new_column_foo`, depends on
+ `column_bar` in `source_bar` and `column_baz` in `source_baz`. The second column,
+ `new_column_qux`, depends on `column_quuz` in `source_bar`.
+
+ ```python
+ from dagster import (
+ AssetKey,
+ MaterializeResult,
+ TableColumnDep,
+ TableColumnLineage,
+ asset,
+ )
+
+
+ @asset(deps=[AssetKey("source_bar"), AssetKey("source_baz")])
+ def my_asset():
+ yield MaterializeResult(
+ metadata={
+ "dagster/column_lineage": TableColumnLineage(
+ deps_by_column={
+ "new_column_foo": [
+ TableColumnDep(
+ asset_key=AssetKey("source_bar"),
+ column_name="column_bar",
+ ),
+ TableColumnDep(
+ asset_key=AssetKey("source_baz"),
+ column_name="column_baz",
+ ),
+ ],
+ "new_column_qux": [
+ TableColumnDep(
+ asset_key=AssetKey("source_bar"),
+ column_name="column_quuz",
+ ),
+ ],
+ }
+ )
+ }
+ )
+ ```
+
+
+
+
+
+
class dagster.TableColumnDep
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Object representing an identifier for a column in an asset.
+
+
+
+
+
+
+
+
+
+
+## Code references
+
+The following functions are used to attach source code references to your assets.
+For more information, refer to the [Linking to asset definition code with code references](https://docs.dagster.io/guides/dagster/code-references) guide.
+
+
+
dagster.with_source_code_references
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Wrapper function which attaches local code reference metadata to the provided asset definitions.
+ This points to the filepath and line number where the asset body is defined.
+
+ Parameters: assets_defs (Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset), CacheableAssetsDefinition]]) – The asset definitions to which source code metadata should be attached.Returns: The asset definitions with source code metadata attached.Return type: Sequence[[AssetsDefinition](assets.mdx#dagster.AssetsDefinition)]
+
+
+
+
+
+
dagster.link_code_references_to_git
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Wrapper function which converts local file path code references to source control URLs
+ based on the provided source control URL and branch.
+
+ Parameters:
+ - assets_defs (Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset), CacheableAssetsDefinition]]) – The asset definitions to which source control metadata should be attached.
+ - git_url (str) – The base URL for the source control system. For example,
+ - git_branch (str) – The branch in the source control system, such as “master”.
+ - file_path_mapping ([*FilePathMapping*](#dagster.FilePathMapping)) – Specifies the mapping between local file paths and their corresponding paths in a source control repository.
+
+
+ Example:
+
+ ```python
+ defs = Definitions(
+ assets=link_code_references_to_git(
+ with_source_code_references([my_dbt_assets]),
+ git_url="https://github.com/dagster-io/dagster",
+ git_branch="master",
+ file_path_mapping=AnchorBasedFilePathMapping(
+ local_file_anchor=Path(__file__),
+ file_anchor_path_in_repository="python_modules/my_module/my-module/__init__.py",
+ ),
+ )
+ )
+ ```
+
+
+
+
+
+
class dagster.FilePathMapping
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Base class which defines a file path mapping function. These functions are used to map local file paths
+ to their corresponding paths in a source control repository.
+
+ In many cases where a source control repository is reproduced exactly on a local machine, the included
+ AnchorBasedFilePathMapping class can be used to specify a direct mapping between the local file paths and the
+ repository paths. However, in cases where the repository structure differs from the local structure, a custom
+ mapping function can be provided to handle these cases.
+
+
+
abstract convert_to_source_control_path
+
+
+ Maps a local file path to the corresponding path in a source control repository.
+
+ Parameters: local_path (Path) – The local file path to map.Returns: The corresponding path in the hosted source control repository, relative to the repository root.Return type: str
+
+
+
+
+
+
+
+
+
+
class dagster.AnchorBasedFilePathMapping
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Specifies the mapping between local file paths and their corresponding paths in a source control repository,
+ using a specific file “anchor” as a reference point. All other paths are calculated relative to this anchor file.
+
+ For example, if the chosen anchor file is /Users/dagster/Documents/python_modules/my_module/my-module/__init__.py
+ locally, and python_modules/my_module/my-module/__init__.py in a source control repository, in order to map a
+ different file /Users/dagster/Documents/python_modules/my_module/my-module/my_asset.py to the repository path,
+ the mapping function will position the file in the repository relative to the anchor file’s position in the repository,
+ resulting in python_modules/my_module/my-module/my_asset.py.
+
+ Parameters:
+ - local_file_anchor (Path) – The path to a local file that is present in the repository.
+ - file_anchor_path_in_repository (str) – The path to the anchor file in the repository.
+
+
+ Example:
+
+ ```python
+ mapping_fn = AnchorBasedFilePathMapping(
+ local_file_anchor=Path(__file__),
+ file_anchor_path_in_repository="python_modules/my_module/my-module/__init__.py",
+ )
+ ```
+
+
convert_to_source_control_path
+
+
+ Maps a local file path to the corresponding path in a source control repository
+ based on the anchor file and its corresponding path in the repository.
+
+ Parameters: local_path (Path) – The local file path to map.Returns: The corresponding path in the hosted source control repository, relative to the repository root.Return type: str
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/ops.mdx b/docs/docs-beta/docs/api/python-api/ops.mdx
new file mode 100644
index 0000000000000..9997cc38eb341
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/ops.mdx
@@ -0,0 +1,575 @@
+---
+title: 'ops'
+title_meta: 'ops API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'ops Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Ops
+
+The foundational unit of computation in Dagster.
+
+
+
+
+## Defining ops
+
+
+
@dagster.op
+
+
+ Create an op with the specified parameters from the decorated function.
+
+ Ins and outs will be inferred from the type signature of the decorated function
+ if not explicitly provided.
+
+ The decorated function will be used as the op’s compute function. The signature of the
+ decorated function is more flexible than that of the `compute_fn` in the core API; it may:
+
+ 1. Return a value. This value will be wrapped in an [`Output`](#dagster.Output) and yielded by the compute function.
+ 2. Return an [`Output`](#dagster.Output). This output will be yielded by the compute function.
+ 3. Yield [`Output`](#dagster.Output) or other [event objects](#events)`event objects`. Same as default compute behavior.
+ Note that options 1) and 2) are incompatible with yielding other events – if you would like
+ to decorate a function that yields events, it must also wrap its eventual output in an
+ [`Output`](#dagster.Output) and yield it.
+
+ @op supports `async def` functions as well, including async generators when yielding multiple
+ events or outputs. Note that async ops will generally be run on their own unless using a custom
+ [`Executor`](internals.mdx#dagster.Executor) implementation that supports running them together.
+
+ Parameters:
+ - name (Optional[str]) – Name of op. Must be unique within any [`GraphDefinition`](graphs.mdx#dagster.GraphDefinition)
+ - description (Optional[str]) – Human-readable description of this op. If not provided, and
+ - ins (Optional[Dict[str, [*In*](#dagster.In)]]) – Information about the inputs to the op. Information provided here will be combined
+ - out (Optional[Union[[*Out*](#dagster.Out), Dict[str, [*Out*](#dagster.Out)]]]) – Information about the op outputs. Information provided here will be combined with
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The schema for the config. If set, Dagster will check
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by this op.
+ - tags (Optional[Dict[str, Any]]) – Arbitrary metadata for the op. Frameworks may
+ - code_version (Optional[str]) – (Experimental) Version of the logic encapsulated by the op. If set,
+ - retry_policy (Optional[[*RetryPolicy*](#dagster.RetryPolicy)]) – The retry policy for this op.
+
+
+ Examples:
+
+ ```python
+ @op
+ def hello_world():
+ print('hello')
+
+ @op
+ def echo(msg: str) -> str:
+ return msg
+
+ @op(
+ ins={'msg': In(str)},
+ out=Out(str)
+ )
+ def echo_2(msg): # same as above
+ return msg
+
+ @op(
+ out={'word': Out(), 'num': Out()}
+ )
+ def multi_out() -> Tuple[str, int]:
+ return 'cool', 4
+ ```
+
+
+
+
+
+
class dagster.OpDefinition
+
+
+ Defines an op, the functional unit of user-defined computation.
+
+ End users should prefer the [`@op`](#dagster.op) decorator. OpDefinition is generally intended to be
+ used by framework authors or for programatically generated ops.
+
+ Parameters:
+ - name (str) – Name of the op. Must be unique within any [`GraphDefinition`](graphs.mdx#dagster.GraphDefinition) or
+ - input_defs (List[InputDefinition]) – Inputs of the op.
+ - compute_fn (Callable) –
+
+ The core of the op, the function that performs the actual
+ computation. The signature of this function is determined by `input_defs`, and
+ optionally, an injected first argument, `context`, a collection of information
+ provided by the system.
+
+ - output_defs (List[OutputDefinition]) – Outputs of the op.
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The schema for the config. If set, Dagster will check
+ - description (Optional[str]) – Human-readable description of the op.
+ - tags (Optional[Dict[str, Any]]) – Arbitrary metadata for the op. Frameworks may
+ - required_resource_keys (Optional[Set[str]]) – Set of resources handles required by this op.
+ - code_version (Optional[str]) – (Experimental) Version of the code encapsulated by the op. If set,
+ - retry_policy (Optional[[*RetryPolicy*](#dagster.RetryPolicy)]) – The retry policy for this op.
+ - pool (Optional[str]) – A string that identifies the pool that governs this op’s execution.
+
+
+ Examples:
+
+ ```python
+ def _add_one(_context, inputs):
+ yield Output(inputs["num"] + 1)
+
+ OpDefinition(
+ name="add_one",
+ ins={"num": In(int)},
+ outs={"result": Out(int)},
+ compute_fn=_add_one,
+ )
+ ```
+
+
alias
+
+ Creates a copy of this op with the given name.
+
+
+
+
+
tag
+
+ Creates a copy of this op with the given tags.
+
+
+
+
+
with_hooks
+
+ Creates a copy of this op with the given hook definitions.
+
+
+
+
+
with_retry_policy
+
+ Creates a copy of this op with the given retry policy.
+
+
+
+
+
property config_schema
+
+
+ The config schema for this op.
+
+ Type: IDefinitionConfigSchema
+
+
+
+
+
+
property ins
+
+
+ A mapping from input name to the In object that represents that input.
+
+ Type: Mapping[str, [In](#dagster.In)]
+
+
+
+
+
+
property name
+
+
+ The name of this op.
+
+ Type: str
+
+
+
+
+
+
property outs
+
+
+ A mapping from output name to the Out object that represents that output.
+
+ Type: Mapping[str, [Out](#dagster.Out)]
+
+
+
+
+
+
property required_resource_keys
+
+
+ A set of keys for resources that must be provided to this OpDefinition.
+
+ Type: AbstractSet[str]
+
+
+
+
+
+
property retry_policy
+
+
+ The RetryPolicy for this op.
+
+ Type: Optional[[RetryPolicy](#dagster.RetryPolicy)]
+
+
+
+
+
+
property tags
+
+
+ The tags for this op.
+
+ Type: Mapping[str, str]
+
+
+
+
+
+
property version
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `code_version` instead..
+
+ :::
+
+ Version of the code encapsulated by the op. If set, this is used as a
+ default code version for all outputs.
+
+ Type: str
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Ins & outs
+
+
+
class dagster.In
+
+
+ Defines an argument to an op’s compute function.
+
+ Inputs may flow from previous op’s outputs, or be stubbed using config. They may optionally
+ be typed using the Dagster type system.
+
+ Parameters:
+ - dagster_type (Optional[Union[Type, [*DagsterType*](types.mdx#dagster.DagsterType)]]]) – The type of this input. Should only be set if the correct type can not
+ - description (Optional[str]) – Human-readable description of the input.
+ - default_value (Optional[Any]) – The default value to use if no input is provided.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – A dict of metadata for the input.
+ - asset_key (Optional[Union[[*AssetKey*](assets.mdx#dagster.AssetKey), InputContext -> AssetKey]]) – (Experimental) An AssetKey
+ - asset_partitions (Optional[Union[Set[str], InputContext -> Set[str]]]) – (Experimental) A
+ - input_manager_key (Optional[str]) – (Experimental) The resource key for the
+
+
+
+
+
+
+
+
class dagster.Out
+
+
+ Defines an output from an op’s compute function.
+
+ Ops can have multiple outputs, in which case outputs cannot be anonymous.
+
+ Many ops have only one output, in which case the user can provide a single output definition
+ that will be given the default name, “result”.
+
+ Outs may be typed using the Dagster type system.
+
+ Parameters:
+ - dagster_type (Optional[Union[Type, [*DagsterType*](types.mdx#dagster.DagsterType)]]]) – The type of this output. Should only be set if the correct type can not
+ - description (Optional[str]) – Human-readable description of the output.
+ - is_required (bool) – Whether the presence of this field is required. (default: True)
+ - io_manager_key (Optional[str]) – The resource key of the output manager used for this output.
+ - metadata (Optional[Dict[str, Any]]) – A dict of the metadata for the output.
+ - code_version (Optional[str]) – (Experimental) Version of the code that generates this output. In
+
+
+
+
+
+
+
+
+
+
+
+
+## Execution
+
+
+
class dagster.RetryPolicy
+
+
+ A declarative policy for when to request retries when an exception occurs during op execution.
+
+ Parameters:
+ - max_retries (int) – The maximum number of retries to attempt. Defaults to 1.
+ - delay (Optional[Union[int,float]]) – The time in seconds to wait between the retry being requested and the next attempt
+ - backoff (Optional[[*Backoff*](#dagster.Backoff)]) – A modifier for delay as a function of retry attempt number.
+ - jitter (Optional[[*Jitter*](#dagster.Jitter)]) – A randomizing modifier for delay, applied after backoff calculation.
+
+
+
+
+
+
+
+
class dagster.Backoff
+
+
+ A modifier for delay as a function of attempt number.
+
+ LINEAR: attempt_num * delay
+ EXPONENTIAL: ((2 ^ attempt_num) - 1) * delay
+
+
+
+
+
+
+
class dagster.Jitter
+
+
+ A randomizing modifier for delay, applied after backoff calculation.
+
+ FULL: between 0 and the calculated delay based on backoff: random() * backoff_delay
+ PLUS_MINUS: +/- the delay: backoff_delay + ((2 * (random() * delay)) - delay)
+
+
+
+
+
+
+
+
+
+
+
+
+## Events
+
+The objects that can be yielded by the body of ops’ compute functions to communicate with the
+Dagster framework.
+
+(Note that [`Failure`](#dagster.Failure) and [`RetryRequested`](#dagster.RetryRequested) are intended to be raised from ops rather than yielded.)
+
+
+
+### Event types
+
+
+
class dagster.Output
+
+
+ Event corresponding to one of an op’s outputs.
+
+ Op compute functions must explicitly yield events of this type when they have more than
+ one output, or when they also yield events of other types, or when defining a op using the
+ [`OpDefinition`](#dagster.OpDefinition) API directly.
+
+ Outputs are values produced by ops that will be consumed by downstream ops in a job.
+ They are type-checked at op boundaries when their corresponding [`Out`](#dagster.Out)
+ or the downstream [`In`](#dagster.In) is typed.
+
+ Parameters:
+ - value (Any) – The value returned by the compute function.
+ - output_name (str) – Name of the corresponding Out. (default: “result”)
+ - metadata (Optional[Dict[str, Union[str, float, int, [*MetadataValue*](metadata.mdx#dagster.MetadataValue)]]]) – Arbitrary metadata about the output. Keys are displayed string labels, and values are
+ - data_version (Optional[DataVersion]) – experimental
+ - tags (Optional[Mapping[str, str]]) – (Experimental) Tags that will be attached to the asset
+
+
+
+
property data_version
+
+
+ A data version that was manually set on the Output.
+
+ Type: Optional[DataVersion]
+
+
+
+
+
+
property output_name
+
+
+ Name of the corresponding [`Out`](#dagster.Out).
+
+ Type: str
+
+
+
+
+
+
property value
+
+
+ The value returned by the compute function.
+
+ Type: Any
+
+
+
+
+
+
+
+
+
+
class dagster.AssetMaterialization
+
+
+ Event indicating that an op has materialized an asset.
+
+ Op compute functions may yield events of this type whenever they wish to indicate to the
+ Dagster framework (and the end user) that they have produced a materialized value as a
+ side effect of computation. Unlike outputs, asset materializations can not be passed to other
+ ops, and their persistence is controlled by op logic, rather than by the Dagster
+ framework.
+
+ Op authors should use these events to organize metadata about the side effects of their
+ computations, enabling tooling like the Assets dashboard in the Dagster UI.
+
+ Parameters:
+ - asset_key (Union[str, List[str], [*AssetKey*](assets.mdx#dagster.AssetKey)]) – A key to identify the materialized asset across
+ - description (Optional[str]) – A longer human-readable description of the materialized value.
+ - partition (Optional[str]) – The name of the partition
+ - tags (Optional[Mapping[str, str]]) – A mapping containing tags for the materialization.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the asset. Keys are displayed string labels, and values are
+
+
+
+
static file
+
+
+ Static constructor for standard materializations corresponding to files on disk.
+
+ Parameters:
+ - path (str) – The path to the file.
+ - description (Optional[str]) – A human-readable description of the materialization.
+
+
+
+
+
+
+
+
+
+
+
+
class dagster.ExpectationResult
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ If using assets, use AssetCheckResult and @asset_check instead..
+
+ :::
+
+ Event corresponding to a data quality test.
+
+ Op compute functions may yield events of this type whenever they wish to indicate to the
+ Dagster framework (and the end user) that a data quality test has produced a (positive or
+ negative) result.
+
+ Parameters:
+ - success (bool) – Whether the expectation passed or not.
+ - label (Optional[str]) – Short display name for expectation. Defaults to “result”.
+ - description (Optional[str]) – A longer human-readable description of the expectation.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
+
+
+
+
+
+
+
+
class dagster.TypeCheck
+
+
+ Event corresponding to a successful typecheck.
+
+ Events of this type should be returned by user-defined type checks when they need to encapsulate
+ additional metadata about a type check’s success or failure. (i.e., when using
+ `as_dagster_type()`, `@usable_as_dagster_type`, or the underlying
+ [`PythonObjectDagsterType()`](types.mdx#dagster.PythonObjectDagsterType) API.)
+
+ Op compute functions should generally avoid yielding events of this type to avoid confusion.
+
+ Parameters:
+ - success (bool) – `True` if the type check succeeded, `False` otherwise.
+ - description (Optional[str]) – A human-readable description of the type check.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
+
+
+
+
+
+
+
+
class dagster.Failure
+
+
+ Event indicating op failure.
+
+ Raise events of this type from within op compute functions or custom type checks in order to
+ indicate an unrecoverable failure in user code to the Dagster machinery and return
+ structured metadata about the failure.
+
+ Parameters:
+ - description (Optional[str]) – A human-readable description of the failure.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
+ - allow_retries (Optional[bool]) – Whether this Failure should respect the retry policy or bypass it and immediately fail.
+
+
+
+
+
+
+
+
class dagster.RetryRequested
+
+
+ An exception to raise from an op to indicate that it should be retried.
+
+ Parameters:
+ - max_retries (Optional[int]) – The max number of retries this step should attempt before failing
+ - seconds_to_wait (Optional[Union[float,int]]) – Seconds to wait before restarting the step after putting the step in
+
+
+ Example:
+
+ ```python
+ @op
+ def flakes():
+ try:
+ flakey_operation()
+ except Exception as e:
+ raise RetryRequested(max_retries=3) from e
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/partitions.mdx b/docs/docs-beta/docs/api/python-api/partitions.mdx
new file mode 100644
index 0000000000000..f502f38b85b56
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/partitions.mdx
@@ -0,0 +1,1659 @@
+---
+title: 'partitions definitions'
+title_meta: 'partitions definitions API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'partitions definitions Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Partitions Definitions
+
+
+
class dagster.PartitionsDefinition
+
+
+ Defines a set of partitions, which can be attached to a software-defined asset or job.
+
+ Abstract class with implementations for different kinds of partitions.
+
+
+
abstract get_partition_keys
+
+
+ Returns a list of strings representing the partition keys of the PartitionsDefinition.
+
+ Parameters:
+ - current_time (Optional[datetime]) – A datetime object representing the current time, only
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: Sequence[str]
+
+
+
+
+
+
+
+
+
+
class dagster.HourlyPartitionsDefinition
+
+
+ A set of hourly partitions.
+
+ The first partition in the set will start on the start_date at midnight. The last partition
+ in the set will end before the current time, unless the end_offset argument is set to a
+ positive number. If minute_offset is provided, the start and end times of each partition
+ will be minute_offset past the hour.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions. Can
+ - end_date (Union[datetime.datetime, str, None]) – The last date(excluding) in the set of partitions.
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d. Note that if a non-UTC
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+
+
+ ```python
+ HourlyPartitionsDefinition(start_date=datetime(2022, 03, 12))
+ # creates partitions (2022-03-12-00:00, 2022-03-12-01:00), (2022-03-12-01:00, 2022-03-12-02:00), ...
+
+ HourlyPartitionsDefinition(start_date=datetime(2022, 03, 12), minute_offset=15)
+ # creates partitions (2022-03-12-00:15, 2022-03-12-01:15), (2022-03-12-01:15, 2022-03-12-02:15), ...
+ ```
+
+
get_cron_schedule
+
+
+ The schedule executes at the cadence specified by the partitioning, but may overwrite
+ the minute/hour/day offset of the partitioning.
+
+ This is useful e.g. if you have partitions that span midnight to midnight but you want to
+ schedule a job that runs at 2 am.
+
+
+
+
+
+
+
property day_offset
+
+
+ For a weekly or monthly partitions definition, returns the day to “split” partitions
+ by. Each partition will start on this day, and end before this day in the following
+ week/month. Returns 0 if the day_offset parameter is unset in the
+ WeeklyPartitionsDefinition, MonthlyPartitionsDefinition, or the provided cron schedule.
+
+ For weekly partitions, returns a value between 0 (representing Sunday) and 6 (representing
+ Saturday). Providing a value of 1 means that a partition will exist weekly from Monday to
+ the following Sunday.
+
+ For monthly partitions, returns a value between 0 (the first day of the month) and 31 (the
+ last possible day of the month).
+
+ Type: int
+
+
+
+
+
+
property end
+
+
+
+
+
+
+
property hour_offset
+
+
+ Number of hours past 00:00 to “split” partitions. Defaults to 0.
+
+ For example, returns 1 if each partition starts at 01:00.
+
+ Type: int
+
+
+
+
+
+
property minute_offset
+
+
+ Number of minutes past the hour to “split” partitions. Defaults to 0.
+
+ For example, returns 15 if each partition starts at 15 minutes past the hour.
+
+ Type: int
+
+
+
+
+
+
property schedule_type
+
+
+ An enum representing the partition cadence (hourly, daily,
+ weekly, or monthly).
+
+ Type: Optional[ScheduleType]
+
+
+
+
+
+
property start
+
+
+
+
+
+
+
+
+
+
+
class dagster.DailyPartitionsDefinition
+
+
+ A set of daily partitions.
+
+ The first partition in the set will start at the start_date at midnight. The last partition
+ in the set will end before the current time, unless the end_offset argument is set to a
+ positive number. If minute_offset and/or hour_offset are used, the start and end times of
+ each partition will be hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions. Can
+ - end_date (Union[datetime.datetime, str, None]) – The last date(excluding) in the set of partitions.
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+
+
+ ```python
+ DailyPartitionsDefinition(start_date="2022-03-12")
+ # creates partitions (2022-03-12-00:00, 2022-03-13-00:00), (2022-03-13-00:00, 2022-03-14-00:00), ...
+
+ DailyPartitionsDefinition(start_date="2022-03-12", minute_offset=15, hour_offset=16)
+ # creates partitions (2022-03-12-16:15, 2022-03-13-16:15), (2022-03-13-16:15, 2022-03-14-16:15), ...
+ ```
+
+
get_cron_schedule
+
+
+ The schedule executes at the cadence specified by the partitioning, but may overwrite
+ the minute/hour/day offset of the partitioning.
+
+ This is useful e.g. if you have partitions that span midnight to midnight but you want to
+ schedule a job that runs at 2 am.
+
+
+
+
+
+
+
property day_offset
+
+
+ For a weekly or monthly partitions definition, returns the day to “split” partitions
+ by. Each partition will start on this day, and end before this day in the following
+ week/month. Returns 0 if the day_offset parameter is unset in the
+ WeeklyPartitionsDefinition, MonthlyPartitionsDefinition, or the provided cron schedule.
+
+ For weekly partitions, returns a value between 0 (representing Sunday) and 6 (representing
+ Saturday). Providing a value of 1 means that a partition will exist weekly from Monday to
+ the following Sunday.
+
+ For monthly partitions, returns a value between 0 (the first day of the month) and 31 (the
+ last possible day of the month).
+
+ Type: int
+
+
+
+
+
+
property end
+
+
+
+
+
+
+
property hour_offset
+
+
+ Number of hours past 00:00 to “split” partitions. Defaults to 0.
+
+ For example, returns 1 if each partition starts at 01:00.
+
+ Type: int
+
+
+
+
+
+
property minute_offset
+
+
+ Number of minutes past the hour to “split” partitions. Defaults to 0.
+
+ For example, returns 15 if each partition starts at 15 minutes past the hour.
+
+ Type: int
+
+
+
+
+
+
property schedule_type
+
+
+ An enum representing the partition cadence (hourly, daily,
+ weekly, or monthly).
+
+ Type: Optional[ScheduleType]
+
+
+
+
+
+
property start
+
+
+
+
+
+
+
+
+
+
+
class dagster.WeeklyPartitionsDefinition
+
+
+ Defines a set of weekly partitions.
+
+ The first partition in the set will start at the start_date. The last partition in the set will
+ end before the current time, unless the end_offset argument is set to a positive number. If
+ day_offset is provided, the start and end date of each partition will be day of the week
+ corresponding to day_offset (0 indexed with Sunday as the start of the week). If
+ minute_offset and/or hour_offset are used, the start and end times of each partition will be
+ hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions will
+ - end_date (Union[datetime.datetime, str, None]) – The last date(excluding) in the set of partitions.
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - day_offset (int) – Day of the week to “split” the partition. Defaults to 0 (Sunday).
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+
+
+ ```python
+ WeeklyPartitionsDefinition(start_date="2022-03-12")
+ # creates partitions (2022-03-13-00:00, 2022-03-20-00:00), (2022-03-20-00:00, 2022-03-27-00:00), ...
+
+ WeeklyPartitionsDefinition(start_date="2022-03-12", minute_offset=15, hour_offset=3, day_offset=6)
+ # creates partitions (2022-03-12-03:15, 2022-03-19-03:15), (2022-03-19-03:15, 2022-03-26-03:15), ...
+ ```
+
+
get_cron_schedule
+
+
+ The schedule executes at the cadence specified by the partitioning, but may overwrite
+ the minute/hour/day offset of the partitioning.
+
+ This is useful e.g. if you have partitions that span midnight to midnight but you want to
+ schedule a job that runs at 2 am.
+
+
+
+
+
+
+
property day_offset
+
+
+ For a weekly or monthly partitions definition, returns the day to “split” partitions
+ by. Each partition will start on this day, and end before this day in the following
+ week/month. Returns 0 if the day_offset parameter is unset in the
+ WeeklyPartitionsDefinition, MonthlyPartitionsDefinition, or the provided cron schedule.
+
+ For weekly partitions, returns a value between 0 (representing Sunday) and 6 (representing
+ Saturday). Providing a value of 1 means that a partition will exist weekly from Monday to
+ the following Sunday.
+
+ For monthly partitions, returns a value between 0 (the first day of the month) and 31 (the
+ last possible day of the month).
+
+ Type: int
+
+
+
+
+
+
property end
+
+
+
+
+
+
+
property hour_offset
+
+
+ Number of hours past 00:00 to “split” partitions. Defaults to 0.
+
+ For example, returns 1 if each partition starts at 01:00.
+
+ Type: int
+
+
+
+
+
+
property minute_offset
+
+
+ Number of minutes past the hour to “split” partitions. Defaults to 0.
+
+ For example, returns 15 if each partition starts at 15 minutes past the hour.
+
+ Type: int
+
+
+
+
+
+
property schedule_type
+
+
+ An enum representing the partition cadence (hourly, daily,
+ weekly, or monthly).
+
+ Type: Optional[ScheduleType]
+
+
+
+
+
+
property start
+
+
+
+
+
+
+
+
+
+
+
class dagster.MonthlyPartitionsDefinition
+
+
+ A set of monthly partitions.
+
+ The first partition in the set will start at the soonest first of the month after start_date
+ at midnight. The last partition in the set will end before the current time, unless the
+ end_offset argument is set to a positive number. If day_offset is provided, the start and
+ end date of each partition will be day_offset. If minute_offset and/or hour_offset are used,
+ the start and end times of each partition will be hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions will be
+ - end_date (Union[datetime.datetime, str, None]) – The last date(excluding) in the set of partitions.
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - day_offset (int) – Day of the month to “split” the partition. Defaults to 1.
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+
+
+ ```python
+ MonthlyPartitionsDefinition(start_date="2022-03-12")
+ # creates partitions (2022-04-01-00:00, 2022-05-01-00:00), (2022-05-01-00:00, 2022-06-01-00:00), ...
+
+ MonthlyPartitionsDefinition(start_date="2022-03-12", minute_offset=15, hour_offset=3, day_offset=5)
+ # creates partitions (2022-04-05-03:15, 2022-05-05-03:15), (2022-05-05-03:15, 2022-06-05-03:15), ...
+ ```
+
+
get_cron_schedule
+
+
+ The schedule executes at the cadence specified by the partitioning, but may overwrite
+ the minute/hour/day offset of the partitioning.
+
+ This is useful e.g. if you have partitions that span midnight to midnight but you want to
+ schedule a job that runs at 2 am.
+
+
+
+
+
+
+
property day_offset
+
+
+ For a weekly or monthly partitions definition, returns the day to “split” partitions
+ by. Each partition will start on this day, and end before this day in the following
+ week/month. Returns 0 if the day_offset parameter is unset in the
+ WeeklyPartitionsDefinition, MonthlyPartitionsDefinition, or the provided cron schedule.
+
+ For weekly partitions, returns a value between 0 (representing Sunday) and 6 (representing
+ Saturday). Providing a value of 1 means that a partition will exist weekly from Monday to
+ the following Sunday.
+
+ For monthly partitions, returns a value between 0 (the first day of the month) and 31 (the
+ last possible day of the month).
+
+ Type: int
+
+
+
+
+
+
property end
+
+
+
+
+
+
+
property hour_offset
+
+
+ Number of hours past 00:00 to “split” partitions. Defaults to 0.
+
+ For example, returns 1 if each partition starts at 01:00.
+
+ Type: int
+
+
+
+
+
+
property minute_offset
+
+
+ Number of minutes past the hour to “split” partitions. Defaults to 0.
+
+ For example, returns 15 if each partition starts at 15 minutes past the hour.
+
+ Type: int
+
+
+
+
+
+
property schedule_type
+
+
+ An enum representing the partition cadence (hourly, daily,
+ weekly, or monthly).
+
+ Type: Optional[ScheduleType]
+
+
+
+
+
+
property start
+
+
+
+
+
+
+
+
+
+
+
class dagster.TimeWindowPartitionsDefinition
+
+
+ A set of partitions where each partition corresponds to a time window.
+
+ The provided cron_schedule determines the bounds of the time windows. E.g. a cron_schedule of
+ “0 0 \* \* \*” will result in daily partitions that start at midnight and end at midnight of the
+ following day.
+
+ The string partition_key associated with each partition corresponds to the start of the
+ partition’s time window.
+
+ The first partition in the set will start on at the first cron_schedule tick that is equal to
+ or after the given start datetime. The last partition in the set will end before the current
+ time, unless the end_offset argument is set to a positive number.
+
+ We recommended limiting partition counts for each asset to 25,000 partitions or fewer.
+
+ Parameters:
+ - cron_schedule (str) – Determines the bounds of the time windows.
+ - start (datetime) – The first partition in the set will start on at the first cron_schedule
+ - timezone (Optional[str]) – The timezone in which each time should exist.
+ - end (datetime) – The last partition (excluding) in the set.
+ - fmt (str) – The date format to use for partition_keys. Note that if a non-UTC timezone is
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+
+
+
+
get_cron_schedule
+
+
+ The schedule executes at the cadence specified by the partitioning, but may overwrite
+ the minute/hour/day offset of the partitioning.
+
+ This is useful e.g. if you have partitions that span midnight to midnight but you want to
+ schedule a job that runs at 2 am.
+
+
+
+
+
+
+
property day_offset
+
+
+ For a weekly or monthly partitions definition, returns the day to “split” partitions
+ by. Each partition will start on this day, and end before this day in the following
+ week/month. Returns 0 if the day_offset parameter is unset in the
+ WeeklyPartitionsDefinition, MonthlyPartitionsDefinition, or the provided cron schedule.
+
+ For weekly partitions, returns a value between 0 (representing Sunday) and 6 (representing
+ Saturday). Providing a value of 1 means that a partition will exist weekly from Monday to
+ the following Sunday.
+
+ For monthly partitions, returns a value between 0 (the first day of the month) and 31 (the
+ last possible day of the month).
+
+ Type: int
+
+
+
+
+
+
property end
+
+
+
+
+
+
+
property hour_offset
+
+
+ Number of hours past 00:00 to “split” partitions. Defaults to 0.
+
+ For example, returns 1 if each partition starts at 01:00.
+
+ Type: int
+
+
+
+
+
+
property minute_offset
+
+
+ Number of minutes past the hour to “split” partitions. Defaults to 0.
+
+ For example, returns 15 if each partition starts at 15 minutes past the hour.
+
+ Type: int
+
+
+
+
+
+
property schedule_type
+
+
+ An enum representing the partition cadence (hourly, daily,
+ weekly, or monthly).
+
+ Type: Optional[ScheduleType]
+
+
+
+
+
+
property start
+
+
+
+
+
+
+
+
+
+
+
class dagster.TimeWindow
+
+
+ An interval that is closed at the start and open at the end.
+
+
+
start
+
+
+ A datetime that marks the start of the window.
+
+ Type: datetime
+
+
+
+
+
+
end
+
+
+ A datetime that marks the end of the window.
+
+ Type: datetime
+
+
+
+
+
+
+
+
+
+
class dagster.StaticPartitionsDefinition
+
+
+ A statically-defined set of partitions.
+
+ We recommended limiting partition counts for each asset to 25,000 partitions or fewer.
+
+ Example:
+
+ ```python
+ from dagster import StaticPartitionsDefinition, asset
+
+ oceans_partitions_def = StaticPartitionsDefinition(
+ ["arctic", "atlantic", "indian", "pacific", "southern"]
+ )
+
+ @asset(partitions_def=oceans_partitions_defs)
+ def ml_model_for_each_ocean():
+ ...
+ ```
+
+
get_partition_keys
+
+
+ Returns a list of strings representing the partition keys of the PartitionsDefinition.
+
+ Parameters:
+ - current_time (Optional[datetime]) – A datetime object representing the current time, only
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: Sequence[str]
+
+
+
+
+
+
+
+
+
+
class dagster.MultiPartitionsDefinition
+
+
+ Takes the cross-product of partitions from two partitions definitions.
+
+ For example, with a static partitions definition where the partitions are [“a”, “b”, “c”]
+ and a daily partitions definition, this partitions definition will have the following
+ partitions:
+
+ 2020-01-01|a
+ 2020-01-01|b
+ 2020-01-01|c
+ 2020-01-02|a
+ 2020-01-02|b
+ …
+
+ We recommended limiting partition counts for each asset to 25,000 partitions or fewer.
+
+ Parameters: partitions_defs (Mapping[str, [*PartitionsDefinition*](#dagster.PartitionsDefinition)]) – A mapping of dimension name to partitions definition. The total set of partitions will
+ be the cross-product of the partitions from each PartitionsDefinition.
+
+
partitions_defs
+
+
+ A sequence of PartitionDimensionDefinition objects, each of which contains a dimension
+ name and a PartitionsDefinition. The total set of partitions will be the cross-product
+ of the partitions from each PartitionsDefinition. This sequence is ordered by
+ dimension name, to ensure consistent ordering of the partitions.
+
+ Type: Sequence[PartitionDimensionDefinition]
+
+
+
+
+
+
get_partition_keys
+
+
+ Returns a list of MultiPartitionKeys representing the partition keys of the
+ PartitionsDefinition.
+
+ Parameters:
+ - current_time (Optional[datetime]) – A datetime object representing the current time, only
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: Sequence[MultiPartitionKey]
+
+
+
+
+
+
+
+
+
+
class dagster.MultiPartitionKey
+
+
+ A multi-dimensional partition key stores the partition key for each dimension.
+ Subclasses the string class to keep partition key type as a string.
+
+ Contains additional methods to access the partition key for each dimension.
+ Creates a string representation of the partition key for each dimension, separated by a pipe (|).
+ Orders the dimensions by name, to ensure consistent string representation.
+
+
+
+
+
+
+
class dagster.DynamicPartitionsDefinition
+
+
+ A partitions definition whose partition keys can be dynamically added and removed.
+
+ This is useful for cases where the set of partitions is not known at definition time,
+ but is instead determined at runtime.
+
+ Partitions can be added and removed using instance.add_dynamic_partitions and
+ instance.delete_dynamic_partition methods.
+
+ We recommended limiting partition counts for each asset to 25,000 partitions or fewer.
+
+ Parameters:
+ - name (Optional[str]) – The name of the partitions definition.
+ - partition_fn (Optional[Callable[[Optional[datetime]], Union[Sequence[Partition], Sequence[str]]]]) – deprecated
+
+
+ Examples:
+
+ ```python
+ fruits = DynamicPartitionsDefinition(name="fruits")
+
+ @sensor(job=my_job)
+ def my_sensor(context):
+ return SensorResult(
+ run_requests=[RunRequest(partition_key="apple")],
+ dynamic_partitions_requests=[fruits.build_add_request(["apple"])]
+ )
+ ```
+
+
get_partition_keys
+
+
+ Returns a list of strings representing the partition keys of the
+ PartitionsDefinition.
+
+ Parameters:
+ - current_time (Optional[datetime]) – A datetime object representing the current time, only
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: Sequence[str]
+
+
+
+
+
+
+
+
+
+
class dagster.PartitionKeyRange
+
+
+ Defines a range of partitions.
+
+
+
start
+
+
+ The starting partition key in the range (inclusive).
+
+ Type: str
+
+
+
+
+
+
end
+
+
+ The ending partition key in the range (inclusive).
+
+ Type: str
+
+
+
+ Creates a schedule from a job that targets
+ time window-partitioned or statically-partitioned assets. The job can also be
+ multi-partitioned, as long as one of the partition dimensions is time-partitioned.
+
+ The schedule executes at the cadence specified by the time partitioning of the job or assets.
+
+ Example:
+ ```python
+ ######################################
+ # Job that targets partitioned assets
+ ######################################
+
+ from dagster import (
+ DailyPartitionsDefinition,
+ asset,
+ build_schedule_from_partitioned_job,
+ define_asset_job,
+ Definitions,
+ )
+
+ @asset(partitions_def=DailyPartitionsDefinition(start_date="2020-01-01"))
+ def asset1():
+ ...
+
+ asset1_job = define_asset_job("asset1_job", selection=[asset1])
+
+ # The created schedule will fire daily
+ asset1_job_schedule = build_schedule_from_partitioned_job(asset1_job)
+
+ defs = Definitions(assets=[asset1], schedules=[asset1_job_schedule])
+
+ ################
+ # Non-asset job
+ ################
+
+ from dagster import DailyPartitionsDefinition, build_schedule_from_partitioned_job, jog
+
+
+ @job(partitions_def=DailyPartitionsDefinition(start_date="2020-01-01"))
+ def do_stuff_partitioned():
+ ...
+
+ # The created schedule will fire daily
+ do_stuff_partitioned_schedule = build_schedule_from_partitioned_job(
+ do_stuff_partitioned,
+ )
+
+ defs = Definitions(schedules=[do_stuff_partitioned_schedule])
+ ```
+
+
+
+
+
+
+
+
+
+
+# Partition Mapping
+
+
+
class dagster.PartitionMapping
+
+
+ Defines a correspondence between the partitions in an asset and the partitions in an asset
+ that it depends on.
+
+ Overriding PartitionMapping outside of Dagster is not supported. The abstract methods of this
+ class may change at any time.
+
+
+
abstract get_downstream_partitions_for_partitions
+
+
+ Returns the subset of partition keys in the downstream asset that use the data in the given
+ partition key subset of the upstream asset.
+
+ Parameters:
+ - upstream_partitions_subset (Union[[*PartitionKeyRange*](#dagster.PartitionKeyRange), PartitionsSubset]) – The
+ - downstream_partitions_def ([*PartitionsDefinition*](#dagster.PartitionsDefinition)) – The partitions definition for the
+
+
+
+
+
+ Returns a UpstreamPartitionsResult object containing the partition keys the downstream
+ partitions subset was mapped to in the upstream partitions definition.
+
+ Valid upstream partitions will be included in UpstreamPartitionsResult.partitions_subset.
+ Invalid upstream partitions will be included in UpstreamPartitionsResult.required_but_nonexistent_subset.
+
+ For example, if an upstream asset is time-partitioned and starts in June 2023, and the
+ downstream asset is time-partitioned and starts in May 2023, this function would return a
+ UpstreamPartitionsResult(PartitionsSubset(“2023-06-01”), required_but_nonexistent_subset=PartitionsSubset(“2023-05-01”))
+ when downstream_partitions_subset contains 2023-05-01 and 2023-06-01.
+
+
+
+
+
+
+
+
+
+
+
class dagster.TimeWindowPartitionMapping
+
+
+ The default mapping between two TimeWindowPartitionsDefinitions.
+
+ A partition in the downstream partitions definition is mapped to all partitions in the upstream
+ asset whose time windows overlap it.
+
+ This means that, if the upstream and downstream partitions definitions share the same time
+ period, then this mapping is essentially the identity partition mapping - plus conversion of
+ datetime formats.
+
+ If the upstream time period is coarser than the downstream time period, then each partition in
+ the downstream asset will map to a single (larger) upstream partition. E.g. if the downstream is
+ hourly and the upstream is daily, then each hourly partition in the downstream will map to the
+ daily partition in the upstream that contains that hour.
+
+ If the upstream time period is finer than the downstream time period, then each partition in the
+ downstream asset will map to multiple upstream partitions. E.g. if the downstream is daily and
+ the upstream is hourly, then each daily partition in the downstream asset will map to the 24
+ hourly partitions in the upstream that occur on that day.
+
+
+
start_offset
+
+
+ If not 0, then the starts of the upstream windows are shifted by this
+ offset relative to the starts of the downstream windows. For example, if start_offset=-1
+ and end_offset=0, then the downstream partition “2022-07-04” would map to the upstream
+ partitions “2022-07-03” and “2022-07-04”. If the upstream and downstream
+ PartitionsDefinitions are different, then the offset is in the units of the downstream.
+ Defaults to 0.
+
+ Type: int
+
+
+
+
+
+
end_offset
+
+
+ If not 0, then the ends of the upstream windows are shifted by this
+ offset relative to the ends of the downstream windows. For example, if start_offset=0
+ and end_offset=1, then the downstream partition “2022-07-04” would map to the upstream
+ partitions “2022-07-04” and “2022-07-05”. If the upstream and downstream
+ PartitionsDefinitions are different, then the offset is in the units of the downstream.
+ Defaults to 0.
+
+ Type: int
+
+
+
+
+
+
allow_nonexistent_upstream_partitions
+
+
+ Defaults to false. If true, does not
+ raise an error when mapped upstream partitions fall outside the start-end time window of the
+ partitions def. For example, if the upstream partitions def starts on “2023-01-01” but
+ the downstream starts on “2022-01-01”, setting this bool to true would return no
+ partition keys when get_upstream_partitions_for_partitions is called with “2022-06-01”.
+ When set to false, would raise an error.
+
+ Type: bool
+
+
+ Expects that the upstream and downstream assets are partitioned in the same way, and maps
+ partitions in the downstream asset to the same partition in the upstream asset.
+
+
+
+
+
class dagster.AllPartitionMapping
+
+
+ Maps every partition in the downstream asset to every partition in the upstream asset.
+
+ Commonly used in the case when the downstream asset is not partitioned, in which the entire
+ downstream asset depends on all partitions of the upstream asset.
+
+
+
+
+
+
+
class dagster.LastPartitionMapping
+
+
+ Maps all dependencies to the last partition in the upstream asset.
+
+ Commonly used in the case when the downstream asset is not partitioned, in which the entire
+ downstream asset depends on the last partition of the upstream asset.
+
+
+
+
+
+
+
class dagster.StaticPartitionMapping
+
+
+ Define an explicit correspondence between two StaticPartitionsDefinitions.
+
+ Parameters: downstream_partition_keys_by_upstream_partition_key (Dict[str, str | Collection[str]]) – The single or multi-valued correspondence from upstream keys to downstream keys.
+
+
+
+
+
+
class dagster.SpecificPartitionsPartitionMapping
+
+
+ Maps to a specific subset of partitions in the upstream asset.
+
+ Example:
+
+ ```python
+ from dagster import SpecificPartitionsPartitionMapping, StaticPartitionsDefinition, asset
+
+ @asset(partitions_def=StaticPartitionsDefinition(["a", "b", "c"]))
+ def upstream():
+ ...
+
+ @asset(
+ ins={
+ "upstream": AssetIn(partition_mapping=SpecificPartitionsPartitionMapping(["a"]))
+ }
+ )
+ def a_downstream(upstream):
+ ...
+ ```
+
+
+
+
+
+
class dagster.MultiToSingleDimensionPartitionMapping
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Defines a correspondence between an single-dimensional partitions definition
+ and a MultiPartitionsDefinition. The single-dimensional partitions definition must be
+ a dimension of the MultiPartitionsDefinition.
+
+ This class handles the case where the upstream asset is multipartitioned and the
+ downstream asset is single dimensional, and vice versa.
+
+ For a partition key X, this partition mapping assumes that any multi-partition key with
+ X in the selected dimension is a dependency.
+
+ Parameters: partition_dimension_name (Optional[str]) – The name of the partition dimension in the
+ MultiPartitionsDefinition that matches the single-dimension partitions definition.
+
+
+
+
+
+
class dagster.MultiPartitionMapping
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Defines a correspondence between two MultiPartitionsDefinitions.
+
+ Accepts a mapping of upstream dimension name to downstream DimensionPartitionMapping, representing
+ the explicit correspondence between the upstream and downstream MultiPartitions dimensions
+ and the partition mapping used to calculate the downstream partitions.
+
+ Examples:
+
+ ```python
+ weekly_abc = MultiPartitionsDefinition(
+ {
+ "abc": StaticPartitionsDefinition(["a", "b", "c"]),
+ "weekly": WeeklyPartitionsDefinition("2023-01-01"),
+ }
+ )
+ daily_123 = MultiPartitionsDefinition(
+ {
+ "123": StaticPartitionsDefinition(["1", "2", "3"]),
+ "daily": DailyPartitionsDefinition("2023-01-01"),
+ }
+ )
+
+ MultiPartitionMapping(
+ {
+ "abc": DimensionPartitionMapping(
+ dimension_name="123",
+ partition_mapping=StaticPartitionMapping({"a": "1", "b": "2", "c": "3"}),
+ ),
+ "weekly": DimensionPartitionMapping(
+ dimension_name="daily",
+ partition_mapping=TimeWindowPartitionMapping(),
+ )
+ }
+ )
+ ```
+ For upstream or downstream dimensions not explicitly defined in the mapping, Dagster will
+ assume an AllPartitionsMapping, meaning that all upstream partitions in those dimensions
+ will be mapped to all downstream partitions in those dimensions.
+
+ Examples:
+
+ ```python
+ weekly_abc = MultiPartitionsDefinition(
+ {
+ "abc": StaticPartitionsDefinition(["a", "b", "c"]),
+ "daily": DailyPartitionsDefinition("2023-01-01"),
+ }
+ )
+ daily_123 = MultiPartitionsDefinition(
+ {
+ "123": StaticPartitionsDefinition(["1", "2", "3"]),
+ "daily": DailyPartitionsDefinition("2023-01-01"),
+ }
+ )
+
+ MultiPartitionMapping(
+ {
+ "daily": DimensionPartitionMapping(
+ dimension_name="daily",
+ partition_mapping=IdentityPartitionMapping(),
+ )
+ }
+ )
+
+ # Will map `daily_123` partition key {"123": "1", "daily": "2023-01-01"} to the upstream:
+ # {"abc": "a", "daily": "2023-01-01"}
+ # {"abc": "b", "daily": "2023-01-01"}
+ # {"abc": "c", "daily": "2023-01-01"}
+ ```
+ Parameters: downstream_mappings_by_upstream_dimension (Mapping[str, DimensionPartitionMapping]) – A
+ mapping that defines an explicit correspondence between one dimension of the upstream
+ MultiPartitionsDefinition and one dimension of the downstream MultiPartitionsDefinition.
+ Maps a string representing upstream dimension name to downstream DimensionPartitionMapping,
+ containing the downstream dimension name and partition mapping.
+
+
+
+
+
+
+
+
+
+
+# Backfill Policy (Experimental)
+
+
+
class dagster.BackfillPolicy
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A BackfillPolicy specifies how Dagster should attempt to backfill a partitioned asset.
+
+ There are two main kinds of backfill policies: single-run and multi-run.
+
+ An asset with a single-run backfill policy will take a single run to backfill all of its
+ partitions at once.
+
+ An asset with a multi-run backfill policy will take multiple runs to backfill all of its
+ partitions. Each run will backfill a subset of the partitions. The number of partitions to
+ backfill in each run is controlled by the max_partitions_per_run parameter.
+
+ For example:
+
+ - If an asset has 100 partitions, and the max_partitions_per_run is set to 10, then it will
+ - If an asset has 100 partitions, and the max_partitions_per_run is set to 11, then it will
+
+
+ Warning:
+
+ Constructing an BackfillPolicy directly is not recommended as the API is subject to change.
+ BackfillPolicy.single_run() and BackfillPolicy.multi_run(max_partitions_per_run=x) are the
+ recommended APIs.
+
+
+
static multi_run
+
+
+ Creates a BackfillPolicy that executes the entire backfill in multiple runs.
+ Each run will backfill [max_partitions_per_run] number of partitions.
+
+ Parameters: max_partitions_per_run (Optional[int]) – The maximum number of partitions in each run of
+ the multiple runs. Defaults to 1.
+
+
+
+
+
+
static single_run
+
+ Creates a BackfillPolicy that executes the entire backfill in a single run.
+
+
+
+
+
+
+
+
+
+
+
+
+# Partitioned Config
+
+
+
class dagster.PartitionedConfig
+
+
+ Defines a way of configuring a job where the job can be run on one of a discrete set of
+ partitions, and each partition corresponds to run configuration for the job.
+
+ Setting PartitionedConfig as the config for a job allows you to launch backfills for that job
+ and view the run history across partitions.
+
+
+
get_partition_keys
+
+
+ Returns a list of partition keys, representing the full set of partitions that
+ config can be applied to.
+
+ Parameters: current_time (Optional[datetime]) – A datetime object representing the current time. Only
+ applicable to time-based partitions definitions.Returns: Sequence[str]
+
+
+
+
+
+
property partitions_def
+
+
+ The partitions definition associated with this PartitionedConfig.
+
+ Type: T_PartitionsDefinition
+
+
+
+
+
+
property run_config_for_partition_fn
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `run_config_for_partition_key_fn` instead..
+
+ :::
+
+ A function that accepts a partition
+ and returns a dictionary representing the config to attach to runs for that partition.
+ Deprecated as of 1.3.3.
+
+ Type: Optional[Callable[[Partition], Mapping[str, Any]]]
+
+
+
+
+
+
property run_config_for_partition_key_fn
+
+
+ A function that accepts a partition key
+ and returns a dictionary representing the config to attach to runs for that partition.
+
+ Type: Optional[Callable[[str], Union[[RunConfig](config.mdx#dagster.RunConfig), Mapping[str, Any]]]]
+
+
+
+
+
+
property tags_for_partition_fn
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `tags_for_partition_key_fn` instead..
+
+ :::
+
+ A function that
+ accepts a partition and returns a dictionary of tags to attach to runs for
+ that partition. Deprecated as of 1.3.3.
+
+ Type: Optional[Callable[[Partition], Mapping[str, str]]]
+
+
+
+
+
+
property tags_for_partition_key_fn
+
+
+ A function that
+ accepts a partition key and returns a dictionary of tags to attach to runs for
+ that partition.
+
+ Type: Optional[Callable[[str], Mapping[str, str]]]
+
+
+
+
+
+
+
+
+
+
dagster.static_partitioned_config
+
+
+ Creates a static partitioned config for a job.
+
+ The provided partition_keys is a static list of strings identifying the set of partitions. The
+ list of partitions is static, so while the run config returned by the decorated function may
+ change over time, the list of valid partition keys does not.
+
+ This has performance advantages over dynamic_partitioned_config in terms of loading different
+ partition views in the Dagster UI.
+
+ The decorated function takes in a partition key and returns a valid run config for a particular
+ target job.
+
+ Parameters:
+ - partition_keys (Sequence[str]) – A list of valid partition keys, which serve as the range of
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – deprecated
+ - tags_for_partition_key_fn (Optional[Callable[[str], Mapping[str, str]]]) – A function that
+
+
+ Returns: PartitionedConfig
+
+
+
+
+
+
dagster.dynamic_partitioned_config
+
+
+ Creates a dynamic partitioned config for a job.
+
+ The provided partition_fn returns a list of strings identifying the set of partitions, given
+ an optional datetime argument (representing the current time). The list of partitions returned
+ may change over time.
+
+ The decorated function takes in a partition key and returns a valid run config for a particular
+ target job.
+
+ Parameters:
+ - partition_fn (Callable[[datetime.datetime], Sequence[str]]) – A function that generates a
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – deprecated
+
+
+ Returns: PartitionedConfig
+
+
+
+
+
+
dagster.hourly_partitioned_config
+
+
+ Defines run config over a set of hourly partitions.
+
+ The decorated function should accept a start datetime and end datetime, which represent the date
+ partition the config should delineate.
+
+ The decorated function should return a run config dictionary.
+
+ The resulting object created by this decorator can be provided to the config argument of a Job.
+ The first partition in the set will start at the start_date at midnight. The last partition in
+ the set will end before the current time, unless the end_offset argument is set to a positive
+ number. If minute_offset is provided, the start and end times of each partition will be
+ minute_offset past the hour.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions. Can
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – A function that
+
+
+ ```python
+ @hourly_partitioned_config(start_date=datetime(2022, 03, 12))
+ # creates partitions (2022-03-12-00:00, 2022-03-12-01:00), (2022-03-12-01:00, 2022-03-12-02:00), ...
+
+ @hourly_partitioned_config(start_date=datetime(2022, 03, 12), minute_offset=15)
+ # creates partitions (2022-03-12-00:15, 2022-03-12-01:15), (2022-03-12-01:15, 2022-03-12-02:15), ...
+ ```
+
+
+
+
+
+
dagster.daily_partitioned_config
+
+
+ Defines run config over a set of daily partitions.
+
+ The decorated function should accept a start datetime and end datetime, which represent the bounds
+ of the date partition the config should delineate.
+
+ The decorated function should return a run config dictionary.
+
+ The resulting object created by this decorator can be provided to the config argument of a Job.
+ The first partition in the set will start at the start_date at midnight. The last partition in
+ the set will end before the current time, unless the end_offset argument is set to a positive
+ number. If minute_offset and/or hour_offset are used, the start and end times of each partition
+ will be hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions. Can
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – A function that
+
+
+ ```python
+ @daily_partitioned_config(start_date="2022-03-12")
+ # creates partitions (2022-03-12-00:00, 2022-03-13-00:00), (2022-03-13-00:00, 2022-03-14-00:00), ...
+
+ @daily_partitioned_config(start_date="2022-03-12", minute_offset=15, hour_offset=16)
+ # creates partitions (2022-03-12-16:15, 2022-03-13-16:15), (2022-03-13-16:15, 2022-03-14-16:15), ...
+ ```
+
+
+
+
+
+
dagster.weekly_partitioned_config
+
+
+ Defines run config over a set of weekly partitions.
+
+ The decorated function should accept a start datetime and end datetime, which represent the date
+ partition the config should delineate.
+
+ The decorated function should return a run config dictionary.
+
+ The resulting object created by this decorator can be provided to the config argument of a Job.
+ The first partition in the set will start at the start_date. The last partition in the set will
+ end before the current time, unless the end_offset argument is set to a positive number. If
+ day_offset is provided, the start and end date of each partition will be day of the week
+ corresponding to day_offset (0 indexed with Sunday as the start of the week). If
+ minute_offset and/or hour_offset are used, the start and end times of each partition will be
+ hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions will
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - day_offset (int) – Day of the week to “split” the partition. Defaults to 0 (Sunday).
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – A function that
+
+
+ ```python
+ @weekly_partitioned_config(start_date="2022-03-12")
+ # creates partitions (2022-03-13-00:00, 2022-03-20-00:00), (2022-03-20-00:00, 2022-03-27-00:00), ...
+
+ @weekly_partitioned_config(start_date="2022-03-12", minute_offset=15, hour_offset=3, day_offset=6)
+ # creates partitions (2022-03-12-03:15, 2022-03-19-03:15), (2022-03-19-03:15, 2022-03-26-03:15), ...
+ ```
+
+
+
+
+
+
dagster.monthly_partitioned_config
+
+
+ Defines run config over a set of monthly partitions.
+
+ The decorated function should accept a start datetime and end datetime, which represent the date
+ partition the config should delineate.
+
+ The decorated function should return a run config dictionary.
+
+ The resulting object created by this decorator can be provided to the config argument of a Job.
+ The first partition in the set will start at midnight on the soonest first of the month after
+ start_date. The last partition in the set will end before the current time, unless the
+ end_offset argument is set to a positive number. If day_offset is provided, the start and end
+ date of each partition will be day_offset. If minute_offset and/or hour_offset are used, the
+ start and end times of each partition will be hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions will be
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - day_offset (int) – Day of the month to “split” the partition. Defaults to 1.
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – A function that
+
+
+ ```python
+ @monthly_partitioned_config(start_date="2022-03-12")
+ # creates partitions (2022-04-01-00:00, 2022-05-01-00:00), (2022-05-01-00:00, 2022-06-01-00:00), ...
+
+ @monthly_partitioned_config(start_date="2022-03-12", minute_offset=15, hour_offset=3, day_offset=5)
+ # creates partitions (2022-04-05-03:15, 2022-05-05-03:15), (2022-05-05-03:15, 2022-06-05-03:15), ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/pipes.mdx b/docs/docs-beta/docs/api/python-api/pipes.mdx
new file mode 100644
index 0000000000000..60f8019af296c
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/pipes.mdx
@@ -0,0 +1,416 @@
+---
+title: 'dagster pipes'
+title_meta: 'dagster pipes API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dagster pipes Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Dagster Pipes
+
+[Dagster Pipes](https://docs.dagster.io/guides/build/external-pipelines/) is a toolkit for building integrations between Dagster and external execution environments. This reference outlines the APIs included with the `dagster` library, which should be used in the orchestration environment.
+
+For a detailed look at the Pipes process, including how to customize it, refer to the [Dagster Pipes details and customization guide](https://docs.dagster.io/guides/build/external-pipelines/dagster-pipes-details-and-customization).
+
+Looking to write code in an external process? Refer to the API reference for the separately-installed [dagster-pipes](https://docs.dagster.io/api/python-api/libraries/dagster-pipes) library.
+
+
+
+
+## Sessions
+
+
+
class dagster.PipesSession
+
+
+ Object representing a pipes session.
+
+ A pipes session is defined by a pair of [`PipesContextInjector`](#dagster.PipesContextInjector) and
+ [`PipesMessageReader`](#dagster.PipesMessageReader) objects. At the opening of the session, the context injector
+ writes context data to an externally accessible location, and the message reader starts
+ monitoring an externally accessible location. These locations are encoded in parameters stored
+ on a PipesSession object.
+
+ During the session, an external process should be started and the parameters injected into its
+ environment. The typical way to do this is to call [`PipesSession.get_bootstrap_env_vars()`](#dagster.PipesSession.get_bootstrap_env_vars)
+ and pass the result as environment variables.
+
+ During execution, results (e.g. asset materializations) are reported by the external process and
+ buffered on the PipesSession object. The buffer can periodically be cleared and yielded to
+ Dagster machinery by calling yield from PipesSession.get_results().
+
+ When the external process exits, the session can be closed. Closing consists of handling any
+ unprocessed messages written by the external process and cleaning up any resources used for
+ context injection and message reading.
+
+ Parameters:
+ - context_data (PipesContextData) – The context for the executing op/asset.
+ - message_handler ([*PipesMessageHandler*](#dagster.PipesMessageHandler)) – The message handler to use for processing messages
+ - context_injector_params (PipesParams) – Parameters yielded by the context injector,
+ - message_reader_params (PipesParams) – Parameters yielded by the message reader, indicating
+ - created_at (datetime) – The time at which the session was created. Useful as cutoff for
+
+
+
+
get_bootstrap_cli_arguments
+
+
+ Encode context injector and message reader params as CLI arguments.
+
+ Passing CLI arguments is an alternative way to expose the pipes I/O parameters to a pipes process.
+ Using environment variables should be preferred when possible.
+
+ Returns: CLI arguments pass to the external process. The values are
+ serialized as json, compressed with zlib, and then base64-encoded.Return type: Mapping[str, str]
+
+
+
+
+
+
get_bootstrap_env_vars
+
+
+ Encode context injector and message reader params as environment variables.
+
+ Passing environment variables is the typical way to expose the pipes I/O parameters
+ to a pipes process.
+
+ Returns: Environment variables to pass to the external process. The values are
+ serialized as json, compressed with gzip, and then base-64-encoded.Return type: Mapping[str, str]
+
+
+
+
+
+
get_bootstrap_params
+
+
+ Get the params necessary to bootstrap a launched pipes process. These parameters are typically
+ are as environment variable. See get_bootstrap_env_vars. It is the context injector’s
+ responsibility to decide how to pass these parameters to the external environment.
+
+ Returns: Parameters to pass to the external process and their corresponding
+ values that must be passed by the context injector.Return type: Mapping[str, str]
+
+
+
+
+
+
get_custom_messages
+
+
+ Get the sequence of deserialized JSON data that was reported from the external process using
+ report_custom_message.
+
+ Returns: Sequence[Any]
+
+
+
+
+
+
+
get_reported_results
+
+
+ `PipesExecutionResult` objects only explicitly received from the external process.
+
+ Returns: Result reported by external process.Return type: Sequence[PipesExecutionResult]
+
+
+
+
+
+
get_results
+
+
+ `PipesExecutionResult` objects reported from the external process,
+ potentially modified by Pipes.
+
+ Parameters:
+ - implicit_materializations (bool) – Create MaterializeResults for expected assets
+ - metadata (Optional[Mapping[str, [*MetadataValue*](metadata.mdx#dagster.MetadataValue)]]) – Arbitrary metadata that will be attached to all
+
+
+ Returns: Result reported by external process.Return type: Sequence[PipesExecutionResult]
+
+
+
+
+
+
+
+
+
+
dagster.open_pipes_session
+
+
+ Context manager that opens and closes a pipes session.
+
+ This context manager should be used to wrap the launch of an external process using the pipe
+ protocol to report results back to Dagster. The yielded [`PipesSession`](#dagster.PipesSession) should be used
+ to (a) obtain the environment variables that need to be provided to the external process; (b)
+ access results streamed back from the external process.
+
+ This method is an alternative to [`PipesClient`](#dagster.PipesClient) subclasses for users who want more
+ control over how pipes processes are launched. When using open_pipes_session, it is the user’s
+ responsibility to inject the message reader and context injector parameters available on the
+ yielded PipesSession and pass them to the appropriate API when launching the external process.
+ Typically these parameters should be set as environment variables.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](execution.mdx#dagster.AssetExecutionContext)]) – The context for the current op/asset execution.
+ - context_injector ([*PipesContextInjector*](#dagster.PipesContextInjector)) – The context injector to use to inject context into the external process.
+ - message_reader ([*PipesMessageReader*](#dagster.PipesMessageReader)) – The message reader to use to read messages from the external process.
+ - extras (Optional[PipesExtras]) – Optional extras to pass to the external process via the injected context.
+
+
+ Yields: PipesSession – Interface for interacting with the external process.
+ ```python
+ import subprocess
+ from dagster import open_pipes_session
+
+ extras = {"foo": "bar"}
+
+ @asset
+ def ext_asset(context: AssetExecutionContext):
+ with open_pipes_session(
+ context=context,
+ extras={"foo": "bar"},
+ context_injector=PipesTempFileContextInjector(),
+ message_reader=PipesTempFileMessageReader(),
+ ) as pipes_session:
+ subprocess.Popen(
+ ["/bin/python", "/path/to/script.py"],
+ env={**pipes_session.get_bootstrap_env_vars()}
+ )
+ while process.poll() is None:
+ yield from pipes_session.get_results()
+
+ yield from pipes_session.get_results()
+ ```
+
+
+
+
+
+
+
+
+
+
+## Clients
+
+
+
class dagster.PipesClient
+
+
+ Pipes client base class.
+
+ Pipes clients for specific external environments should subclass this.
+
+
+
abstract run
+
+
+ Synchronously execute an external process with the pipes protocol. Derived
+ clients must have context and extras arguments, but also can add arbitrary
+ arguments that are appropriate for their own implementation.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](execution.mdx#dagster.AssetExecutionContext)]) – The context from the executing op/asset.
+ - extras (Optional[PipesExtras]) – Arbitrary data to pass to the external environment.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster.PipesSubprocessClient
+
+
+ A pipes client that runs a subprocess with the given command and environment.
+
+ By default parameters are injected via environment variables. Context is passed via
+ a temp file, and structured messages are read from from a temp file.
+
+ Parameters:
+ - env (Optional[Mapping[str, str]]) – An optional dict of environment variables to pass to the
+ - cwd (Optional[str]) – Working directory in which to launch the subprocess command.
+ - context_injector (Optional[[*PipesContextInjector*](#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](#dagster.PipesMessageReader)]) – A message reader to use to read messages from
+ - forward_termination (bool) – Whether to send a SIGINT signal to the subprocess
+ - forward_stdio (bool) – Whether to forward stdout and stderr from the subprocess to the
+ - termination_timeout_seconds (float) – How long to wait after forwarding termination
+
+
+
+
run
+
+
+ Synchronously execute a subprocess with in a pipes session.
+
+ Parameters:
+ - command (Union[str, Sequence[str]]) – The command to run. Will be passed to subprocess.Popen().
+ - context (Union[[*OpExecutionContext*](execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](execution.mdx#dagster.AssetExecutionContext)]) – The context from the executing op or asset.
+ - extras (Optional[PipesExtras]) – An optional dict of extra parameters to pass to the subprocess.
+ - env (Optional[Mapping[str, str]]) – An optional dict of environment variables to pass to the subprocess.
+ - cwd (Optional[str]) – Working directory in which to launch the subprocess command.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Advanced
+
+Most Pipes users won’t need to use the APIs in the following sections unless they are customizing the Pipes protocol.
+
+Refer to the [Dagster Pipes details and customization guide](https://docs.dagster.io/guides/build/external-pipelines/dagster-pipes-details-and-customization) for more information.
+
+
+
+
+### Context injectors
+
+Context injectors write context payloads to an externally accessible location and yield a set of parameters encoding the location for inclusion in the bootstrap payload.
+
+
+
class dagster.PipesContextInjector
+
+
+
+
+
+
+
class dagster.PipesEnvContextInjector
+
+ Context injector that injects context data into the external process by injecting it directly into the external process environment.
+
+
+
+
+
class dagster.PipesFileContextInjector
+
+
+ Context injector that injects context data into the external process by writing it to a
+ specified file.
+
+ Parameters: path (str) – The path of a file to which to write context data. The file will be deleted on
+ close of the pipes session.
+
+
+
+
+
+
class dagster.PipesTempFileContextInjector
+
+ Context injector that injects context data into the external process by writing it to an
+ automatically-generated temporary file.
+
+
+
+
+
+
+
+
+### Message readers
+
+Message readers read messages (and optionally log files) from an externally accessible location and yield a set of parameters encoding the location in the bootstrap payload.
+
+
+
class dagster.PipesMessageReader
+
+
+
+
+
+
+
class dagster.PipesBlobStoreMessageReader
+
+
+ Message reader that reads a sequence of message chunks written by an external process into a
+ blob store such as S3, Azure blob storage, or GCS.
+
+ The reader maintains a counter, starting at 1, that is synchronized with a message writer in
+ some pipes process. The reader starts a thread that periodically attempts to read a chunk
+ indexed by the counter at some location expected to be written by the pipes process. The chunk
+ should be a file with each line corresponding to a JSON-encoded pipes message. When a chunk is
+ successfully read, the messages are processed and the counter is incremented. The
+ `PipesBlobStoreMessageWriter` on the other end is expected to similarly increment a
+ counter (starting from 1) on successful write, keeping counters on the read and write end in
+ sync.
+
+ If log_readers is passed, the message reader will start the passed log readers when the
+ opened message is received from the external process.
+
+ Parameters:
+ - interval (float) – interval in seconds between attempts to download a chunk
+ - log_readers (Optional[Sequence[PipesLogReader]]) – A set of log readers to use to read logs.
+
+
+
+
+
+
+
+
class dagster.PipesFileMessageReader
+
+
+ Message reader that reads messages by tailing a specified file.
+
+ Parameters:
+ - path (str) – The path of the file to which messages will be written. The file will be deleted
+ - include_stdio_in_messages (bool) – Whether to include stdout/stderr logs in the messages produced by the message writer in the external process.
+ - cleanup_file (bool) – Whether to delete the file on close of the pipes session.
+
+
+
+
+
+
+
+
class dagster.PipesTempFileMessageReader
+
+ Message reader that reads messages by tailing an automatically-generated temporary file.
+
+
+
+
+
class dagster.PipesMessageHandler
+
+
+ Class to process `PipesMessage` objects received from a pipes process.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](execution.mdx#dagster.AssetExecutionContext)]) – The context for the executing op/asset.
+ - message_reader ([*PipesMessageReader*](#dagster.PipesMessageReader)) – The message reader used to read messages from the
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/repositories.mdx b/docs/docs-beta/docs/api/python-api/repositories.mdx
new file mode 100644
index 0000000000000..b72ac8c88e3f1
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/repositories.mdx
@@ -0,0 +1,552 @@
+---
+title: 'repositories'
+title_meta: 'repositories API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'repositories Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Repositories
+
+
+
dagster.repository RepositoryDefinition
+
+
+ Create a repository from the decorated function.
+
+ In most cases, [`Definitions`](definitions.mdx#dagster.Definitions) should be used instead.
+
+ The decorated function should take no arguments and its return value should one of:
+
+ 1. `List[Union[JobDefinition, ScheduleDefinition, SensorDefinition]]`.
+ Use this form when you have no need to lazy load jobs or other definitions. This is the
+ typical use case.
+
+ 2. A dict of the form:
+ ```python
+ {
+ 'jobs': Dict[str, Callable[[], JobDefinition]],
+ 'schedules': Dict[str, Callable[[], ScheduleDefinition]]
+ 'sensors': Dict[str, Callable[[], SensorDefinition]]
+ }
+ ```
+ This form is intended to allow definitions to be created lazily when accessed by name,
+ which can be helpful for performance when there are many definitions in a repository, or
+ when constructing the definitions is costly.
+
+ 3. A [`RepositoryData`](#dagster.RepositoryData). Return this object if you need fine-grained
+ control over the construction and indexing of definitions within the repository, e.g., to
+ create definitions dynamically from .yaml files in a directory.
+
+ Parameters:
+ - name (Optional[str]) – The name of the repository. Defaults to the name of the decorated
+ - description (Optional[str]) – A string description of the repository.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata for the repository. Not
+ - top_level_resources (Optional[Mapping[str, [*ResourceDefinition*](resources.mdx#dagster.ResourceDefinition)]]) – A dict of top-level
+
+
+ Example:
+
+ ```python
+ ######################################################################
+ # A simple repository using the first form of the decorated function
+ ######################################################################
+
+ @op(config_schema={n: Field(Int)})
+ def return_n(context):
+ return context.op_config['n']
+
+ @job
+ def simple_job():
+ return_n()
+
+ @job
+ def some_job():
+ ...
+
+ @sensor(job=some_job)
+ def some_sensor():
+ if foo():
+ yield RunRequest(
+ run_key= ...,
+ run_config={
+ 'ops': {'return_n': {'config': {'n': bar()}}}
+ }
+ )
+
+ @job
+ def my_job():
+ ...
+
+ my_schedule = ScheduleDefinition(cron_schedule="0 0 * * *", job=my_job)
+
+ @repository
+ def simple_repository():
+ return [simple_job, some_sensor, my_schedule]
+
+ ######################################################################
+ # A simple repository using the first form of the decorated function
+ # and custom metadata that will be displayed in the UI
+ ######################################################################
+
+ ...
+
+ @repository(
+ name='my_repo',
+ metadata={
+ 'team': 'Team A',
+ 'repository_version': '1.2.3',
+ 'environment': 'production',
+ })
+ def simple_repository():
+ return [simple_job, some_sensor, my_schedule]
+
+ ######################################################################
+ # A lazy-loaded repository
+ ######################################################################
+
+ def make_expensive_job():
+ @job
+ def expensive_job():
+ for i in range(10000):
+ return_n.alias(f'return_n_{i}')()
+
+ return expensive_job
+
+ def make_expensive_schedule():
+ @job
+ def other_expensive_job():
+ for i in range(11000):
+ return_n.alias(f'my_return_n_{i}')()
+
+ return ScheduleDefinition(cron_schedule="0 0 * * *", job=other_expensive_job)
+
+ @repository
+ def lazy_loaded_repository():
+ return {
+ 'jobs': {'expensive_job': make_expensive_job},
+ 'schedules': {'expensive_schedule': make_expensive_schedule}
+ }
+
+
+ ######################################################################
+ # A complex repository that lazily constructs jobs from a directory
+ # of files in a bespoke YAML format
+ ######################################################################
+
+ class ComplexRepositoryData(RepositoryData):
+ def __init__(self, yaml_directory):
+ self._yaml_directory = yaml_directory
+
+ def get_all_jobs(self):
+ return [
+ self._construct_job_def_from_yaml_file(
+ self._yaml_file_for_job_name(file_name)
+ )
+ for file_name in os.listdir(self._yaml_directory)
+ ]
+
+ ...
+
+ @repository
+ def complex_repository():
+ return ComplexRepositoryData('some_directory')
+ ```
+
+
+
+
+
+
class dagster.RepositoryDefinition
+
+
+ Define a repository that contains a group of definitions.
+
+ Users should typically not create objects of this class directly. Instead, use the
+ `@repository()` decorator.
+
+ Parameters:
+ - name (str) – The name of the repository.
+ - repository_data ([*RepositoryData*](#dagster.RepositoryData)) – Contains the definitions making up the repository.
+ - description (Optional[str]) – A string description of the repository.
+ - metadata (Optional[MetadataMapping]) – Arbitrary metadata for the repository. Not
+
+
+
+
get_all_jobs
+
+
+ Return all jobs in the repository as a list.
+
+ Note that this will construct any job in the lazily evaluated dictionary that has
+ not yet been constructed.
+
+ Returns: All jobs in the repository.Return type: List[[JobDefinition](jobs.mdx#dagster.JobDefinition)]
+
+
+
+
+
+
get_asset_value_loader
+
+
+ Returns an object that can load the contents of assets as Python objects.
+
+ Invokes load_input on the [`IOManager`](io-managers.mdx#dagster.IOManager) associated with the assets. Avoids
+ spinning up resources separately for each asset.
+
+ Usage:
+
+ ```python
+ with my_repo.get_asset_value_loader() as loader:
+ asset1 = loader.load_asset_value("asset1")
+ asset2 = loader.load_asset_value("asset2")
+ ```
+
+
+
+
+
+
get_job
+
+
+ Get a job by name.
+
+ If this job is present in the lazily evaluated dictionary passed to the
+ constructor, but has not yet been constructed, only this job is constructed, and
+ will be cached for future calls.
+
+ Parameters: name (str) – Name of the job to retrieve.Returns: The job definition corresponding to
+ the given name.Return type: [JobDefinition](jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
get_schedule_def
+
+
+ Get a schedule definition by name.
+
+ Parameters: name (str) – The name of the schedule.Returns: The schedule definition.Return type: [ScheduleDefinition](schedules-sensors.mdx#dagster.ScheduleDefinition)
+
+
+
+
+
+
get_sensor_def
+
+
+ Get a sensor definition by name.
+
+ Parameters: name (str) – The name of the sensor.Returns: The sensor definition.Return type: [SensorDefinition](schedules-sensors.mdx#dagster.SensorDefinition)
+
+
+
+
+
+
has_job
+
+
+ Check if a job with a given name is present in the repository.
+
+ Parameters: name (str) – The name of the job.Returns: bool
+
+
+
+
+
+
has_schedule_def
+
+ bool: Check if a schedule with a given name is present in the repository.
+
+
+
+
+
has_sensor_def
+
+ bool: Check if a sensor with a given name is present in the repository.
+
+
+
+
+
load_asset_value
+
+
+ Load the contents of an asset as a Python object.
+
+ Invokes load_input on the [`IOManager`](io-managers.mdx#dagster.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()`](#dagster.RepositoryDefinition.get_asset_value_loader), which avoids spinning up
+ resources separately for each asset.
+
+ Parameters:
+ - asset_key (Union[[*AssetKey*](assets.mdx#dagster.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`](io-managers.mdx#dagster.IOManager)
+ - resource_config (Optional[Any]) – A dictionary of resource configurations to be passed
+
+
+ Returns: The contents of an asset as a Python object.
+
+
+
+
+
+
property asset_checks_defs_by_key
+
+
+ The assets checks defined in the repository.
+
+ Type: Mapping[[AssetCheckKey](asset-checks.mdx#dagster.AssetCheckKey), [AssetChecksDefinition](asset-checks.mdx#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
property assets_defs_by_key
+
+
+ The assets definitions defined in the repository.
+
+ Type: Mapping[[AssetKey](assets.mdx#dagster.AssetKey), [AssetsDefinition](assets.mdx#dagster.AssetsDefinition)]
+
+
+
+
+
+
property description
+
+
+ A human-readable description of the repository.
+
+ Type: Optional[str]
+
+
+
+
+
+
property job_names
+
+
+ Names of all jobs in the repository.
+
+ Type: List[str]
+
+
+
+
+
+
property metadata
+
+
+ Arbitrary metadata for the repository.
+
+ Type: Optional[MetadataMapping]
+
+
+
+
+
+
property name
+
+
+ The name of the repository.
+
+ Type: str
+
+
+
+
+
+
property schedule_defs
+
+
+ All schedules in the repository.
+
+ Type: List[[ScheduleDefinition](schedules-sensors.mdx#dagster.ScheduleDefinition)]
+
+
+
+
+
+
property sensor_defs
+
+
+ All sensors in the repository.
+
+ Type: Sequence[[SensorDefinition](schedules-sensors.mdx#dagster.SensorDefinition)]
+
+
+
+
+
+
property source_assets_by_key
+
+
+ The source assets defined in the repository.
+
+ Type: Mapping[[AssetKey](assets.mdx#dagster.AssetKey), [SourceAsset](assets.mdx#dagster.SourceAsset)]
+
+
+
+
+
+
+
+
+
+
class dagster.RepositoryData
+
+
+ Users should usually rely on the [`@repository`](#dagster.repository) decorator to create new
+ repositories, which will in turn call the static constructors on this class. However, users may
+ subclass [`RepositoryData`](#dagster.RepositoryData) for fine-grained control over access to and lazy creation
+ of repository members.
+
+
+
abstract get_all_jobs
+
+
+ Return all jobs in the repository as a list.
+
+ Returns: All jobs in the repository.Return type: List[[JobDefinition](jobs.mdx#dagster.JobDefinition)]
+
+
+
+
+
+
get_all_schedules
+
+
+ Return all schedules in the repository as a list.
+
+ Returns: All jobs in the repository.Return type: List[[ScheduleDefinition](schedules-sensors.mdx#dagster.ScheduleDefinition)]
+
+
+
+
+
+
get_all_sensors
+
+ Sequence[SensorDefinition]: Return all sensors in the repository as a list.
+
+
+
+
+
get_asset_checks_defs_by_key
+
+ Mapping[AssetCheckKey, AssetChecksDefinition]: Get the asset checks definitions for the repository.
+
+
+
+
+
get_assets_defs_by_key
+
+ Mapping[AssetKey, AssetsDefinition]: Get the asset definitions for the repository.
+
+
+
+
+
get_job
+
+
+ Get a job by name.
+
+ Parameters: job_name (str) – Name of the job to retrieve.Returns: The job definition corresponding to the given name.Return type: [JobDefinition](jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
get_job_names
+
+
+ Get the names of all jobs in the repository.
+
+ Returns: List[str]
+
+
+
+
+
+
get_schedule
+
+
+ Get a schedule by name.
+
+ Parameters: schedule_name (str) – name of the schedule to retrieve.Returns: The schedule definition corresponding to the given name.Return type: [ScheduleDefinition](schedules-sensors.mdx#dagster.ScheduleDefinition)
+
+
+
+
+
+
get_schedule_names
+
+
+ Get the names of all schedules in the repository.
+
+ Returns: List[str]
+
+
+
+
+
+
get_sensor
+
+
+ Get a sensor by name.
+
+ Parameters: sensor_name (str) – name of the sensor to retrieve.Returns: The sensor definition corresponding to the given name.Return type: [SensorDefinition](schedules-sensors.mdx#dagster.SensorDefinition)
+
+
+
+
+
+
get_sensor_names
+
+ Sequence[str]: Get the names of all sensors in the repository.
+
+
+
+
+
get_source_assets_by_key
+
+ Mapping[AssetKey, SourceAsset]: Get the source assets for the repository.
+
+
+
+
+
has_job
+
+
+ Check if a job with a given name is present in the repository.
+
+ Parameters: job_name (str) – The name of the job.Returns: bool
+
+
+
+
+
+
has_schedule
+
+ Check if a schedule with a given name is present in the repository.
+
+
+
+
+
has_sensor
+
+ Check if a sensor with a given name is present in the repository.
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/resources.mdx b/docs/docs-beta/docs/api/python-api/resources.mdx
new file mode 100644
index 0000000000000..c86d98a9b55d9
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/resources.mdx
@@ -0,0 +1,474 @@
+---
+title: 'resources'
+title_meta: 'resources API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'resources Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Resources
+
+
+## Pythonic resource system
+
+The following classes are used as part of the new [Pythonic resources system](https://docs.dagster.io/guides/build/external-resources/).
+
+
+
class dagster.ConfigurableResource
+
+
+ Base class for Dagster resources that utilize structured config.
+
+ This class is a subclass of both [`ResourceDefinition`](#dagster.ResourceDefinition) and [`Config`](config.mdx#dagster.Config).
+
+ Example definition:
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ prefix: str
+
+ def output(self, text: str) -> None:
+ print(f"{self.prefix}{text}")
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def asset_that_uses_writer(writer: WriterResource):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[asset_that_uses_writer],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+ You can optionally use this class to model configuration only and vend an object
+ of a different type for use at runtime. This is useful for those who wish to
+ have a separate object that manages configuration and a separate object at runtime. Or
+ where you want to directly use a third-party class that you do not control.
+
+ To do this you override the create_resource methods to return a different object.
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ str: prefix
+
+ def create_resource(self, context: InitResourceContext) -> Writer:
+ # Writer is pre-existing class defined else
+ return Writer(self.prefix)
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def use_preexisting_writer_as_resource(writer: ResourceParam[Writer]):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[use_preexisting_writer_as_resource],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+
+
+
+
+
+
class dagster.ResourceDefinition
+
+
+ Core class for defining resources.
+
+ Resources are scoped ways to make external resources (like database connections) available to
+ ops and assets during job execution and to clean up after execution resolves.
+
+ If resource_fn yields once rather than returning (in the manner of functions decorable with
+ `@contextlib.contextmanager`) then the body of the
+ function after the yield will be run after execution resolves, allowing users to write their
+ own teardown/cleanup logic.
+
+ Depending on your executor, resources may be instantiated and cleaned up more than once in a
+ job execution.
+
+ Parameters:
+ - resource_fn (Callable[[[*InitResourceContext*](#dagster.InitResourceContext)], Any]) – User-provided function to instantiate
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The schema for the config. If set, Dagster will check
+ - description (Optional[str]) – A human-readable description of the resource.
+ - required_resource_keys – (Optional[Set[str]]) Keys for the resources required by this
+ - version (Optional[str]) – experimental
+
+
+
+
static hardcoded_resource
+
+
+ A helper function that creates a `ResourceDefinition` with a hardcoded object.
+
+ Parameters:
+ - value (Any) – The value that will be accessible via context.resources.resource_name.
+ - description ([Optional[str]]) – The description of the resource. Defaults to None.
+
+
+ Returns: A hardcoded resource.Return type: [[ResourceDefinition](#dagster.ResourceDefinition)]
+
+
+
+
+
+
static mock_resource
+
+
+ A helper function that creates a `ResourceDefinition` which wraps a `mock.MagicMock`.
+
+ Parameters: description ([Optional[str]]) – The description of the resource. Defaults to None.Returns:
+ A resource that creates the magic methods automatically and helps
+ you mock existing resources.
+
+ Return type: [[ResourceDefinition](#dagster.ResourceDefinition)]
+
+
+
+
+
+
static none_resource
+
+
+ A helper function that returns a none resource.
+
+ Parameters: description ([Optional[str]]) – The description of the resource. Defaults to None.Returns: A resource that does nothing.Return type: [[ResourceDefinition](#dagster.ResourceDefinition)]
+
+
+
+
+
+
static string_resource
+
+
+ Creates a `ResourceDefinition` which takes in a single string as configuration
+ and returns this configured string to any ops or assets which depend on it.
+
+ Parameters: description ([Optional[str]]) – The description of the string resource. Defaults to None.Returns:
+ A resource that takes in a single string as configuration and
+ returns that string.
+
+ Return type: [[ResourceDefinition](#dagster.ResourceDefinition)]
+
+
+
+
+
+
property description
+
+ A human-readable description of the resource.
+
+
+
+
+
property required_resource_keys
+
+ A set of the resource keys that this resource depends on. These keys will be made available
+ to the resource’s init context during execution, and the resource will not be instantiated
+ until all required resources are available.
+
+
+
+
+
property version
+
+ A string which can be used to identify a particular code version of a resource definition.
+
+
+
+
+
+
+
+
+
class dagster.InitResourceContext
+
+
+ The context object available as the argument to the initialization function of a [`dagster.ResourceDefinition`](#dagster.ResourceDefinition).
+
+ Users should not instantiate this object directly. To construct an InitResourceContext for testing purposes, use [`dagster.build_init_resource_context()`](#dagster.build_init_resource_context).
+
+ Example:
+
+ ```python
+ from dagster import resource, InitResourceContext
+
+ @resource
+ def the_resource(init_context: InitResourceContext):
+ init_context.log.info("Hello, world!")
+ ```
+
+
property instance
+
+ The Dagster instance configured for the current execution context.
+
+
+
+
+
property log
+
+ The Dagster log manager configured for the current execution context.
+
+
+
+
+
property log_manager
+
+ The log manager for this run of the job.
+
+
+
+
+
property resource_config
+
+ The configuration data provided by the run config. The schema
+ for this data is defined by the `config_field` argument to
+ [`ResourceDefinition`](#dagster.ResourceDefinition).
+
+
+
+
+
property resource_def
+
+ The definition of the resource currently being constructed.
+
+
+
+
+
property resources
+
+ The resources that are available to the resource that we are initializing.
+
+
+
+
+
property run_id
+
+ The id for this run of the job or pipeline. When initializing resources outside of
+ execution context, this will be None.
+
+
+
+
+
+
+
+
+
+
dagster.make_values_resource
+
+
+ A helper function that creates a `ResourceDefinition` to take in user-defined values.
+
+ >
+
+ This is useful for sharing values between ops.
+
+
+
+Parameters: **kwargs – Arbitrary keyword arguments that will be passed to the config schema of the
+returned resource definition. If not set, Dagster will accept any config provided for
+the resource.
+For example:
+
+ ```python
+ @op(required_resource_keys={"globals"})
+ def my_op(context):
+ print(context.resources.globals["my_str_var"])
+
+ @job(resource_defs={"globals": make_values_resource(my_str_var=str, my_int_var=int)})
+ def my_job():
+ my_op()
+ ```
+Returns: A resource that passes in user-defined values.Return type: [ResourceDefinition](#dagster.ResourceDefinition)
+
+
+
+
+
+
dagster.build_init_resource_context
+
+
+ Builds resource initialization context from provided parameters.
+
+ `build_init_resource_context` can be used as either a function or context manager. If there is a
+ provided resource to `build_init_resource_context` that is a context manager, then it must be
+ used as a context manager. This function can be used to provide the context argument to the
+ invocation of a resource.
+
+ Parameters:
+ - resources (Optional[Dict[str, Any]]) – The resources to provide to the context. These can be
+ - config (Optional[Any]) – The resource config to provide to the context.
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The dagster instance configured for the context.
+
+
+ Examples:
+
+ ```python
+ context = build_init_resource_context()
+ resource_to_init(context)
+
+ with build_init_resource_context(
+ resources={"foo": context_manager_resource}
+ ) as context:
+ resource_to_init(context)
+ ```
+
+
+
+
+
+
dagster.build_resources
+
+
+ Context manager that yields resources using provided resource definitions and run config.
+
+ This API allows for using resources in an independent context. Resources will be initialized
+ with the provided run config, and optionally, dagster_run. The resulting resources will be
+ yielded on a dictionary keyed identically to that provided for resource_defs. Upon exiting the
+ context, resources will also be torn down safely.
+
+ Parameters:
+ - resources (Mapping[str, Any]) – Resource instances or definitions to build. All
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The dagster instance configured to instantiate
+ - resource_config (Optional[Mapping[str, Any]]) – A dict representing the config to be
+ - dagster_run (Optional[PipelineRun]) – The pipeline run to provide during resource
+ - log_manager (Optional[[*DagsterLogManager*](loggers.mdx#dagster.DagsterLogManager)]) – Log Manager to use during resource
+
+
+ Examples:
+
+ ```python
+ from dagster import resource, build_resources
+
+ @resource
+ def the_resource():
+ return "foo"
+
+ with build_resources(resources={"from_def": the_resource, "from_val": "bar"}) as resources:
+ assert resources.from_def == "foo"
+ assert resources.from_val == "bar"
+ ```
+
+
+
+
+
+
dagster.with_resources
+
+
+ Adds dagster resources to copies of resource-requiring dagster definitions.
+
+ An error will be thrown if any provided definitions have a conflicting
+ resource definition provided for a key provided to resource_defs. Resource
+ config can be provided, with keys in the config dictionary corresponding to
+ the keys for each resource definition. If any definition has unsatisfied
+ resource keys after applying with_resources, an error will be thrown.
+
+ Parameters:
+ - definitions (Iterable[ResourceAddable]) – Dagster definitions to provide resources to.
+ - resource_defs (Mapping[str, object]) – Mapping of resource keys to objects to satisfy
+ - resource_config_by_key (Optional[Mapping[str, Any]]) – Specifies config for provided resources. The key in this dictionary
+
+
+ Examples:
+
+ ```python
+ from dagster import asset, resource, with_resources
+
+ @resource(config_schema={"bar": str})
+ def foo_resource():
+ ...
+
+ @asset(required_resource_keys={"foo"})
+ def asset1(context):
+ foo = context.resources.foo
+ ...
+
+ @asset(required_resource_keys={"foo"})
+ def asset2(context):
+ foo = context.resources.foo
+ ...
+
+ asset1_with_foo, asset2_with_foo = with_resources(
+ [asset1, asset2],
+ resource_defs={
+ "foo": foo_resource
+ },
+ resource_config_by_key={
+ "foo": {
+ "config": {"bar": ...}
+ }
+ }
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+## Utilities
+
+
+
class dagster.EnvVar
+
+
+ Class used to represent an environment variable in the Dagster config system.
+
+ This class is intended to be used to populate config fields or resources.
+ The environment variable will be resolved to a string value when the config is
+ loaded.
+
+ To access the value of the environment variable, use the get_value method.
+
+
+
+
+
+
+
+
+
+
+## Legacy resource system
+
+The following classes are used as part of the [legacy resource system](https://legacy-docs.dagster.io/concepts/resources-legacy).
+
+
+
@dagster.resource
+
+
+ Define a resource.
+
+ The decorated function should accept an [`InitResourceContext`](#dagster.InitResourceContext) and return an instance of
+ the resource. This function will become the `resource_fn` of an underlying
+ [`ResourceDefinition`](#dagster.ResourceDefinition).
+
+ If the decorated function yields once rather than returning (in the manner of functions
+ decorable with `@contextlib.contextmanager`) then
+ the body of the function after the yield will be run after execution resolves, allowing users
+ to write their own teardown/cleanup logic.
+
+ Parameters:
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of the resource.
+ - version (Optional[str]) – (Experimental) The version of a resource function. Two wrapped
+ - required_resource_keys (Optional[Set[str]]) – Keys for the resources required by this resource.
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/schedules-sensors.mdx b/docs/docs-beta/docs/api/python-api/schedules-sensors.mdx
new file mode 100644
index 0000000000000..ca5dd08c5cd9d
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/schedules-sensors.mdx
@@ -0,0 +1,1307 @@
+---
+title: 'schedules and sensors'
+title_meta: 'schedules and sensors API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'schedules and sensors Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Schedules and sensors
+
+Dagster offers several ways to run data pipelines without manual intervation, including traditional scheduling and event-based triggers. [Automating your Dagster pipelines](https://docs.dagster.io/guides/automate/) can boost efficiency and ensure that data is produced consistently and reliably.
+
+
+
+
+## Run requests
+
+
+
class dagster.RunRequest
+
+
+
+
+
+
+
class dagster.SkipReason
+
+
+ Represents a skipped evaluation, where no runs are requested. May contain a message to indicate
+ why no runs were requested.
+
+
+
skip_message
+
+
+ A message displayed in the Dagster UI for why this evaluation resulted
+ in no requested runs.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Schedules
+
+[Schedules](https://docs.dagster.io/guides/automate/schedules/) are Dagster’s way to support traditional ways of automation, such as specifying a job should run at Mondays at 9:00AM. Jobs triggered by schedules can contain a subset of [assets](https://docs.dagster.io/guides/build/assets/) or [ops](https://legacy-docs.dagster.io/concepts/ops-jobs-graphs/ops).
+
+
+
@dagster.schedule
+
+
+ Creates a schedule following the provided cron schedule and requests runs for the provided job.
+
+ The decorated function takes in a [`ScheduleEvaluationContext`](#dagster.ScheduleEvaluationContext) as its only
+ argument, and does one of the following:
+
+ 1. Return a [`RunRequest`](#dagster.RunRequest) object.
+ 2. Return a list of [`RunRequest`](#dagster.RunRequest) objects.
+ 3. Return a [`SkipReason`](#dagster.SkipReason) object, providing a descriptive message of why no runs were requested.
+ 4. Return nothing (skipping without providing a reason)
+ 5. Return a run config dictionary.
+ 6. Yield a [`SkipReason`](#dagster.SkipReason) or yield one ore more [`RunRequest`](#dagster.RunRequest) objects.
+ Returns a [`ScheduleDefinition`](#dagster.ScheduleDefinition).
+
+ Parameters:
+ - cron_schedule (Union[str, Sequence[str]]) – A valid cron string or sequence of cron strings
+ - name (Optional[str]) – The name of the schedule.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the schedule and can
+ - tags_fn (Optional[Callable[[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)], Optional[Dict[str, str]]]]) – A function
+ - metadata (Optional[Mapping[str, Any]]) – A set of metadata entries that annotate the
+ - should_execute (Optional[Callable[[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)], bool]]) – A function that runs at
+ - execution_timezone (Optional[str]) – Timezone in which the schedule should run.
+ - description (Optional[str]) – A human-readable description of the schedule.
+ - job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The job
+ - default_status (DefaultScheduleStatus) – If set to `RUNNING`, the schedule will immediately be active when starting Dagster. The default status can be overridden from the Dagster UI or via the GraphQL API.
+ - required_resource_keys (Optional[Set[str]]) – The set of resource keys required by the schedule.
+ - target (Optional[Union[CoercibleToAssetSelection, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The target that the schedule will execute.
+
+
+
+
+
+
+
+
class dagster.ScheduleDefinition
+
+
+ Defines a schedule that targets a job.
+
+ Parameters:
+ - name (Optional[str]) – The name of the schedule to create. Defaults to the job name plus
+ - cron_schedule (Union[str, Sequence[str]]) – A valid cron string or sequence of cron strings
+ - execution_fn (Callable[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)]) –
+
+ The core evaluation function for the schedule, which is run at an interval to determine whether a run should be launched or not. Takes a [`ScheduleEvaluationContext`](#dagster.ScheduleEvaluationContext).
+
+ - run_config (Optional[Union[[*RunConfig*](config.mdx#dagster.RunConfig), Mapping]]) – The config that parameterizes this execution,
+ - run_config_fn (Optional[Callable[[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)], [Mapping]]]) – A function that
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the schedule
+ - tags_fn (Optional[Callable[[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)], Optional[Mapping[str, str]]]]) – A
+ - should_execute (Optional[Callable[[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)], bool]]) – A function that runs
+ - execution_timezone (Optional[str]) –
+
+ - description (Optional[str]) – A human-readable description of the schedule.
+ - job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition)]]) – The job that should execute when this
+ - default_status (DefaultScheduleStatus) – If set to `RUNNING`, the schedule will start as running. The default status can be overridden from the Dagster UI or via the GraphQL API.
+ - required_resource_keys (Optional[Set[str]]) – The set of resource keys required by the schedule.
+ - target (Optional[Union[CoercibleToAssetSelection, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The target that the schedule will execute.
+ - metadata (Optional[Mapping[str, Any]]) – A set of metadata entries that annotate the
+
+
+
+
property cron_schedule
+
+
+ The cron schedule representing when this schedule will be evaluated.
+
+ Type: Union[str, Sequence[str]]
+
+
+
+
+
+
property default_status
+
+
+ The default status for this schedule when it is first loaded in
+ a code location.
+
+ Type: DefaultScheduleStatus
+
+
+
+
+
+
property description
+
+
+ A description for this schedule.
+
+ Type: Optional[str]
+
+
+
+
+
+
property environment_vars
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Setting this property no longer has any effect..
+
+ :::
+
+ Environment variables to export to the cron schedule.
+
+ Type: Mapping[str, str]
+
+
+
+
+
+
property execution_timezone
+
+
+ The timezone in which this schedule will be evaluated.
+
+ Type: Optional[str]
+
+
+
+
+
+
property job
+
+
+ The job that is
+ targeted by this schedule.
+
+ Type: Union[[JobDefinition](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]
+
+
+
+
+
+
property job_name
+
+
+ The name of the job targeted by this schedule.
+
+ Type: str
+
+
+
+
+
+
property metadata
+
+
+ The metadata for this schedule.
+
+ Type: Mapping[str, str]
+
+
+
+
+
+
property name
+
+
+ The name of the schedule.
+
+ Type: str
+
+
+
+
+
+
property required_resource_keys
+
+
+ The set of keys for resources that must be provided to this schedule.
+
+ Type: Set[str]
+
+
+
+
+
+
property tags
+
+
+ The tags for this schedule.
+
+ Type: Mapping[str, str]
+
+
+
+
+
+
+
+
+
+
class dagster.ScheduleEvaluationContext
+
+
+ The context object available as the first argument to various functions defined on a [`dagster.ScheduleDefinition`](#dagster.ScheduleDefinition).
+
+ A `ScheduleEvaluationContext` object is passed as the first argument to `run_config_fn`, `tags_fn`,
+ and `should_execute`.
+
+ Users should not instantiate this object directly. To construct a `ScheduleEvaluationContext` for testing purposes, use [`dagster.build_schedule_context()`](#dagster.build_schedule_context).
+
+ Example:
+
+ ```python
+ from dagster import schedule, ScheduleEvaluationContext
+
+ @schedule
+ def the_schedule(context: ScheduleEvaluationContext):
+ ...
+ ```
+
+
property instance
+
+
+ The current [`DagsterInstance`](internals.mdx#dagster.DagsterInstance).
+
+ Type: [DagsterInstance](internals.mdx#dagster.DagsterInstance)
+
+
+
+
+
+
property resources
+
+ Mapping of resource key to resource definition to be made available
+ during schedule execution.
+
+
+
+
+
property scheduled_execution_time
+
+ The time in which the execution was scheduled to happen. May differ slightly
+ from both the actual execution time and the time at which the run config is computed.
+
+
+
+
+
+
+
+
+
dagster.build_schedule_context
+
+
+ Builds schedule execution context using the provided parameters.
+
+ The instance provided to `build_schedule_context` must be persistent;
+ [`DagsterInstance.ephemeral()`](internals.mdx#dagster.DagsterInstance) will result in an error.
+
+ Parameters:
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The Dagster instance configured to run the schedule.
+ - scheduled_execution_time (datetime) – The time in which the execution was scheduled to
+
+
+ Examples:
+
+ ```python
+ context = build_schedule_context(instance)
+ ```
+
+
+
+
+
+
dagster.build_schedule_from_partitioned_job
+
+
+ Creates a schedule from a job that targets
+ time window-partitioned or statically-partitioned assets. The job can also be
+ multi-partitioned, as long as one of the partition dimensions is time-partitioned.
+
+ The schedule executes at the cadence specified by the time partitioning of the job or assets.
+
+ Example:
+ ```python
+ ######################################
+ # Job that targets partitioned assets
+ ######################################
+
+ from dagster import (
+ DailyPartitionsDefinition,
+ asset,
+ build_schedule_from_partitioned_job,
+ define_asset_job,
+ Definitions,
+ )
+
+ @asset(partitions_def=DailyPartitionsDefinition(start_date="2020-01-01"))
+ def asset1():
+ ...
+
+ asset1_job = define_asset_job("asset1_job", selection=[asset1])
+
+ # The created schedule will fire daily
+ asset1_job_schedule = build_schedule_from_partitioned_job(asset1_job)
+
+ defs = Definitions(assets=[asset1], schedules=[asset1_job_schedule])
+
+ ################
+ # Non-asset job
+ ################
+
+ from dagster import DailyPartitionsDefinition, build_schedule_from_partitioned_job, jog
+
+
+ @job(partitions_def=DailyPartitionsDefinition(start_date="2020-01-01"))
+ def do_stuff_partitioned():
+ ...
+
+ # The created schedule will fire daily
+ do_stuff_partitioned_schedule = build_schedule_from_partitioned_job(
+ do_stuff_partitioned,
+ )
+
+ defs = Definitions(schedules=[do_stuff_partitioned_schedule])
+ ```
+
+
+
+ Default scheduler implementation that submits runs from the long-lived `dagster-daemon`
+ process. Periodically checks each running schedule for execution times that don’t yet
+ have runs and launches them.
+
+
+
+
+
+
+
+
+
+
+## Sensors
+
+[Sensors](https://docs.dagster.io/guides/automate/sensors/) are typically used to poll, listen, and respond to external events. For example, you could configure a sensor to run a job or materialize an asset in response to specific events.
+
+
+
@dagster.sensor
+
+
+ Creates a sensor where the decorated function is used as the sensor’s evaluation function.
+
+ The decorated function may:
+
+ 1. Return a RunRequest object.
+ 2. Return a list of RunRequest objects.
+ 3. Return a SkipReason object, providing a descriptive message of why no runs were requested.
+ 4. Return nothing (skipping without providing a reason)
+ 5. Yield a SkipReason or yield one or more RunRequest objects.
+ Takes a `SensorEvaluationContext`.
+
+ Parameters:
+ - name (Optional[str]) – The name of the sensor. Defaults to the name of the decorated
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The job to be executed when the sensor fires.
+ - jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]]) – (experimental) A list of jobs to be executed when the sensor fires.
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - asset_selection (Optional[Union[str, Sequence[str], Sequence[[*AssetKey*](assets.mdx#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]], [*AssetSelection*](assets.mdx#dagster.AssetSelection)]]) – (Experimental) an asset selection to launch a run for if the sensor condition is met.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+ - target (Optional[Union[CoercibleToAssetSelection, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The target that the sensor will execute.
+
+
+
+
+
+
+
+
class dagster.SensorDefinition
+
+
+ Define a sensor that initiates a set of runs based on some external state.
+
+ Parameters:
+ - evaluation_fn (Callable[[SensorEvaluationContext]]) –
+
+ The core evaluation function for the
+ sensor, which is run at an interval to determine whether a run should be launched or
+ not. Takes a `SensorEvaluationContext`.
+
+ - name (Optional[str]) – The name of the sensor to create. Defaults to name of evaluation_fn
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - job (Optional[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJob]) – The job to execute when this sensor fires.
+ - jobs (Optional[Sequence[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJob]]) – (experimental) A list of jobs to execute when this sensor fires.
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - asset_selection (Optional[Union[str, Sequence[str], Sequence[[*AssetKey*](assets.mdx#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]], [*AssetSelection*](assets.mdx#dagster.AssetSelection)]]) – (Experimental) an asset selection to launch a run for if the sensor condition is met.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+ - target (Optional[Union[CoercibleToAssetSelection, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The target that the sensor will execute.
+
+
+
+
property default_status
+
+
+ The default status for this sensor when it is first loaded in
+ a code location.
+
+ Type: DefaultSensorStatus
+
+
+
+
+
+
property description
+
+
+ A description for this sensor.
+
+ Type: Optional[str]
+
+
+
+
+
+
property job
+
+
+ The job that is
+ targeted by this schedule.
+
+ Type: Union[[GraphDefinition](graphs.mdx#dagster.GraphDefinition), [JobDefinition](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]
+
+
+
+
+
+
property job_name
+
+
+ The name of the job that is targeted by this sensor.
+
+ Type: Optional[str]
+
+
+
+
+
+
property jobs
+
+
+ A list of jobs
+ that are targeted by this schedule.
+
+ Type: List[Union[[GraphDefinition](graphs.mdx#dagster.GraphDefinition), [JobDefinition](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]
+
+
+
+
+
+
property minimum_interval_seconds
+
+
+ The minimum number of seconds between sequential evaluations of this sensor.
+
+ Type: Optional[int]
+
+
+
+
+
+
property name
+
+
+ The name of this sensor.
+
+ Type: str
+
+
+
+
+
+
property required_resource_keys
+
+
+ The set of keys for resources that must be provided to this sensor.
+
+ Type: Set[str]
+
+
+
+
+
+
+
+
+
+
class dagster.SensorEvaluationContext
+
+
+ The context object available as the argument to the evaluation function of a [`dagster.SensorDefinition`](#dagster.SensorDefinition).
+
+ Users should not instantiate this object directly. To construct a
+ SensorEvaluationContext for testing purposes, use `dagster.
+ build_sensor_context()`.
+
+
+
instance_ref
+
+
+ The serialized instance configured to run the schedule
+
+ Type: Optional[[InstanceRef](internals.mdx#dagster._core.instance.InstanceRef)]
+
+
+
+
+
+
cursor
+
+
+ The cursor, passed back from the last sensor evaluation via
+ the cursor attribute of SkipReason and RunRequest
+
+ Type: Optional[str]
+
+
+
+
+
+
last_tick_completion_time
+
+
+ The last time that the sensor was evaluated (UTC).
+
+ Type: float
+
+
+
+
+
+
last_run_key
+
+
+ DEPRECATED The run key of the RunRequest most recently created by this
+ sensor. Use the preferred cursor attribute instead.
+
+ Type: str
+
+
+
+
+
+
log_key
+
+
+ The log key to use for this sensor tick.
+
+ Type: Optional[List[str]]
+
+
+
+
+
+
repository_name
+
+
+ The name of the repository that the sensor belongs to.
+
+ Type: Optional[str]
+
+
+
+
+
+
repository_def
+
+
+ The repository or that
+ the sensor belongs to. If needed by the sensor top-level resource definitions will be
+ pulled from this repository. You can provide either this or definitions.
+
+ Type: Optional[[RepositoryDefinition](repositories.mdx#dagster.RepositoryDefinition)]
+
+
+
+
+
+
instance
+
+
+ The deserialized instance can also be passed in
+ directly (primarily useful in testing contexts).
+
+ Type: Optional[[DagsterInstance](internals.mdx#dagster.DagsterInstance)]
+
+
+
+
+
+
definitions
+
+
+ Definitions object that the sensor is defined in.
+ If needed by the sensor, top-level resource definitions will be pulled from these
+ definitions. You can provide either this or repository_def.
+
+ Type: Optional[[Definitions](definitions.mdx#dagster.Definitions)]
+
+
+
+
+
+
resources
+
+
+ A dict of resource keys to resource
+ definitions to be made available during sensor execution.
+
+ Type: Optional[Dict[str, Any]]
+
+
+
+
+
+
last_sensor_start_time
+
+
+ The last time that the sensor was started (UTC).
+
+ Type: float
+
+
+
+
+
+
code_location_origin
+
+
+ The code location that the sensor is in.
+
+ Type: Optional[CodeLocationOrigin]
+
+
+
+ Updates the cursor value for this sensor, which will be provided on the context for the
+ next sensor evaluation.
+
+ This can be used to keep track of progress and avoid duplicate work across sensor
+ evaluations.
+
+ Parameters: cursor (Optional[str])
+
+
+
+
+
+
property cursor
+
+ The cursor value for this sensor, which was set in an earlier sensor evaluation.
+
+
+
+
+
property instance
+
+
+ The current DagsterInstance.
+
+ Type: [DagsterInstance](internals.mdx#dagster.DagsterInstance)
+
+
+
+
+
+
property is_first_tick_since_sensor_start
+
+ Flag representing if this is the first tick since the sensor was started.
+
+
+
+
+
property last_run_key
+
+
+ The run key supplied to the most recent RunRequest produced by this sensor.
+
+ Type: Optional[str]
+
+
+
+
+
+
property last_sensor_start_time
+
+
+ Timestamp representing the last time this sensor was started. Can be
+ used in concert with last_tick_completion_time to determine if this is the first tick since the
+ sensor was started.
+
+ Type: Optional[float]
+
+
+
+
+
+
property last_tick_completion_time
+
+
+ Timestamp representing the last time this sensor completed an evaluation.
+
+ Type: Optional[float]
+
+
+
+
+
+
property repository_def
+
+
+ The RepositoryDefinition that this sensor resides in.
+
+ Type: Optional[[RepositoryDefinition](repositories.mdx#dagster.RepositoryDefinition)]
+
+
+
+
+
+
property repository_name
+
+
+ The name of the repository that this sensor resides in.
+
+ Type: Optional[str]
+
+
+
+
+
+
property resources
+
+
+ A mapping from resource key to instantiated resources for this sensor.
+
+ Type: Resources
+
+
+
+
+
+
+
+
+
+
dagster.build_sensor_context
+
+
+ Builds sensor execution context using the provided parameters.
+
+ This function can be used to provide a context to the invocation of a sensor definition.If
+ provided, the dagster instance must be persistent; DagsterInstance.ephemeral() will result in an
+ error.
+
+ Parameters:
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The dagster instance configured to run the sensor.
+ - cursor (Optional[str]) – A cursor value to provide to the evaluation of the sensor.
+ - repository_name (Optional[str]) – The name of the repository that the sensor belongs to.
+ - repository_def (Optional[[*RepositoryDefinition*](repositories.mdx#dagster.RepositoryDefinition)]) – The repository that the sensor belongs to.
+ - resources (Optional[Mapping[str, [*ResourceDefinition*](resources.mdx#dagster.ResourceDefinition)]]) – A set of resource definitions
+ - definitions (Optional[[*Definitions*](definitions.mdx#dagster.Definitions)]) – Definitions object that the sensor is defined in.
+ - last_sensor_start_time (Optional[float]) – The last time the sensor was started.
+
+
+ Examples:
+
+ ```python
+ context = build_sensor_context()
+ my_sensor(context)
+ ```
+
+
+
+
+
+
@dagster.asset_sensor
+
+
+ Creates an asset sensor where the decorated function is used as the asset sensor’s evaluation
+ function.
+
+ If the asset has been materialized multiple times between since the last sensor tick, the
+ evaluation function will only be invoked once, with the latest materialization.
+
+ The decorated function may:
+
+ 1. Return a RunRequest object.
+ 2. Return a list of RunRequest objects.
+ 3. Return a SkipReason object, providing a descriptive message of why no runs were requested.
+ 4. Return nothing (skipping without providing a reason)
+ 5. Yield a SkipReason or yield one or more RunRequest objects.
+ Takes a `SensorEvaluationContext` and an EventLogEntry corresponding to an
+ AssetMaterialization event.
+
+ Parameters:
+ - asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – The asset_key this sensor monitors.
+ - name (Optional[str]) – The name of the sensor. Defaults to the name of the decorated
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The
+ - jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]]) – (experimental) A list of jobs to be executed when the sensor fires.
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+
+
+ Example:
+
+ ```python
+ from dagster import AssetKey, EventLogEntry, SensorEvaluationContext, asset_sensor
+
+
+ @asset_sensor(asset_key=AssetKey("my_table"), job=my_job)
+ def my_asset_sensor(context: SensorEvaluationContext, asset_event: EventLogEntry):
+ return RunRequest(
+ run_key=context.cursor,
+ run_config={
+ "ops": {
+ "read_materialization": {
+ "config": {
+ "asset_key": asset_event.dagster_event.asset_key.path,
+ }
+ }
+ }
+ },
+ )
+ ```
+
+
+
+
+
+
class dagster.AssetSensorDefinition
+
+
+ Define an asset sensor that initiates a set of runs based on the materialization of a given
+ asset.
+
+ If the asset has been materialized multiple times between since the last sensor tick, the
+ evaluation function will only be invoked once, with the latest materialization.
+
+ Parameters:
+ - name (str) – The name of the sensor to create.
+ - asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – The asset_key this sensor monitors.
+ - asset_materialization_fn (Callable[[SensorEvaluationContext, [*EventLogEntry*](internals.mdx#dagster.EventLogEntry)], Union[Iterator[Union[[*RunRequest*](#dagster.RunRequest), [*SkipReason*](#dagster.SkipReason)]], [*RunRequest*](#dagster.RunRequest), [*SkipReason*](#dagster.SkipReason)]]) –
+
+ The core
+ evaluation function for the sensor, which is run at an interval to determine whether a
+ run should be launched or not. Takes a `SensorEvaluationContext` and
+ an EventLogEntry corresponding to an AssetMaterialization event.
+
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The job
+ - jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]]) – (experimental) A list of jobs to be executed when the sensor fires.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+
+
+
+
property asset_key
+
+
+ The key of the asset targeted by this sensor.
+
+ Type: [AssetKey](assets.mdx#dagster.AssetKey)
+
+
+
+
+
+
+
+
+
+
class dagster.RunStatusSensorDefinition
+
+
+ Define a sensor that reacts to a given status of job execution, where the decorated
+ function will be evaluated when a run is at the given status.
+
+ Parameters:
+ - name (str) – The name of the sensor. Defaults to the name of the decorated function.
+ - run_status ([*DagsterRunStatus*](internals.mdx#dagster.DagsterRunStatus)) – The status of a run which will be
+ - run_status_sensor_fn (Callable[[[*RunStatusSensorContext*](#dagster.RunStatusSensorContext)], Union[[*SkipReason*](#dagster.SkipReason), DagsterRunReaction]]) – The core
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), UnresolvedAssetJobDefinition, [*JobSelector*](#dagster.JobSelector), [*RepositorySelector*](#dagster.RepositorySelector), CodeLocationSelector]]]) – The jobs in the current repository that will be monitored by this sensor. Defaults to
+ - monitor_all_code_locations (bool) – If set to True, the sensor will monitor all runs in the
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - request_job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition)]]) – The job a RunRequest should
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+ - request_jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition)]]]) – (experimental)
+
+
+
+
+
+
+
+
class dagster.RunStatusSensorContext
+
+
+ The `context` object available to a decorated function of `run_status_sensor`.
+
+
+
property dagster_event
+
+ The event associated with the job run status.
+
+
+
+
+
property dagster_run
+
+ The run of the job.
+
+
+
+
+
property instance
+
+ The current instance.
+
+
+
+
+
property log
+
+ The logger for the current sensor evaluation.
+
+
+
+
+
property partition_key
+
+
+ The partition key of the relevant run.
+
+ Type: Optional[str]
+
+
+
+
+
+
property sensor_name
+
+ The name of the sensor.
+
+
+
+
+
+
+
+
+
class dagster.RunFailureSensorContext
+
+
+ The `context` object available to a decorated function of `run_failure_sensor`.
+
+
+
+ The step failure event for each step in the run that failed.
+
+ Examples:
+
+ ```python
+ error_strings_by_step_key = {
+ # includes the stack trace
+ event.step_key: event.event_specific_data.error.to_string()
+ for event in context.get_step_failure_events()
+ }
+ ```
+
+
+
+
+
+
property failure_event
+
+
+ The run failure event.
+
+ If the run failed because of an error inside a step, get_step_failure_events will have more
+ details on the step failure.
+
+
+
+
+
+
+
+
+
+
+
class dagster.JobSelector
+
+
+
+
+
+
+
class dagster.RepositorySelector
+
+
+
+
+
+
+
dagster.build_run_status_sensor_context
+
+
+ Builds run status sensor context from provided parameters.
+
+ This function can be used to provide the context argument when directly invoking a function
+ decorated with @run_status_sensor or @run_failure_sensor, such as when writing unit tests.
+
+ Parameters:
+ - sensor_name (str) – The name of the sensor the context is being constructed for.
+ - dagster_event ([*DagsterEvent*](execution.mdx#dagster.DagsterEvent)) – A DagsterEvent with the same event type as the one that
+ - dagster_instance ([*DagsterInstance*](internals.mdx#dagster.DagsterInstance)) – The dagster instance configured for the context.
+ - dagster_run ([*DagsterRun*](internals.mdx#dagster.DagsterRun)) – DagsterRun object from running a job
+ - resources (Optional[Mapping[str, object]]) – A dictionary of resources to be made available
+ - repository_def (Optional[[*RepositoryDefinition*](repositories.mdx#dagster.RepositoryDefinition)]) – experimental
+
+
+ Examples:
+
+ ```python
+ instance = DagsterInstance.ephemeral()
+ result = my_job.execute_in_process(instance=instance)
+
+ dagster_run = result.dagster_run
+ dagster_event = result.get_job_success_event() # or get_job_failure_event()
+
+ context = build_run_status_sensor_context(
+ sensor_name="run_status_sensor_to_invoke",
+ dagster_instance=instance,
+ dagster_run=dagster_run,
+ dagster_event=dagster_event,
+ )
+ run_status_sensor_to_invoke(context)
+ ```
+
+
+
+
+
+
@dagster.run_status_sensor
+
+
+ Creates a sensor that reacts to a given status of job execution, where the decorated
+ function will be run when a job is at the given status.
+
+ Takes a [`RunStatusSensorContext`](#dagster.RunStatusSensorContext).
+
+ Parameters:
+ - run_status ([*DagsterRunStatus*](internals.mdx#dagster.DagsterRunStatus)) – The status of run execution which will be
+ - name (Optional[str]) – The name of the sensor. Defaults to the name of the decorated function.
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), UnresolvedAssetJobDefinition, [*RepositorySelector*](#dagster.RepositorySelector), [*JobSelector*](#dagster.JobSelector), CodeLocationSelector]]]) – Jobs in the current code locations that will be monitored by this sensor. Defaults to None, which means the alert will
+ - monitor_all_code_locations (Optional[bool]) – If set to True, the sensor will monitor all runs in the Dagster deployment.
+ - job_selection (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*RepositorySelector*](#dagster.RepositorySelector), [*JobSelector*](#dagster.JobSelector), CodeLocationSelector]]]) – deprecatedmonitored_jobs instead.) (deprecated in favor of monitored_jobs) Jobs in the current code location that will be
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - request_job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The job that should be
+ - request_jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]]) – (experimental)
+ - monitor_all_repositories (Optional[bool]) – deprecatedmonitor_all_code_locations instead.) (deprecated in favor of monitor_all_code_locations) If set to True, the sensor will monitor all runs in the Dagster instance.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+
+
+
+
+
+
+
+
@dagster.run_failure_sensor
+
+
+ Creates a sensor that reacts to job failure events, where the decorated function will be
+ run when a run fails.
+
+ Takes a [`RunFailureSensorContext`](#dagster.RunFailureSensorContext).
+
+ Parameters:
+ - name (Optional[str]) – The name of the job failure sensor. Defaults to the name of the
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), UnresolvedAssetJobDefinition, [*RepositorySelector*](#dagster.RepositorySelector), [*JobSelector*](#dagster.JobSelector), CodeLocationSelector]]]) – The jobs in the current repository that will be monitored by this failure sensor.
+ - monitor_all_code_locations (bool) – If set to True, the sensor will monitor all runs in the
+ - job_selection (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*RepositorySelector*](#dagster.RepositorySelector), [*JobSelector*](#dagster.JobSelector), CodeLocationSelector]]]) – deprecatedmonitored_jobs instead.) (deprecated in favor of monitored_jobs) The jobs in the current repository that will be
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - request_job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJob]]) – The job a RunRequest should
+ - request_jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJob]]]) – (experimental)
+ - monitor_all_repositories (bool) – deprecatedmonitor_all_code_locations instead.) (deprecated in favor of monitor_all_code_locations) If set to True,
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+
+
+
+
+
+
+
+
class dagster.SensorResult
+
+
+ The result of a sensor evaluation.
+
+
+
run_requests
+
+
+ A list
+ of run requests to be executed.
+
+ Type: Optional[Sequence[[RunRequest](#dagster.RunRequest)]]
+
+
+
+
+
+
skip_reason
+
+
+ A skip message indicating why sensor
+ evaluation was skipped.
+
+ Type: Optional[Union[str, [SkipReason](#dagster.SkipReason)]]
+
+
+
+
+
+
cursor
+
+
+ The cursor value for this sensor, which will be provided on the
+ context for the next sensor evaluation.
+
+ Type: Optional[str]
+
+
+ AddDynamicPartitionsRequest]]]): A list of dynamic partition requests to request dynamic
+ partition addition and deletion. Run requests will be evaluated using the state of the
+ partitions with these changes applied. We recommend limiting partition additions
+ and deletions to a maximum of 25K partitions per sensor evaluation, as this is the maximum
+ recommended partition limit per asset.
+
+
+
+
+
asset_events
+
+
+ (Experimental) A
+ list of materializations, observations, and asset check evaluations that the system
+ will persist on your behalf at the end of sensor evaluation. These events will be not
+ be associated with any particular run, but will be queryable and viewable in the asset catalog.
+
+ Type: Optional[Sequence[Union[[AssetObservation](assets.mdx#dagster.AssetObservation), [AssetMaterialization](ops.mdx#dagster.AssetMaterialization), AssetCheckEvaluation]]]
+
+
+
+
+
+
+
+
+
+
class dagster.AddDynamicPartitionsRequest
+
+ A request to add partitions to a dynamic partitions definition, to be evaluated by a sensor or schedule.
+
+
+
+
+
class dagster.DeleteDynamicPartitionsRequest
+
+ A request to delete partitions to a dynamic partitions definition, to be evaluated by a sensor or schedule.
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/types.mdx b/docs/docs-beta/docs/api/python-api/types.mdx
new file mode 100644
index 0000000000000..1a5c846eb696b
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/types.mdx
@@ -0,0 +1,374 @@
+---
+title: 'types'
+title_meta: 'types API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'types Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Types
+
+Dagster includes facilities for typing the input and output values of ops (“runtime” types).
+
+
+
+
+
+## Built-in types
+
+
+
dagster.Nothing
+
+
+ Use this type only for inputs and outputs, in order to establish an execution dependency without
+ communicating a value. Inputs of this type will not be passed to the op compute function, so
+ it is necessary to use the explicit [`In`](ops.mdx#dagster.In) API to define them rather than
+ the Python 3 type hint syntax.
+
+ All values are considered to be instances of `Nothing`.
+
+ Examples:
+
+ ```python
+ @op
+ def wait(_) -> Nothing:
+ time.sleep(1)
+ return
+
+ @op(
+ ins={"ready": In(dagster_type=Nothing)},
+ )
+ def done(_) -> str:
+ return 'done'
+
+ @job
+ def nothing_job():
+ done(wait())
+
+ # Any value will pass the type check for Nothing
+ @op
+ def wait_int(_) -> Int:
+ time.sleep(1)
+ return 1
+
+ @job
+ def nothing_int_job():
+ done(wait_int())
+ ```
+
+
+
+
+
+
+
+
+
+
+## Making New Types
+
+
+
class dagster.DagsterType
+
+
+ Define a type in dagster. These can be used in the inputs and outputs of ops.
+
+ Parameters:
+ - type_check_fn (Callable[[[*TypeCheckContext*](execution.mdx#dagster.TypeCheckContext), Any], [Union[bool, [*TypeCheck*](ops.mdx#dagster.TypeCheck)]]]) – The function that defines the type check. It takes the value flowing
+ - key (Optional[str]) –
+
+ The unique key to identify types programmatically.
+ The key property always has a value. If you omit key to the argument
+ to the init function, it instead receives the value of `name`. If
+ neither `key` nor `name` is provided, a `CheckError` is thrown.
+
+ In the case of a generic type such as `List` or `Optional`, this is
+ generated programmatically based on the type parameters.
+
+ - name (Optional[str]) – A unique name given by a user. If `key` is `None`, `key`
+ - description (Optional[str]) – A markdown-formatted string, displayed in tooling.
+ - loader (Optional[[*DagsterTypeLoader*](#dagster.DagsterTypeLoader)]) – An instance of a class that
+ - required_resource_keys (Optional[Set[str]]) – Resource keys required by the `type_check_fn`.
+ - is_builtin (bool) – Defaults to False. This is used by tools to display or
+ - kind (DagsterTypeKind) – Defaults to None. This is used to determine the kind of runtime type
+ - typing_type – Defaults to None. A valid python typing type (e.g. Optional[List[int]]) for the
+
+
+
+
type_check
+
+
+ Type check the value against the type.
+
+ Parameters:
+ - context ([*TypeCheckContext*](execution.mdx#dagster.TypeCheckContext)) – The context of the type check.
+ - value (Any) – The value to check.
+
+
+ Returns: The result of the type check.Return type: [TypeCheck](ops.mdx#dagster.TypeCheck)
+
+
+
+
+
+
property description
+
+
+ Description of the type, or None if not provided.
+
+ Type: Optional[str]
+
+
+
+
+
+
property display_name
+
+ Either the name or key (if name is None) of the type, overridden in many subclasses.
+
+
+
+
+
property has_unique_name
+
+
+ Whether the type has a unique name.
+
+ Type: bool
+
+
+
+
+
+
property loader
+
+
+ Loader for this type, if any.
+
+ Type: Optional[[DagsterTypeLoader](#dagster.DagsterTypeLoader)]
+
+
+
+
+
+
property required_resource_keys
+
+
+ Set of resource keys required by the type check function.
+
+ Type: AbstractSet[str]
+
+
+
+
+
+
property typing_type
+
+
+ The python typing type for this type.
+
+ Type: Any
+
+
+
+
+
+
property unique_name
+
+ The unique name of this type. Can be None if the type is not unique, such as container types.
+
+
+
+
+
+
+
+
+
dagster.PythonObjectDagsterType
+
+
+ Define a type in dagster whose typecheck is an isinstance check.
+
+ Specifically, the type can either be a single python type (e.g. int),
+ or a tuple of types (e.g. (int, float)) which is treated as a union.
+
+ Examples:
+
+ ```python
+ ntype = PythonObjectDagsterType(python_type=int)
+ assert ntype.name == 'int'
+ assert_success(ntype, 1)
+ assert_failure(ntype, 'a')
+ ```
+ ```python
+ ntype = PythonObjectDagsterType(python_type=(int, float))
+ assert ntype.name == 'Union[int, float]'
+ assert_success(ntype, 1)
+ assert_success(ntype, 1.5)
+ assert_failure(ntype, 'a')
+ ```
+ Parameters:
+ - python_type (Union[Type, Tuple[Type, ...]) – The dagster typecheck function calls instanceof on
+ - name (Optional[str]) – Name the type. Defaults to the name of `python_type`.
+ - key (Optional[str]) – Key of the type. Defaults to name.
+ - description (Optional[str]) – A markdown-formatted string, displayed in tooling.
+ - loader (Optional[[*DagsterTypeLoader*](#dagster.DagsterTypeLoader)]) – An instance of a class that
+
+
+
+
+
+
+
+
dagster.dagster_type_loader
+
+
+ Create an dagster type loader that maps config data to a runtime value.
+
+ The decorated function should take the execution context and parsed config value and return the
+ appropriate runtime value.
+
+ Parameters: config_schema ([*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The schema for the config that’s passed to the decorated
+ function.
+ Examples:
+
+ ```python
+ @dagster_type_loader(Permissive())
+ def load_dict(_context, value):
+ return value
+ ```
+
+
+
+
+
+
class dagster.DagsterTypeLoader
+
+
+ Dagster type loaders are used to load unconnected inputs of the dagster type they are attached
+ to.
+
+ The recommended way to define a type loader is with the
+ [`@dagster_type_loader`](#dagster.dagster_type_loader) decorator.
+
+
+
+
+
+
+
class dagster.DagsterTypeLoaderContext
+
+
+ The context object provided to a [`@dagster_type_loader`](#dagster.dagster_type_loader)-decorated function during execution.
+
+ Users should not construct this object directly.
+
+
+
property job_def
+
+ The underlying job definition being executed.
+
+
+
+
+
property op_def
+
+ The op for which type loading is occurring.
+
+
+
+
+
property resources
+
+ The resources available to the type loader, specified by the required_resource_keys argument of the decorator.
+
+
+
+
+
+
+
+
+
dagster.usable_as_dagster_type
+
+
+ Decorate a Python class to make it usable as a Dagster Type.
+
+ This is intended to make it straightforward to annotate existing business logic classes to
+ make them dagster types whose typecheck is an isinstance check against that python class.
+
+ Parameters:
+ - python_type (cls) – The python type to make usable as python type.
+ - name (Optional[str]) – Name of the new Dagster type. If `None`, the name (`__name__`) of
+ - description (Optional[str]) – A user-readable description of the type.
+ - loader (Optional[[*DagsterTypeLoader*](#dagster.DagsterTypeLoader)]) – An instance of a class that
+
+
+ Examples:
+
+ ```python
+ # dagster_aws.s3.file_manager.S3FileHandle
+ @usable_as_dagster_type
+ class S3FileHandle(FileHandle):
+ def __init__(self, s3_bucket, s3_key):
+ self._s3_bucket = check.str_param(s3_bucket, 's3_bucket')
+ self._s3_key = check.str_param(s3_key, 's3_key')
+
+ @property
+ def s3_bucket(self):
+ return self._s3_bucket
+
+ @property
+ def s3_key(self):
+ return self._s3_key
+
+ @property
+ def path_desc(self):
+ return self.s3_path
+
+ @property
+ def s3_path(self):
+ return 's3://{bucket}/{key}'.format(bucket=self.s3_bucket, key=self.s3_key)
+ ```
+
+
+
+
+
+
dagster.make_python_type_usable_as_dagster_type
+
+ Take any existing python type and map it to a dagster type (generally created with
+ [`DagsterType`](#dagster.DagsterType)) This can only be called once
+ on a given python type.
+
+
+
+
+
+### Testing Types
+
+
+
dagster.check_dagster_type
+
+
+ Test a custom Dagster type.
+
+ Parameters:
+ - dagster_type (Any) – The Dagster type to test. Should be one of the
+ - value (Any) – The runtime value to test.
+
+
+ Returns: The result of the type check.Return type: [TypeCheck](ops.mdx#dagster.TypeCheck)
+ Examples:
+
+ ```python
+ assert check_dagster_type(Dict[Any, Any], {'foo': 'bar'}).success
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/docs/api/python-api/utilities.mdx b/docs/docs-beta/docs/api/python-api/utilities.mdx
new file mode 100644
index 0000000000000..a6e620570530b
--- /dev/null
+++ b/docs/docs-beta/docs/api/python-api/utilities.mdx
@@ -0,0 +1,223 @@
+---
+title: 'utilities'
+title_meta: 'utilities API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'utilities Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Utilities
+
+
+
dagster.file_relative_path
+
+
+ Get a path relative to the currently executing Python file.
+
+ This function is useful when one needs to load a file that is relative to the position of
+ the current file. (Such as when you encode a configuration file path in source file and want
+ in runnable in any current working directory)
+
+ Parameters:
+ - dunderfile (str) – Should always be `__file__`.
+ - relative_path (str) – Path to get relative to the currently executing file.
+
+
+ Examples:
+
+ ```python
+ file_relative_path(__file__, 'path/relative/to/file')
+ ```
+
+
+
+
+
+
dagster.config_from_files
+
+
+ Constructs run config from YAML files.
+
+ Parameters: config_files (List[str]) – List of paths or glob patterns for yaml files
+ to load and parse as the run config.Returns: A run config dictionary constructed from provided YAML files.Return type: Dict[str, Any]Raises:
+ - FileNotFoundError – When a config file produces no results
+ - [DagsterInvariantViolationError](errors.mdx#dagster.DagsterInvariantViolationError)DagsterInvariantViolationError – When one of the YAML files is invalid and has a parse
+
+
+
+
+
+
+
+
dagster.config_from_pkg_resources
+
+
+ Load a run config from a package resource, using `pkg_resources.resource_string()`.
+
+ Example:
+
+ ```python
+ config_from_pkg_resources(
+ pkg_resource_defs=[
+ ('dagster_examples.airline_demo.environments', 'local_base.yaml'),
+ ('dagster_examples.airline_demo.environments', 'local_warehouse.yaml'),
+ ],
+ )
+ ```
+ Parameters: pkg_resource_defs (List[(str, str)]) – List of pkg_resource modules/files to
+ load as the run config.Returns: A run config dictionary constructed from the provided yaml stringsReturn type: Dict[Str, Any]Raises: [DagsterInvariantViolationError](errors.mdx#dagster.DagsterInvariantViolationError)DagsterInvariantViolationError – When one of the YAML documents is invalid and has a
+ parse error.
+
+
+
+
+
+
dagster.config_from_yaml_strings
+
+
+ Static constructor for run configs from YAML strings.
+
+ Parameters: yaml_strings (List[str]) – List of yaml strings to parse as the run config.Returns: A run config dictionary constructed from the provided yaml stringsReturn type: Dict[Str, Any]Raises: [DagsterInvariantViolationError](errors.mdx#dagster.DagsterInvariantViolationError)DagsterInvariantViolationError – When one of the YAML documents is invalid and has a
+ parse error.
+
+
+
+
+
+
dagster.get_dagster_logger
+
+
+ Creates a python logger whose output messages will be captured and converted into Dagster log
+ messages. This means they will have structured information such as the step_key, run_id, etc.
+ embedded into them, and will show up in the Dagster event log.
+
+ This can be used as a more convenient alternative to context.log in most cases. If log level
+ is not set explicitly, defaults to DEBUG.
+
+ Parameters: name (Optional[str]) – If supplied, will create a logger with the name “dagster.builtin.\{name}”,
+ with properties inherited from the base Dagster logger. If omitted, the returned logger
+ will be named “dagster.builtin”.Returns: A logger whose output will be captured by Dagster.Return type: `logging.Logger`
+ Example:
+
+ ```python
+ from dagster import get_dagster_logger, op
+
+ @op
+ def hello_op():
+ log = get_dagster_logger()
+ for i in range(5):
+ # do something
+ log.info(f"Did {i+1} things!")
+ ```
+
+
+
+
+
+
class dagster.ExperimentalWarning
+
+
+
+
+
+
+
class dagster._utils.warnings.PreviewWarning
+
+
+
+
+
+
+
class dagster._utils.warnings.BetaWarning
+
+
+
+
+
+
+
dagster.make_email_on_run_failure_sensor
+
+
+ Create a job failure sensor that sends email via the SMTP protocol.
+
+ Parameters:
+ - email_from (str) – The sender email address to send the message from.
+ - email_password (str) – The password of the sender.
+ - email_to (List[str]) – The receipt email addresses to send the message to.
+ - email_body_fn (Optional(Callable[[[*RunFailureSensorContext*](schedules-sensors.mdx#dagster.RunFailureSensorContext)], str])) – Function which
+ - email_subject_fn (Optional(Callable[[[*RunFailureSensorContext*](schedules-sensors.mdx#dagster.RunFailureSensorContext)], str])) – Function which
+ - smtp_host (str) – The hostname of the SMTP server. Defaults to “smtp.gmail.com”.
+ - smtp_type (str) – The protocol; either “SSL” or “STARTTLS”. Defaults to SSL.
+ - smtp_port (Optional[int]) – The SMTP port. Defaults to 465 for SSL, 587 for STARTTLS.
+ - smtp_user (Optional[str]) – The SMTP user for authenticatication in the SMTP server. Defaults to the value of email_from.
+ - name – (Optional[str]): The name of the sensor. Defaults to “email_on_job_failure”.
+ - webserver_base_url – (Optional[str]): The base url of your dagster-webserver instance. Specify this to allow
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*RepositorySelector*](schedules-sensors.mdx#dagster.RepositorySelector), [*JobSelector*](schedules-sensors.mdx#dagster.JobSelector)]]]) – The jobs that will be monitored by this failure sensor. Defaults to None, which means the alert will
+ - monitor_all_code_locations (bool) – If set to True, the sensor will monitor all runs in the
+ - job_selection (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*RepositorySelector*](schedules-sensors.mdx#dagster.RepositorySelector), [*JobSelector*](schedules-sensors.mdx#dagster.JobSelector)]]]) – deprecatedmonitored_jobs instead.) (deprecated in favor of monitored_jobs) The jobs that will be monitored by this failure
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - monitor_all_repositories (bool) – deprecatedmonitor_all_code_locations instead.) If set to True, the sensor will monitor all runs in the
+
+
+ Examples:
+
+ ```python
+ email_on_run_failure = make_email_on_run_failure_sensor(
+ email_from="no-reply@example.com",
+ email_password=os.getenv("ALERT_EMAIL_PASSWORD"),
+ email_to=["xxx@example.com"],
+ )
+
+ @repository
+ def my_repo():
+ return [my_job + email_on_run_failure]
+ ```
+ ```python
+ def my_message_fn(context: RunFailureSensorContext) -> str:
+ return (
+ f"Job {context.dagster_run.job_name} failed!"
+ f"Error: {context.failure_event.message}"
+ )
+
+ email_on_run_failure = make_email_on_run_failure_sensor(
+ email_from="no-reply@example.com",
+ email_password=os.getenv("ALERT_EMAIL_PASSWORD"),
+ email_to=["xxx@example.com"],
+ email_body_fn=my_message_fn,
+ email_subject_fn=lambda _: "Dagster Alert",
+ webserver_base_url="http://mycoolsite.com",
+ )
+ ```
+
+
+
+
+
+
class dagster._utils.forked_pdb.ForkedPdb
+
+
+ A pdb subclass that may be used from a forked multiprocessing child.
+
+ Examples:
+
+ ```python
+ from dagster._utils.forked_pdb import ForkedPdb
+
+ @solid
+ def complex_solid(_):
+ # some complicated stuff
+
+ ForkedPdb().set_trace()
+
+ # some other complicated stuff
+ ```
+ You can initiate pipeline execution via the webserver and use the pdb debugger to examine/step through
+ execution at the breakpoint.
+
+
+
+
+
+
diff --git a/docs/docs-beta/docusaurus.config.ts b/docs/docs-beta/docusaurus.config.ts
index 64123c4deb1c2..177fcb835cbf3 100644
--- a/docs/docs-beta/docusaurus.config.ts
+++ b/docs/docs-beta/docusaurus.config.ts
@@ -101,23 +101,23 @@ const config: Config = {
docId: 'api/index',
position: 'left',
},
- {
- label: 'Dagster University',
- href: 'https://courses.dagster.io',
- position: 'left',
- },
//{
// label: 'Changelog',
// type: 'doc',
// docId: 'changelog',
// position: 'right',
//},
- // {
- // label: 'Feedback',
- // href: 'https://github.com/dagster-io/dagster/discussions/24816',
- // position: 'right',
- // className: 'feedback-nav-link',
- // },
+ {
+ label: 'Versions',
+ type: 'docsVersionDropdown',
+ position: 'right'
+ },
+ {
+ label: 'Feedback',
+ href: 'https://github.com/dagster-io/dagster/discussions/27332',
+ position: 'right',
+ className: 'feedback-nav-link',
+ },
],
},
image: 'images/og.png',
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/about/changelog.md b/docs/docs-beta/versioned_docs/version-1.9.10/about/changelog.md
new file mode 100644
index 0000000000000..b25c26a67d5d6
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/about/changelog.md
@@ -0,0 +1,11 @@
+---
+title: 'Changelog'
+hide_title: true
+toc_max_heading_level: 2
+---
+
+import Changes, {toc as ChangesTOC} from "@site/../../CHANGES.md"
+
+
+
+export const toc = ChangesTOC;
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/about/community.md b/docs/docs-beta/versioned_docs/version-1.9.10/about/community.md
new file mode 100644
index 0000000000000..830fd0f01dc44
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/about/community.md
@@ -0,0 +1,90 @@
+---
+title: 'Community'
+sidebar_position: 10
+---
+
+# Community
+
+No ecosystem is more heterogeneous than data processing. We enthusiastically welcome community contributions to code and to docs, issue, and bug reports on [GitHub](https://www.github.com/dagster-io/dagster/), and questions, feature requests, and discussion in our [Slack](https://dagster.io/slack).
+
+We're committed to making the Dagster project inclusive and welcoming. Working on this project should feel friendly, collegial, and relaxed for everyone.
+
+## Open source commitment
+
+:::tip
+Interested in hacking on Dagster? Check out the [Contributing guide](/about/contributing) to get started!
+:::
+
+Dagster itself, defined as the code and intellectual property in the [Dagster public repository](https://www.github.com/dagster-io/dagster/), will forever and always be free. The Dagster core team also builds and operates the commercial product [Dagster+](https://dagster.io/plus) as a distinct and well-defined layer on top of Dagster. We believe that this structure will be the most sustainable for serving our users as the Dagster team and product grows.
+
+## Code of conduct
+
+### Our pledge
+
+As members, contributors, and core team members, we pledge to make participation in our community a harassment-free experience for everyone, regardless of age, body size, visible, or invisible disability, ethnicity, sex characteristics, gender identity and expression, level of experience, education, socio-economic status, nationality, personal appearance, race, caste, color, religion, or sexual identity and orientation.
+
+We pledge to act and interact in ways that contribute to an open, welcoming, diverse, inclusive, and healthy community.
+
+### Our standards
+
+Examples of behavior that contribute to a positive environment for our community include:
+
+- Demonstrating empathy and kindness toward other people
+- Being respectful of differing opinions, viewpoints, and experiences
+- Giving and gracefully accepting constructive feedback
+- Accepting responsibility, apologizing to those affected by our mistakes, and learning from the experience
+- Focusing on what's best not just for us as individuals, but for the overall community
+
+Examples of unacceptable behavior include:
+
+- The use of sexualized language or imagery, and sexual attention or advances of any kind
+- Trolling, insulting or derogatory comments, and personal or political attacks
+- Public or private harassment
+- Publishing others' private information, such as a physical or email address, without their explicit permission
+- Other conduct which could reasonably be considered inappropriate in a professional setting
+
+### Slack content guidelines
+
+The Dagster core team will do its best to respond to every message but can't guarantee a response to everyone. Don't treat the community as your own personal customer support service.
+
+Community members are expected to follow the guidelines for discussion in the Dagster community. Make an effort to contribute quality content so that our community can spend more time hanging out and talking about issues rather than cleaning and filtering our communication channels.
+
+- Start discussions in the [right channels](https://app.slack.com/client/TCDGQDUKF/browse-channels)
+- Craft questions with sufficient detail to reproduce issues or address any concerns
+- Use threads for discussions
+- Create new discussions, rather than diverting discussions to new directions midstream
+- Don't demand attention by cross-posting discussions, sending messages outside threads to bump conversations, or explicitly mentioning `@channel`, `@here`, or community members
+- Don't solicit community members about your products and services
+
+### Scope
+
+This Code of Conduct applies within all community spaces, and also applies when an individual is officially representing the community in public spaces. Examples of representing our community include using an official email address, posting on an official social media account, or acting as an appointed representative at an online or offline event.
+
+### Enforcement
+
+#### Responsibilities
+
+The Dagster core team is responsible for clarifying and enforcing our standards of acceptable behavior and will take appropriate and fair corrective action in response to any behavior that they deem inappropriate, threatening, offensive, or harmful. For the full list of core team members, refer to [the Dagster Labs team page](https://dagster.io/about).
+
+The Dagster core team has the right and responsibility to remove, edit, or reject comments, commits, code, wiki edits, issues, and other contributions that aren't aligned to this Code of Conduct, and will communicate reasons for moderation decisions when appropriate.
+
+#### Reporting
+
+Instances of abusive, harassing, or otherwise unacceptable behavior may be reported to the Dagster core team responsible for enforcement at [conduct@dagsterlabs.com](mailto:conduct@dagsterlabs.com). All complaints will be reviewed and investigated promptly and fairly. The Dagster core team is obligated to respect the privacy and security of the reporter of any incident.
+
+#### Guidelines
+
+The Dagster core team will follow these Community Impact Guidelines in determining the consequences for any action they deem in violation of this Code of Conduct:
+
+| Level | Community impact | Consequence |
+|---|----|----|
+| Reminder | Use of inappropriate language or other behavior deemed unprofessional or unwelcome in the community. | A private, written reminder from the Dagster core team, providing clarity around the nature of the violation and an explanation of why the behavior was inappropriate. |
+| Warning | A violation through a single incident or series of actions. | A warning with consequences for continued behavior. No interaction with the people involved, including unsolicited interaction with those enforcing the Code of Conduct, for a specified period of time. This includes avoiding interactions in community spaces and external channels like social media. Violating these terms will lead to a permanent ban. |
+| Permanent ban | Demonstrating a pattern of violation of community standards, including sustained inappropriate behavior, harassment of an individual, or aggression toward or disparagement of classes of individuals. | A permanent ban from any sort of public interaction within the community. |
+
+### Attribution
+
+- This Code of Conduct is adapted from the [Contributor Covenant](https://www.contributor-covenant.org/), version 2.0, available at: [Contributor Covenant Code of Conduct](https://www.contributor-covenant.org/version/2/0/code_of_conduct.html).
+- Community Impact Guidelines were inspired by [Mozilla's code of conduct enforcement ladder](https://github.com/mozilla/diversity).
+- Content Guidelines were inspired by [Discourse's FAQ for public discussion](https://meta.discourse.org/faq) and [dbt's Rules of the Road](https://docs.getdbt.com/docs/contributing/slack-rules-of-the-road).
+- For answers to common questions about this code of conduct, see the [Contributor Covenant FAQ](https://www.contributor-covenant.org/faq), or the [Contributor Covenant Translations](https://www.contributor-covenant.org/translations).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/about/contributing.md b/docs/docs-beta/versioned_docs/version-1.9.10/about/contributing.md
new file mode 100644
index 0000000000000..32d0632a4618f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/about/contributing.md
@@ -0,0 +1,140 @@
+---
+title: Contributing
+sidebar_position: 20
+---
+
+We love to see our community members get involved! If you are planning to contribute to Dagster, you will first need to set up a local development environment.
+
+## Environment setup
+
+1. [Install uv](https://docs.astral.sh/uv/getting-started/installation/). On macOS, you can use `curl` to download the script and execute it with `sh`:
+
+ ```bash
+ curl -LsSf https://astral.sh/uv/install.sh | sh
+ ```
+
+2. Create and activate a virtual environment using uv with a Python version that Dagster supports:
+
+ ```bash
+ uv venv --python 3.12
+ source .venv/bin/activate
+ ```
+
+ Dagster supports Python 3.9 through 3.12.
+
+3. Ensure that you have node installed by running `node -v`, and that you have [yarn](https://yarnpkg.com/lang/en/) installed. If you are on macOS, you can install yarn with Homebrew:
+
+ ```bash
+ brew install yarn
+ ```
+
+4. Clone the Dagster repository to the destination of your choice:
+
+ ```bash
+ git clone git@github.com:dagster-io/dagster.git
+ ```
+
+5. Run `make dev_install` at the root of the repository. This sets up a full Dagster developer environment with all modules and runs tests that do not require heavy external dependencies such as docker. This will take a few minutes. Note that certain sections of the makefile (`sanity_check`, which is part of `rebuild_ui`) require POSIX-compliant shells and will fail on CMD and powershell—if developing on windows, using something like WSL or git-bash is recommended.
+
+ ```bash
+ make dev_install
+ ```
+
+ **Note for Macs with an Apple silicon chip**: Some users have reported installation problems due to missing wheels for arm64 Macs when installing the `grpcio` package. To install the `dagster` development environment using our pre-built wheel of the `grpcio` package for M1, M2, and M3 machines, run `make dev_install_m1_grpcio_wheel` instead of `make dev_install`.
+
+6. Run some tests manually to make sure things are working:
+
+ ```bash
+ python -m pytest python_modules/dagster/dagster_tests
+ ```
+
+## Developing Dagster
+
+Some notes on developing in Dagster:
+
+- **Ruff/Pyright**: We use [ruff](https://github.com/charliermarsh/ruff) for formatting, linting and import sorting, and [pyright](https://github.com/microsoft/pyright) for static type-checking. We test these in our CI/CD pipeline.
+ - Run `make ruff` from the repo root to format, sort imports, and autofix some lint errors. It will also print out errors that need to be manually fixed.
+ - Run `make pyright` from the repo root to analyze the whole repo for type-correctness. Note that the first time you run this, it will take several minutes because a new virtualenv will be constructed.
+- **Line Width**: We use a line width of 100.
+- **IDE**: We recommend setting up your IDE to format and check with ruff on save, but you can always run `make ruff` in the root Dagster directory before submitting a pull request. If you're also using VS Code, you can see what we're using for our `settings.json` [here](https://gist.github.com/natekupp/7a17a9df8d2064e5389cc84aa118a896).
+- **Docker**: Some tests require [Docker Desktop](https://www.docker.com/products/docker-desktop) to be able to run them locally.
+
+## Developing the Dagster webserver/UI
+
+For development, run an instance of the webserver providing GraphQL service on a different port than the webapp, with any pipeline. For example:
+
+```bash
+cd dagster/examples/docs_snippets/docs_snippets/intro_tutorial/basics/connecting_ops/
+dagster-webserver -p 3333 -f complex_job.py
+```
+
+Keep this running. Then, in another terminal, run the local development (autoreloading, etc.) version of the webapp:
+
+```bash
+cd dagster/js_modules/dagster-ui
+make dev_webapp
+```
+
+During development, you might find these commands useful. Run them from `dagster/js_modules/dagster-ui`:
+
+- `yarn ts`: Typescript typechecking
+- `yarn lint`: Linting with autofix
+- `yarn jest`: An interactive Jest test runner that runs only affected tests by default
+
+To run all of them together, run `yarn test`.
+
+## Developing Docs
+
+The [docs style checklist](https://github.com/dagster-io/dagster/blob/master/docs/DOC_CHECKLIST.md) includes a set of style guidelines to adhere to when adding or modifying docs.
+
+To run the Dagster documentation website locally, run the following commands:
+
+```bash
+cd docs
+make next-watch-build # Serves the docs website on http://localhost:3001
+```
+
+Troubleshooting tip: You may need to run `make next-dev-install` first to install dependencies. Also make sure that your Node version is >=12.13.0.
+
+The API documentation is generated from ReStructured Text files (`.rst`), which extracts Python docstrings from the library files. The `.rst` files can be found in the `docs/sphinx/sections/api/apidocs` directory.
+
+If you change any `.rst` files, be sure to run the following command in the `docs` directory:
+
+```bash
+make apidoc-build
+```
+
+The majority of our code snippets are pulled from real Python files. This allows us to test our code snippets and ensure they remain up-to-date.
+
+In `.mdx` files, you'll see a code snippet with `python file=/path/to/file.py startafter=start_marker endbefore=end_marker` at the beginning of the block. For example:
+
+![Code snippet](/images/about/community/md-code-block.png)
+
+You can find the corresponding Python file at `dagster/examples/docs_snippets/docs_snippets/concepts/asset/asset_dependency.py`. The code included in each snippet is the code in the file between the `# start_marker` and `# end_marker` comments.
+
+![Code snippet between marker comments](/images/about/community/py-code-block.png)
+
+To change the code snippet, update the `.py` file, then run the following from the `docs` directory:
+
+```bash
+make mdx-format
+```
+
+You can find more information about developing documentation in `docs/README.md`.
+
+## Picking a GitHub Issue
+
+We encourage you to start with an issue labeled with the tag [`good first issue`](https://github.com/dagster-io/dagster/issues?q=is%3Aopen+is%3Aissue+label%3A%22type%3A+good+first+issue%22) on the [Github issue board](https://github.com/dagster-io/dagster/issues), to get familiar with our codebase as a first-time contributor.
+
+When you are ready for more of a challenge, you can tackle issues with the [most 👍 reactions](https://github.com/dagster-io/dagster/issues?q=is%3Aissue+is%3Aopen+sort%3Areactions-%2B1-desc). We factor engagement into prioritization of the issues. You can also explore other labels and pick any issue based on your interest.
+
+## Submit Your Code
+
+To submit your code, [fork the Dagster repository](https://help.github.com/en/articles/fork-a-repo), create a [new branch](https://help.github.com/en/desktop/contributing-to-projects/creating-a-branch-for-your-work) on your fork, and open [a Pull Request (PR)](https://help.github.com/en/articles/creating-a-pull-request-from-a-fork) once your work is ready for review.
+
+In the PR template, please describe the change, including the motivation/context, test coverage, and any other relevant information. Please note if the PR is a breaking change or if it is related to an open GitHub issue.
+
+A Core reviewer will review your PR in around one business day and provide feedback on any changes it requires to be approved. Once approved and all the tests (including Buildkite!) pass, the reviewer will click the Squash and merge button in GitHub 🥳.
+
+Your PR is now merged into Dagster! We’ll shout out your contribution in the weekly release notes.
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/about/releases.md b/docs/docs-beta/versioned_docs/version-1.9.10/about/releases.md
new file mode 100644
index 0000000000000..713fec9e4d49d
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/about/releases.md
@@ -0,0 +1,55 @@
+---
+title: 'Releases'
+sidebar_position: 30
+---
+
+# Releases and compatibility
+
+We follow [semantic versioning](https://semver.org/) for compatibility between Dagster releases.
+
+## Dagster core
+
+Dagster's public, stable APIs won't break within any major release. For example, if a public, stable API exists in Dagster 1.x.y, upgrading to 1.(x+1).y or 1.x.(y+1) shouldn't result in broken code.
+
+:::tip
+If a version breaks your code, help us out by filing an issue on [GitHub](https://github.com/dagster-io/dagster/issues).
+:::
+
+Our public, stable Python API includes:
+
+- All classes, functions, and variables that are exported at the top-level of the `dagster` package, unless they're marked [experimental](#experimental-apis).
+- Public, non-[experimental](#experimental-apis) methods and properties of public, stable classes. Public methods and properties are those included in the [API reference](/api). Within the codebase, they're marked with a `@public` decorator.
+
+### Experimental APIs
+
+The `Experimental` marker allows us to offer new APIs to users and rapidly iterate based on their feedback. Experimental APIs are marked as such in the [API reference](/api) and usually raise an `ExperimentalWarning` when used.
+
+Experimental APIs may change or disappear within any release, but we try to avoid breaking them within minor releases if they have been around for a long time.
+
+### Deprecated APIs
+
+The `Deprecated` marker indicates that we recommend avoiding an API, usually because there's a preferred option that should be used instead.
+
+Like non-deprecated public stable APIs, deprecated public stable APIs won't break within any major release after 1.0.
+
+## Dagster integration libraries
+
+Dagster's integration libraries haven't yet achieved the same API maturity as Dagster core. For this reason, integration libraries remain on a pre-1.0 versioning track (in general 0.y.z of [semantic versioning](https://semver.org/) and 0.16+ as of Dagster 1.0.0) for the time being. However, 0.16+ library releases remain fully compatible with Dagster 1.x. We will graduate integration libraries one-by-one to the 1.x versioning track as they achieve API maturity.
+
+While technically the 0.y.z phase of semantic versioning is "anything goes", we're conservative about making changes and will provide guidance about when to expect breaking changes:
+
+- Upgrading to a new dot version within a minor release, such as 0.8.1 to 0.8.2, should never result in broken code. An exception to this guarantee is [experimental APIs](#experimental-apis).
+- As often as possible, deprecation warnings will precede removals. For example, if the current version is 0.8.5 and we want to remove an API, we'll issue a deprecation [warning](https://docs.python.org/3/library/warnings.html) when the API is used and remove it from 0.9.0.
+- Upgrading to a new minor version, such as 0.7.5 to 0.8.0, may result in breakages or new deprecation [warnings](https://docs.python.org/3/library/warnings.html).
+
+## Python version support
+
+Each Dagster release strives to support the currently active versions of Python.
+
+When a new version of Python is released, Dagster will work to add support once Dagster's own core dependencies have been updated to support it. **Note**: Some external libraries may not always be compatible with the latest version of Python.
+
+When a version of Python reaches end of life, Dagster will drop support for it at the next convenient non-patch release.
+
+## Changelog
+
+The best way to stay on top of what changes are included in each release is through the [Dagster repository's changelog](https://github.com/dagster-io/dagster/blob/master/CHANGES.md). We call out breaking changes and deprecations in the **Breaking Changes** and **Deprecations** sections.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/about/telemetry.md b/docs/docs-beta/versioned_docs/version-1.9.10/about/telemetry.md
new file mode 100644
index 0000000000000..0a32051d6ef43
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/about/telemetry.md
@@ -0,0 +1,21 @@
+---
+title: "Dagster telemetry"
+sidebar_position: 40
+---
+
+As open source project maintainers, we collect usage statistics to better understand how users engage with Dagster and to inform development priorities. Telemetry data motivates projects such as adding functionality in frequently used parts of the product, and helps us understand adoption of new features.
+
+We collect telemetry from both the frontend and backend. We do not collect any data processed by Dagster pipelines, and we do not collect any identifiable information about your Dagster definitions, including the names of your assets, ops, or jobs.
+
+Front end telemetry is collected from a JavaScript bundle hosted unminified at `https://dagster.io/oss-telemetry.js`. This bundle may change over time.
+
+Backend telemetry collection is logged at `$DAGSTER_HOME/logs/` if `$DAGSTER_HOME` is set or `~/.dagster/logs/` if not set.
+
+Use of telemetry data is governed by the [Dagster Privacy Policy](https://dagster.io/privacy).
+
+If you’d like to opt out, you can add the following to `$DAGSTER_HOME/dagster.yaml` (creating that file if necessary):
+
+```yaml
+telemetry:
+ enabled: false
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/api-lifecycle.md b/docs/docs-beta/versioned_docs/version-1.9.10/api/api-lifecycle.md
new file mode 100644
index 0000000000000..cbd1054427986
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/api-lifecycle.md
@@ -0,0 +1,52 @@
+---
+title: API lifecycle stages
+---
+
+This reference guide outlines the different stages in the lifecycle of Dagster APIs, from preview to deprecation. Understanding these stages helps developers make informed decisions about which APIs to use in their projects.
+
+Dagster's API lifecycle is designed to balance innovation with stability, ensuring that developers can rely on consistent behavior while also benefiting from new features and improvements. This approach allows Dagster to evolve and adapt to changing requirements in the data engineering landscape while maintaining a stable foundation for existing projects.
+
+The lifecycle stages described below provide a clear framework for understanding the maturity and reliability of different Dagster APIs. This transparency helps developers make informed decisions about which APIs to adopt in their projects, based on their specific needs for stability, feature completeness, and long-term support.
+
+## API lifecycle stages
+
+| Stage | Description | Lifetime |
+|---------|-----------------------------------------------------------------------------------------------------------------------|----------------------------------------------------------------------------------------|
+| Preview | This API may have breaking changes in patch version releases. This feature is not considered ready for production use. | Until design is complete, or implementation cancelled |
+| Beta | This API may have breaking changes in minor version releases, with behavior changes in patch releases. | At most, two 1.x releases before it is either considered stable or returned to preview |
+| Generally Available (GA)| Ready for production use, with minimal risk of breaking changes. | Supported until at least 2.0
+| Superseded | This API is still available, but is no longer the best practice. A better alternative is available. | Supported until at least 2.0
+| Deprecated | API is still available but will be removed in the future; avoid new usage. | Will be removed in a minor release, the DeprecationWarning will indicate the next release that will remove the API.
+
+
+## Understanding the stages
+
+### Preview
+- **Purpose**: For early testing and feedback
+- **Stability**: Highly unstable, expect frequent changes
+- **Usage**: Not recommended for production environments
+- **Documentation**: Minimal, typically just a README or unlisted documentation
+
+### Beta
+- **Purpose**: Feature testing with wider audience
+- **Stability**: More stable than Preview, but still subject to changes
+- **Usage**: Can be used in non-critical production environments
+- **Documentation**: How-to guides and API documentation available
+
+### GA (General Availability)
+- **Purpose**: Production-ready features
+- **Stability**: Stable with minimal risk of breaking changes
+- **Usage**: Recommended for all production environments
+- **Documentation**: Comprehensive documentation available
+
+### Superseded
+- **Purpose**: Maintains backwards compatibility while promoting newer alternatives
+- **Stability**: Stable but no longer recommended
+- **Usage**: Existing implementations can continue, but new projects should use the recommended alternative
+- **Documentation**: API docs remain, but usage is discouraged in favor of newer alternatives
+
+### Deprecated
+- **Purpose**: Signals upcoming removal of the API
+- **Stability**: Stable but scheduled for removal
+- **Usage**: Existing implementations should plan migration
+- **Documentation**: API docs remain, with clear warnings about deprecation. Arguments may be removed from function signature
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/index.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/index.mdx
new file mode 100644
index 0000000000000..613bd51f10c2b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/index.mdx
@@ -0,0 +1,7 @@
+---
+title: API reference
+---
+
+These docs aim to cover the entire public surface of the core dagster APIs, as well as public APIs from all provided libraries.
+
+Dagster follows [semantic versioning](https://semver.org/). We attempt to isolate breaking changes to the public APIs to minor versions on a roughly 12-week cadence, and will announce deprecations in Slack and in the release notes to patch versions on a roughly weekly cadence.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/asset-checks.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/asset-checks.mdx
new file mode 100644
index 0000000000000..c2f34459a33b6
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/asset-checks.mdx
@@ -0,0 +1,603 @@
+---
+title: 'asset checks'
+title_meta: 'asset checks API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'asset checks Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Asset Checks
+
+Dagster allows you to define and execute checks on your software-defined assets. Each asset check verifies some property of a data asset, e.g. that is has no null values in a particular column.
+
+
+
@dagster.asset_check
+
+
+ Create a definition for how to execute an asset check.
+
+ Parameters:
+ - asset (Union[[*AssetKey*](assets.mdx#dagster.AssetKey), Sequence[str], str, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]) – The
+ - name (Optional[str]) – The name of the check. If not specified, the name of the decorated
+ - description (Optional[str]) – The description of the check.
+ - blocking (bool) – When enabled, runs that include this check and any downstream assets that
+ - additional_ins (Optional[Mapping[str, [*AssetIn*](assets.mdx#dagster.AssetIn)]]) – A mapping from input name to
+ - additional_deps (Optional[Iterable[CoercibleToAssetDep]]) – Assets that are upstream
+ - required_resource_keys (Optional[Set[str]]) – A set of keys for resources that are required
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The configuration schema for the check’s underlying
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that executes the check.
+ - compute_kind (Optional[str]) – A string to represent the kind of computation that executes
+ - retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that executes the check.
+ - metadata (Optional[Mapping[str, Any]]) – A dictionary of static metadata for the check.
+ - automation_condition (Optional[[*AutomationCondition*](assets.mdx#dagster.AutomationCondition)]) – An AutomationCondition which determines
+
+
+ Produces an [`AssetChecksDefinition`](#dagster.AssetChecksDefinition) object.
+
+ Example:
+
+ ```python
+ from dagster import asset, asset_check, AssetCheckResult
+
+ @asset
+ def my_asset() -> None:
+ ...
+
+ @asset_check(asset=my_asset, description="Check that my asset has enough rows")
+ def my_asset_has_enough_rows() -> AssetCheckResult:
+ num_rows = ...
+ return AssetCheckResult(passed=num_rows > 5, metadata={"num_rows": num_rows})
+ ```
+ Example with a DataFrame Output:
+ ```python
+ from dagster import asset, asset_check, AssetCheckResult
+ from pandas import DataFrame
+
+ @asset
+ def my_asset() -> DataFrame:
+ ...
+
+ @asset_check(asset=my_asset, description="Check that my asset has enough rows")
+ def my_asset_has_enough_rows(my_asset: DataFrame) -> AssetCheckResult:
+ num_rows = my_asset.shape[0]
+ return AssetCheckResult(passed=num_rows > 5, metadata={"num_rows": num_rows})
+ ```
+
+
+
+
+
+
class dagster.AssetCheckResult
+
+
+ The result of an asset check.
+
+
+
asset_key
+
+
+ The asset key that was checked.
+
+ Type: Optional[[AssetKey](assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
check_name
+
+
+ The name of the check.
+
+ Type: Optional[str]
+
+
+
+
+
+
passed
+
+
+ The pass/fail result of the check.
+
+ Type: bool
+
+
+
+
+
+
metadata
+
+
+ Arbitrary metadata about the asset. Keys are displayed string labels, and values are
+ one of the following: string, float, int, JSON-serializable dict, JSON-serializable
+ list, and one of the data classes returned by a MetadataValue static method.
+
+ Type: Optional[Dict[str, RawMetadataValue]]
+
+
+
+
+
+
severity
+
+
+ Severity of the check. Defaults to ERROR.
+
+ Type: [AssetCheckSeverity](#dagster.AssetCheckSeverity)
+
+
+
+
+
+
description
+
+
+ A text description of the result of the check evaluation.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.AssetCheckSpec
+
+
+ Defines information about an asset check, except how to execute it.
+
+ AssetCheckSpec is often used as an argument to decorators that decorator a function that can
+ execute multiple checks - e.g. @asset, and @multi_asset. It defines one of the checks that
+ will be executed inside that function.
+
+ Parameters:
+ - name (str) – Name of the check.
+ - asset (Union[[*AssetKey*](assets.mdx#dagster.AssetKey), Sequence[str], str, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]) – The asset that
+ - description (Optional[str]) – Description for the check.
+ - additional_deps (Optional[Iterable[[*AssetDep*](assets.mdx#dagster.AssetDep)]]) – Additional dependencies for the check. The
+ - metadata (Optional[Mapping[str, Any]]) – A dict of static metadata for this asset check.
+
+
+
+
+
+
+
+
class dagster.AssetCheckSeverity
+
+
+ Severity level for an AssetCheckResult.
+
+ - WARN: a potential issue with the asset
+ - ERROR: a definite issue with the asset
+
+
+ Severity does not impact execution of the asset or downstream assets.
+
+
+
+
+
+
+
class dagster.AssetCheckKey
+
+ Check names are expected to be unique per-asset. Thus, this combination of asset key and
+ check name uniquely identifies an asset check within a deployment.
+
+
+
+
+
@dagster.multi_asset_check
+
+
+ Defines a set of asset checks that can be executed together with the same op.
+
+ Parameters:
+ - specs (Sequence[[*AssetCheckSpec*](#dagster.AssetCheckSpec)]) – Specs for the asset checks.
+ - name (Optional[str]) – The name of the op. If not specified, the name of the decorated
+ - description (Optional[str]) – Description of the op.
+ - required_resource_keys (Optional[Set[str]]) – A set of keys for resources that are required
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The configuration schema for the asset checks’ underlying
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that executes the checks.
+ - compute_kind (Optional[str]) – A string to represent the kind of computation that executes
+ - retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that executes the checks.
+ - can_subset (bool) – Whether the op can emit results for a subset of the asset checks
+ - ins (Optional[Mapping[str, Union[[*AssetKey*](assets.mdx#dagster.AssetKey), [*AssetIn*](assets.mdx#dagster.AssetIn)]]]) – A mapping from input name to AssetIn depended upon by
+
+
+ Examples:
+
+ ```python
+ @multi_asset_check(
+ specs=[
+ AssetCheckSpec("enough_rows", asset="asset1"),
+ AssetCheckSpec("no_dupes", asset="asset1"),
+ AssetCheckSpec("enough_rows", asset="asset2"),
+ ],
+ )
+ def checks():
+ yield AssetCheckResult(passed=True, asset_key="asset1", check_name="enough_rows")
+ yield AssetCheckResult(passed=False, asset_key="asset1", check_name="no_dupes")
+ yield AssetCheckResult(passed=True, asset_key="asset2", check_name="enough_rows")
+ ```
+
+
+
+
+
+
dagster.load_asset_checks_from_modules
+
+
+ Constructs a list of asset checks from the given modules. This is most often used in
+ conjunction with a call to load_assets_from_modules.
+
+ Parameters:
+ - modules (Iterable[ModuleType]) – The Python modules to look for checks inside.
+ - asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the
+
+
+ Returns: A list containing asset checks defined in the given modules.Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
dagster.load_asset_checks_from_current_module
+
+
+ Constructs a list of asset checks from the module where this function is called. This is most
+ often used in conjunction with a call to load_assets_from_current_module.
+
+ Parameters: asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the
+ key_prefix argument to load_assets_from_current_module.Returns: A list containing asset checks defined in the current module.Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
dagster.load_asset_checks_from_package_module
+
+
+ Constructs a list of asset checks from all sub-modules of the given package module. This is
+ most often used in conjunction with a call to load_assets_from_package_module.
+
+ Parameters:
+ - package_module (ModuleType) – The Python module to look for checks inside.
+ - asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the
+
+
+ Returns: A list containing asset checks defined in the package.Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
dagster.load_asset_checks_from_package_name
+
+
+ Constructs a list of asset checks from all sub-modules of the given package. This is most
+ often used in conjunction with a call to load_assets_from_package_name.
+
+ Parameters:
+ - package_name (str) – The name of the Python package to look for checks inside.
+ - asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the
+
+
+ Returns: A list containing asset checks defined in the package.Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
class dagster.AssetChecksDefinition
+
+
+ Defines a set of checks that are produced by the same op or op graph.
+
+ AssetChecksDefinition should not be instantiated directly, but rather produced using the @asset_check decorator or AssetChecksDefinition.create method.
+
+
+
+
+
+
+
dagster.build_last_update_freshness_checks
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs an AssetChecksDefinition that checks the freshness of the provided assets.
+
+ This check passes if the asset is found to be “fresh”, and fails if the asset is found to be
+ “overdue”. An asset is considered fresh if a record (i.e. a materialization or observation)
+ exists with a timestamp greater than the “lower bound” derived from the parameters of this
+ function.
+
+ deadline_cron is a cron schedule that defines the deadline for when we should expect the asset
+ to arrive by; if not provided, we consider the deadline to be the execution time of the check.
+ lower_bound_delta is a timedelta that defines the lower bound for when a record could have
+ arrived by. If the most recent recent record’s timestamp is earlier than
+ deadline-lower_bound_delta, the asset is considered overdue.
+
+ Let’s use two examples, one with a deadline_cron set and one without.
+ Let’s say I have an asset which runs on a schedule every day at 8:00 AM UTC, and usually takes
+ around 45 minutes to complete. To account for operational delays, I would expect the asset to be
+ done materializing every day by 9:00 AM UTC. I would set the deadline_cron to “0 9 * * *”, and
+ the lower_bound_delta to “45 minutes”. This would mean that starting at 9:00 AM, this check
+ will expect a materialization record to have been created no earlier than 8:15 AM. Note that if
+ the check runs at 8:59 AM, the deadline has not yet passed, and we’ll instead be checking for
+ the most recently passed deadline, which is yesterday.
+ Let’s say I have an observable source asset on a data source which I expect should never be more
+ than 3 hours out of date. In this case, there’s no fixed schedule for when the data should be
+ updated, so I would not provide a deadline_cron. Instead, I would set the lower_bound_delta
+ parameter to “3 hours”. This would mean that the check will expect the most recent observation
+ record to indicate data no older than 3 hours, relative to the current time, regardless of when it runs.
+
+ The check result will contain the following metadata:
+ “dagster/freshness_params”: A dictionary containing the parameters used to construct the
+ check
+ “dagster/last_updated_time”: The time of the most recent update to the asset
+ “dagster/overdue_seconds”: (Only present if asset is overdue) The number of seconds that the
+ asset is overdue by.
+ “dagster/overdue_deadline_timestamp”: The timestamp that we are expecting the asset to have
+ arrived by. In the case of a provided deadline_cron, this is the timestamp of the most recent
+ tick of the cron schedule. In the case of no deadline_cron, this is the current time.
+
+ Examples:
+
+ ```python
+ # Example 1: Assets that are expected to be updated every day within 45 minutes of
+ # 9:00 AM UTC
+ from dagster import build_last_update_freshness_checks, AssetKey
+ from .somewhere import my_daily_scheduled_assets_def
+
+ checks_def = build_last_update_freshness_checks(
+ [my_daily_scheduled_assets_def, AssetKey("my_other_daily_asset_key")],
+ lower_bound_delta=datetime.timedelta(minutes=45),
+ deadline_cron="0 9 * * *",
+ )
+
+ # Example 2: Assets that are expected to be updated within 3 hours of the current time
+ from dagster import build_last_update_freshness_checks, AssetKey
+ from .somewhere import my_observable_source_asset
+
+ checks_def = build_last_update_freshness_checks(
+ [my_observable_source_asset, AssetKey("my_other_observable_asset_key")],
+ lower_bound_delta=datetime.timedelta(hours=3),
+ )
+ ```
+ Parameters:
+ - assets (Sequence[Union[CoercibleToAssetKey, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]) – The assets to
+ - lower_bound_delta (datetime.timedelta) – The check will pass if the asset was updated within
+ - deadline_cron (Optional[str]) – Defines the deadline for when we should start checking
+ - timezone (Optional[str]) – The timezone to use when calculating freshness and deadline. If
+
+
+ Returns:
+ AssetChecksDefinition objects which execute freshness checks
+ for the provided assets.
+
+ Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
dagster.build_time_partition_freshness_checks
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Construct an AssetChecksDefinition that checks the freshness of the provided assets.
+
+ This check passes if the asset is considered “fresh” by the time that execution begins. We
+ consider an asset to be “fresh” if there exists a record for the most recent partition, once
+ the deadline has passed.
+
+ deadline_cron is a cron schedule that defines the deadline for when we should expect the most
+ recent partition to arrive by. Once a tick of the cron schedule has passed, this check will fail
+ if the most recent partition has not been observed/materialized.
+
+ Let’s say I have a daily-partitioned asset which runs every day at 8:00 AM UTC, and takes around
+ 45 minutes to complete. To account for operational delays, I would expect the asset to be done
+ materializing every day by 9:00 AM UTC. I would set the deadline_cron to “0 9 * * *”. This
+ means that starting at 9:00 AM, this check will expect a record to exist for the previous day’s
+ partition. Note that if the check runs at 8:59 AM, the deadline has not yet passed, and we’ll
+ instead be checking for the most recently passed deadline, which is yesterday (meaning the
+ partition representing the day before yesterday).
+
+ The timestamp of an observation record is the timestamp indicated by the
+ “dagster/last_updated_timestamp” metadata key. The timestamp of a materialization record is the
+ timestamp at which that record was created.
+
+ The check will fail at runtime if a non-time-window partitioned asset is passed in.
+
+ The check result will contain the following metadata:
+ “dagster/freshness_params”: A dictionary containing the parameters used to construct the
+ check.
+ “dagster/last_updated_time”: (Only present if the asset has been observed/materialized before)
+ The time of the most recent update to the asset.
+ “dagster/overdue_seconds”: (Only present if asset is overdue) The number of seconds that the
+ asset is overdue by.
+ “dagster/overdue_deadline_timestamp”: The timestamp that we are expecting the asset to have
+ arrived by. This is the timestamp of the most recent tick of the cron schedule.
+
+ Examples:
+
+ ```python
+ from dagster import build_time_partition_freshness_checks, AssetKey
+ # A daily partitioned asset that is expected to be updated every day within 45 minutes
+ # of 9:00 AM UTC
+ from .somewhere import my_daily_scheduled_assets_def
+
+ checks_def = build_time_partition_freshness_checks(
+ [my_daily_scheduled_assets_def],
+ deadline_cron="0 9 * * *",
+ )
+ ```
+ Parameters:
+ - assets (Sequence[Union[CoercibleToAssetKey, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]) – The assets to
+ - deadline_cron (str) – The check will pass if the partition time window most recently
+ - timezone (Optional[str]) – The timezone to use when calculating freshness and deadline. If
+
+
+ Returns:
+ AssetChecksDefinition objects which execute freshness
+ checks for the provided assets.
+
+ Return type: Sequence[[AssetChecksDefinition](#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
dagster.build_sensor_for_freshness_checks
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Builds a sensor which kicks off evaluation of freshness checks.
+
+ This sensor will kick off an execution of a check in the following cases:
+ - The check has never been executed before.
+ - The check has been executed before, and the previous result was a success, but it is again
+ possible for the check to be overdue based on the dagster/fresh_until_timestamp metadata
+ on the check result.
+
+ Note that we will not execute if:
+ - The freshness check has been executed before, and the previous result was a failure. This is
+ because whichever run materializes/observes the run to bring the check back to a passing
+ state will end up also running the check anyway, so until that run occurs, there’s no point
+ in evaluating the check.
+ - The freshness check has been executed before, and the previous result was a success, but it is
+ not possible for the check to be overdue based on the dagster/fresh_until_timestamp
+ metadata on the check result. Since the check cannot be overdue, we know the check
+ result would not change with an additional execution.
+
+ Parameters:
+ - freshness_checks (Sequence[[*AssetChecksDefinition*](#dagster.AssetChecksDefinition)]) – The freshness checks to evaluate.
+ - minimum_interval_seconds (Optional[int]) – The duration in seconds between evaluations of the sensor.
+ - name (Optional[str]) – The name of the sensor. Defaults to “freshness_check_sensor”, but a
+ - default_status (Optional[DefaultSensorStatus]) – The default status of the sensor. Defaults
+
+
+ Returns: The sensor that kicks off freshness evaluations.Return type: [SensorDefinition](schedules-sensors.mdx#dagster.SensorDefinition)
+
+
+
+
+
+
dagster.build_column_schema_change_checks
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns asset checks that pass if the column schema of the asset’s latest materialization
+ is the same as the column schema of the asset’s previous materialization.
+
+ The underlying materializations are expected to have a metadata entry with key dagster/column_schema and type [`TableSchema`](metadata.mdx#dagster.TableSchema).
+ To learn more about how to add column schema metadata and other forms of tabular metadata to assets, see
+ [https://docs.dagster.io/guides/build/assets/metadata-and-tags/table-metadata#attaching-column-schema](https://docs.dagster.io/guides/build/assets/metadata-and-tags/table-metadata#attaching-column-schema).
+
+ The resulting checks will fail if any changes are detected in the column schema between
+ materializations, including:
+ - Added columns
+ - Removed columns
+ - Changes to column types
+
+ The check failure message will detail exactly what changed in the schema.
+
+ Parameters:
+ - assets (Sequence[Union[[*AssetKey*](assets.mdx#dagster.AssetKey), str, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]]) – The assets to create
+ - severity ([*AssetCheckSeverity*](#dagster.AssetCheckSeverity)) – The severity if the check fails. Defaults to WARN.
+
+
+ Returns: Sequence[AssetsChecksDefinition]
+ Examples:
+
+ First, define an asset with column schema metadata. You can attach schema metadata either as
+ definition metadata (when schema is known at definition time) or as materialization metadata
+ (when schema is only known at runtime):
+
+ ```python
+ import dagster as dg
+
+ # Using definition metadata when schema is known upfront
+ @dg.asset
+ def people_table():
+ column_names = ...
+ column_types = ...
+
+ columns = [
+ dg.TableColumn(name, column_type)
+ for name, column_type in zip(column_names, column_types)
+ ]
+
+ yield dg.MaterializeResult(
+ metadata={"dagster/column_schema": dg.TableSchema(columns=columns)}
+ )
+ ```
+ Once you have assets with column schema metadata, you can create schema change checks to monitor
+ for changes in the schema between materializations:
+
+ ```python
+ # Create schema change checks for one or more assets
+ schema_checks = dg.build_column_schema_change_checks(
+ assets=[people_table]
+ )
+ ```
+
+
+
+
+
+
dagster.build_metadata_bounds_checks
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns asset checks that pass if the metadata value of the asset’s latest materialization
+ is within the specified range.
+
+ Parameters:
+ - assets (Sequence[Union[[*AssetKey*](assets.mdx#dagster.AssetKey), str, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]]) – The assets to create
+ - severity ([*AssetCheckSeverity*](#dagster.AssetCheckSeverity)) – The severity if the check fails. Defaults to WARN.
+ - metadata_key (str) – The metadata key to check.
+ - min_value (Optional[Union[int, float]]) – The minimum value to check for. If None, no minimum
+ - max_value (Optional[Union[int, float]]) – The maximum value to check for. If None, no maximum
+ - exclusive_min (bool) – If True, the check will fail if the metadata value is equal to min_value.
+ - exclusive_max (bool) – If True, the check will fail if the metadata value is equal to max_value.
+
+
+ Returns: Sequence[AssetsChecksDefinition]
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/assets.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/assets.mdx
new file mode 100644
index 0000000000000..ca89381c2e102
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/assets.mdx
@@ -0,0 +1,2417 @@
+---
+title: 'assets'
+title_meta: 'assets API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'assets Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Assets
+
+An asset is an object in persistent storage, such as a table, file, or persisted machine learning model. An asset definition is a description, in code, of an asset that should exist and how to produce and update that asset.
+
+
+
+
+## Asset definitions
+
+Refer to the [Asset definitions](https://docs.dagster.io/guides/build/assets/defining-assets) documentation for more information.
+
+
+
@dagster.asset
+
+
+ Create a definition for how to compute an asset.
+
+ A software-defined asset is the combination of:
+ 1. An asset key, e.g. the name of a table.
+ 2. A function, which can be run to compute the contents of the asset.
+ 3. A set of upstream assets that are provided as inputs to the function when computing the asset.
+ Unlike an op, whose dependencies are determined by the graph it lives inside, an asset knows
+ about the upstream assets it depends on. The upstream assets are inferred from the arguments
+ to the decorated function. The name of the argument designates the name of the upstream asset.
+
+ An asset has an op inside it to represent the function that computes it. The name of the op
+ will be the segments of the asset key, separated by double-underscores.
+
+ Parameters:
+ - name (Optional[str]) – The name of the asset. If not provided, defaults to the name of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the
+ - ins (Optional[Mapping[str, [*AssetIn*](#dagster.AssetIn)]]) – A dictionary that maps input names to information
+ - deps (Optional[Sequence[Union[[*AssetDep*](#dagster.AssetDep), [*AssetsDefinition*](#dagster.AssetsDefinition), [*SourceAsset*](#dagster.SourceAsset), [*AssetKey*](#dagster.AssetKey), str]]]) – The assets that are upstream dependencies, but do not correspond to a parameter of the
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The configuration schema for the asset’s underlying
+ - metadata (Optional[Dict[str, Any]]) – A dict of metadata entries for the asset.
+ - tags (Optional[Mapping[str, str]]) – Tags for filtering and organizing. These tags are not
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the op.
+ - io_manager_key (Optional[str]) – The resource key of the IOManager used
+ - io_manager_def (Optional[object]) – experimental
+ - dagster_type (Optional[[*DagsterType*](types.mdx#dagster.DagsterType)]) – Allows specifying type validation functions that
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset.
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided,
+ - resource_defs (Optional[Mapping[str, object]]) – experimental
+ - output_required (bool) – Whether the decorated function will always materialize an asset.
+ - automation_condition ([*AutomationCondition*](#dagster.AutomationCondition)) – A condition describing when Dagster should materialize this asset.
+ - backfill_policy ([*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)) – experimental
+ - retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that computes the asset.
+ - code_version (Optional[str]) – Version of the code that generates this asset. In
+ - check_specs (Optional[Sequence[[*AssetCheckSpec*](asset-checks.mdx#dagster.AssetCheckSpec)]]) – Specs for asset checks that
+ - key (Optional[CoeercibleToAssetKey]) – The key for this asset. If provided, cannot specify key_prefix or name.
+ - owners (Optional[Sequence[str]]) – experimentalteam:,
+ - kinds (Optional[Set[str]]) – A list of strings representing the kinds of the asset. These
+ - pool (Optional[str]) – A string that identifies the concurrency pool that governs this asset’s execution.
+ - non_argument_deps (Optional[Union[Set[[*AssetKey*](#dagster.AssetKey)], Set[str]]]) – deprecateddeps instead.) Deprecated, use deps instead.
+
+
+ Examples:
+
+ ```python
+ @asset
+ def my_upstream_asset() -> int:
+ return 5
+
+ @asset
+ def my_asset(my_upstream_asset: int) -> int:
+ return my_upstream_asset + 1
+
+ should_materialize = True
+
+ @asset(output_required=False)
+ def conditional_asset():
+ if should_materialize:
+ yield Output(5) # you must `yield`, not `return`, the result
+
+ # Will also only materialize if `should_materialize` is `True`
+ @asset
+ def downstream_asset(conditional_asset):
+ return conditional_asset + 1
+ ```
+
+
+
+
+
+
class dagster.MaterializeResult
+
+
+ An object representing a successful materialization of an asset. These can be returned from
+ @asset and @multi_asset decorated functions to pass metadata or specify specific assets were
+ materialized.
+
+
+
asset_key
+
+
+ Optional in @asset, required in @multi_asset to discern which asset this refers to.
+
+ Type: Optional[[AssetKey](#dagster.AssetKey)]
+
+
+
+
+
+
metadata
+
+
+ Metadata to record with the corresponding AssetMaterialization event.
+
+ Type: Optional[RawMetadataMapping]
+
+
+
+
+
+
check_results
+
+
+ Check results to record with the
+ corresponding AssetMaterialization event.
+
+ Type: Optional[Sequence[[AssetCheckResult](asset-checks.mdx#dagster.AssetCheckResult)]]
+
+
+
+
+
+
data_version
+
+
+ The data version of the asset that was observed.
+
+ Type: Optional[DataVersion]
+
+
+
+
+
+
tags
+
+
+ Tags to record with the corresponding
+ AssetMaterialization event.
+
+ Type: Optional[Mapping[str, str]]
+
+
+
+
+
+
+
+
+
+
class dagster.AssetSpec
+
+
+ Specifies the core attributes of an asset, except for the function that materializes or
+ observes it.
+
+ An asset spec plus any materialization or observation function for the asset constitutes an
+ “asset definition”.
+
+
+
key
+
+
+ The unique identifier for this asset.
+
+ Type: [AssetKey](#dagster.AssetKey)
+
+
+
+
+
+
deps
+
+
+ The asset keys for the upstream assets that
+ materializing this asset depends on.
+
+ Type: Optional[AbstractSet[[AssetKey](#dagster.AssetKey)]]
+
+
+
+
+
+
description
+
+
+ Human-readable description of this asset.
+
+ Type: Optional[str]
+
+
+
+
+
+
metadata
+
+
+ A dict of static metadata for this asset.
+ For example, users can provide information about the database table this
+ asset corresponds to.
+
+ Type: Optional[Dict[str, Any]]
+
+
+
+
+
+
skippable
+
+
+ Whether this asset can be omitted during materialization, causing downstream
+ dependencies to skip.
+
+ Type: bool
+
+
+
+
+
+
group_name
+
+
+ A string name used to organize multiple assets into groups. If
+ not provided, the name “default” is used.
+
+ Type: Optional[str]
+
+
+
+
+
+
code_version
+
+
+ The version of the code for this specific asset,
+ overriding the code version of the materialization function
+
+ Type: Optional[str]
+
+
+
+
+
+
backfill_policy
+
+
+ BackfillPolicy to apply to the specified asset.
+
+ Type: Optional[[BackfillPolicy](partitions.mdx#dagster.BackfillPolicy)]
+
+
+
+
+
+
owners
+
+
+ A list of strings representing owners of the asset. Each
+ string can be a user’s email address, or a team name prefixed with team:,
+ e.g. team:finops.
+
+ Type: Optional[Sequence[str]]
+
+
+
+
+
+
tags
+
+
+ Tags for filtering and organizing. These tags are not
+ attached to runs of the asset.
+
+ Type: Optional[Mapping[str, str]]
+
+
+
+
+
+
kinds
+
+ (Optional[Set[str]]): A list of strings representing the kinds of the asset. These
+ will be made visible in the Dagster UI.
+
+
+
+
+
partitions_def
+
+
+ Defines the set of partition keys that
+ compose the asset.
+
+ Type: Optional[[PartitionsDefinition](partitions.mdx#dagster.PartitionsDefinition)]
+
+
+
+
+
+
merge_attributes
+
+
+ Returns a new AssetSpec with the specified attributes merged with the current attributes.
+
+ Parameters:
+ - deps (Optional[Iterable[CoercibleToAssetDep]]) – A set of asset dependencies to add to
+ - metadata (Optional[Mapping[str, Any]]) – A set of metadata to add to the asset self.
+ - owners (Optional[Sequence[str]]) – A set of owners to add to the asset self.
+ - tags (Optional[Mapping[str, str]]) – A set of tags to add to the asset self.
+ - kinds (Optional[Set[str]]) – A set of kinds to add to the asset self.
+
+
+ Returns: AssetSpec
+
+
+
+
+
+
replace_attributes
+
+ Returns a new AssetSpec with the specified attributes replaced.
+
+
+
+
+
with_io_manager_key
+
+
+ Returns a copy of this AssetSpec with an extra metadata value that dictates which I/O
+ manager to use to load the contents of this asset in downstream computations.
+
+ Parameters: io_manager_key (str) – The I/O manager key. This will be used as the value for the
+ “dagster/io_manager_key” metadata key.Returns: AssetSpec
+
+
+
+
+
+
+
+
+
+
class dagster.AssetsDefinition
+
+
+ Defines a set of assets that are produced by the same op or graph.
+
+ AssetsDefinitions are typically not instantiated directly, but rather produced using the
+ [`@asset`](#dagster.asset) or [`@multi_asset`](#dagster.multi_asset) decorators.
+
+
+
static from_graph
+
+
+ Constructs an AssetsDefinition from a GraphDefinition.
+
+ Parameters:
+ - graph_def ([*GraphDefinition*](graphs.mdx#dagster.GraphDefinition)) – The GraphDefinition that is an asset.
+ - keys_by_input_name (Optional[Mapping[str, [*AssetKey*](#dagster.AssetKey)]]) – A mapping of the input
+ - keys_by_output_name (Optional[Mapping[str, [*AssetKey*](#dagster.AssetKey)]]) – A mapping of the output
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, key_prefix will be prepended
+ - internal_asset_deps (Optional[Mapping[str, Set[[*AssetKey*](#dagster.AssetKey)]]]) – By default, it is assumed
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - partition_mappings (Optional[Mapping[str, [*PartitionMapping*](partitions.mdx#dagster.PartitionMapping)]]) – Defines how to map partition
+ - resource_defs (Optional[Mapping[str, [*ResourceDefinition*](resources.mdx#dagster.ResourceDefinition)]]) – experimental
+ - group_name (Optional[str]) – A group name for the constructed asset. Assets without a
+ - group_names_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a group name to be
+ - descriptions_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a description to be
+ - metadata_by_output_name (Optional[Mapping[str, Optional[RawMetadataMapping]]]) – Defines metadata to
+ - tags_by_output_name (Optional[Mapping[str, Optional[Mapping[str, str]]]]) – Defines
+ - freshness_policies_by_output_name (Optional[Mapping[str, Optional[FreshnessPolicy]]]) – Defines a
+ - automation_conditions_by_output_name (Optional[Mapping[str, Optional[[*AutomationCondition*](#dagster.AutomationCondition)]]]) – Defines an
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – Defines this asset’s BackfillPolicy
+ - owners_by_key (Optional[Mapping[[*AssetKey*](#dagster.AssetKey), Sequence[str]]]) – Defines
+
+
+
+
+
+
+
+
static from_op
+
+
+ Constructs an AssetsDefinition from an OpDefinition.
+
+ Parameters:
+ - op_def ([*OpDefinition*](ops.mdx#dagster.OpDefinition)) – The OpDefinition that is an asset.
+ - keys_by_input_name (Optional[Mapping[str, [*AssetKey*](#dagster.AssetKey)]]) – A mapping of the input
+ - keys_by_output_name (Optional[Mapping[str, [*AssetKey*](#dagster.AssetKey)]]) – A mapping of the output
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, key_prefix will be prepended
+ - internal_asset_deps (Optional[Mapping[str, Set[[*AssetKey*](#dagster.AssetKey)]]]) – By default, it is assumed
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - partition_mappings (Optional[Mapping[str, [*PartitionMapping*](partitions.mdx#dagster.PartitionMapping)]]) – Defines how to map partition
+ - group_name (Optional[str]) – A group name for the constructed asset. Assets without a
+ - group_names_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a group name to be
+ - descriptions_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a description to be
+ - metadata_by_output_name (Optional[Mapping[str, Optional[RawMetadataMapping]]]) – Defines metadata to
+ - tags_by_output_name (Optional[Mapping[str, Optional[Mapping[str, str]]]]) – Defines
+ - freshness_policies_by_output_name (Optional[Mapping[str, Optional[FreshnessPolicy]]]) – Defines a
+ - automation_conditions_by_output_name (Optional[Mapping[str, Optional[[*AutomationCondition*](#dagster.AutomationCondition)]]]) – Defines an
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – Defines this asset’s BackfillPolicy
+
+
+
+
+
+
+
+
get_asset_spec
+
+
+ Returns a representation of this asset as an [`AssetSpec`](#dagster.AssetSpec).
+
+ If this is a multi-asset, the “key” argument allows selecting which asset to return the
+ spec for.
+
+ Parameters: key (Optional[[*AssetKey*](#dagster.AssetKey)]) – If this is a multi-asset, select which asset to return its
+ AssetSpec. If not a multi-asset, this can be left as None.Returns: AssetSpec
+
+
+
+
+
+
get_partition_mapping
+
+ Returns the partition mapping between keys in this AssetsDefinition and a given input
+ asset key (if any).
+
+
+
+
+
to_source_asset
+
+
+ Returns a representation of this asset as a [`SourceAsset`](#dagster.SourceAsset).
+
+ If this is a multi-asset, the “key” argument allows selecting which asset to return a
+ SourceAsset representation of.
+
+ Parameters: key (Optional[Union[str, Sequence[str], [*AssetKey*](#dagster.AssetKey)]]]) – If this is a multi-asset, select
+ which asset to return a SourceAsset representation of. If not a multi-asset, this
+ can be left as None.Returns: SourceAsset
+
+
+
+
+
+
to_source_assets
+
+
+ Returns a SourceAsset for each asset in this definition.
+
+ Each produced SourceAsset will have the same key, metadata, io_manager_key, etc. as the
+ corresponding asset
+
+
+
+
+
+
+
property asset_deps
+
+ Maps assets that are produced by this definition to assets that they depend on. The
+ dependencies can be either “internal”, meaning that they refer to other assets that are
+ produced by this definition, or “external”, meaning that they refer to assets that aren’t
+ produced by this definition.
+
+
+
+
+
property can_subset
+
+
+ If True, indicates that this AssetsDefinition may materialize any subset of its
+ asset keys in a given computation (as opposed to being required to materialize all asset
+ keys).
+
+ Type: bool
+
+
+
+
+
+
property check_specs
+
+
+ Returns the asset check specs defined on this AssetsDefinition, i.e. the checks that can
+ be executed while materializing the assets.
+
+ Return type: Iterable[AssetsCheckSpec]
+
+
+
+
+
+
property dependency_keys
+
+
+ The asset keys which are upstream of any asset included in this
+ AssetsDefinition.
+
+ Type: Iterable[[AssetKey](#dagster.AssetKey)]
+
+
+
+
+
+
property descriptions_by_key
+
+
+ Returns a mapping from the asset keys in this AssetsDefinition
+ to the descriptions assigned to them. If there is no assigned description for a given AssetKey,
+ it will not be present in this dictionary.
+
+ Type: Mapping[[AssetKey](#dagster.AssetKey), str]
+
+
+
+
+
+
property group_names_by_key
+
+
+ Returns a mapping from the asset keys in this AssetsDefinition
+ to the group names assigned to them. If there is no assigned group name for a given AssetKey,
+ it will not be present in this dictionary.
+
+ Type: Mapping[[AssetKey](#dagster.AssetKey), str]
+
+
+
+
+
+
property key
+
+
+ The asset key associated with this AssetsDefinition. If this AssetsDefinition
+ has more than one asset key, this will produce an error.
+
+ Type: [AssetKey](#dagster.AssetKey)
+
+
+
+
+
+
property keys
+
+
+ The asset keys associated with this AssetsDefinition.
+
+ Type: AbstractSet[[AssetKey](#dagster.AssetKey)]
+
+
+
+
+
+
property node_def
+
+
+ Returns the OpDefinition or GraphDefinition that is used to materialize
+ the assets in this AssetsDefinition.
+
+ Type: NodeDefinition
+
+
+
+
+
+
property op
+
+
+ Returns the OpDefinition that is used to materialize the assets in this
+ AssetsDefinition.
+
+ Type: [OpDefinition](ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
property partitions_def
+
+
+ The PartitionsDefinition for this AssetsDefinition (if any).
+
+ Type: Optional[[PartitionsDefinition](partitions.mdx#dagster.PartitionsDefinition)]
+
+
+
+
+
+
property required_resource_keys
+
+
+ The set of keys for resources that must be provided to this AssetsDefinition.
+
+ Type: Set[str]
+
+
+
+
+
+
property resource_defs
+
+
+ A mapping from resource name to ResourceDefinition for
+ the resources bound to this AssetsDefinition.
+
+ Type: Mapping[str, [ResourceDefinition](resources.mdx#dagster.ResourceDefinition)]
+
+
+
+
+
+
+
+
+
+
class dagster.AssetKey
+
+
+ Object representing the structure of an asset key. Takes in a sanitized string, list of
+ strings, or tuple of strings.
+
+ Example usage:
+
+ ```python
+ from dagster import AssetKey
+
+ AssetKey("asset1")
+ AssetKey(["asset1"]) # same as the above
+ AssetKey(["prefix", "asset1"])
+ AssetKey(["prefix", "subprefix", "asset1"])
+ ```
+ Parameters: path (Union[str, Sequence[str]]) – String, list of strings, or tuple of strings. A list of
+ strings represent the hierarchical structure of the asset_key.
+
+
property path
+
+
+
+
+
+
+
+
+
+
+
dagster.map_asset_specs
+
+
+ Map a function over a sequence of AssetSpecs or AssetsDefinitions, replacing specs in the sequence
+ or specs in an AssetsDefinitions with the result of the function.
+
+ Parameters:
+ - func (Callable[[[*AssetSpec*](#dagster.AssetSpec)], [*AssetSpec*](#dagster.AssetSpec)]) – The function to apply to each AssetSpec.
+ - iterable (Iterable[Union[[*AssetsDefinition*](#dagster.AssetsDefinition), [*AssetSpec*](#dagster.AssetSpec)]]) – The sequence of AssetSpecs or AssetsDefinitions.
+
+
+ Returns:
+ A sequence of AssetSpecs or AssetsDefinitions with the function applied
+ to each spec.
+
+ Return type: Sequence[Union[[AssetsDefinition](#dagster.AssetsDefinition), [AssetSpec](#dagster.AssetSpec)]]
+ Examples:
+
+ ```python
+ from dagster import AssetSpec, map_asset_specs
+
+ asset_specs = [
+ AssetSpec(key="my_asset"),
+ AssetSpec(key="my_asset_2"),
+ ]
+
+ mapped_specs = map_asset_specs(lambda spec: spec.replace_attributes(owners=["nelson@hooli.com"]), asset_specs)
+ ```
+
+
+
+
+
+
+
+
+
+
+## Graph-backed asset definitions
+
+Refer to the [Graph-backed asset](https://docs.dagster.io/guides/build/assets/defining-assets#graph-asset) documentation for more information.
+
+
+
@dagster.graph_asset
+
+
+ Creates a software-defined asset that’s computed using a graph of ops.
+
+ This decorator is meant to decorate a function that composes a set of ops or graphs to define
+ the dependencies between them.
+
+ Parameters:
+ - name (Optional[str]) – The name of the asset. If not provided, defaults to the name of the
+ - description (Optional[str]) – A human-readable description of the asset.
+ - ins (Optional[Mapping[str, [*AssetIn*](#dagster.AssetIn)]]) – A dictionary that maps input names to information
+ - config (Optional[Union[[*ConfigMapping*](config.mdx#dagster.ConfigMapping)], Mapping[str, Any]) –
+
+ Describes how the graph underlying the asset is configured at runtime.
+
+ If a [`ConfigMapping`](config.mdx#dagster.ConfigMapping) object is provided, then the graph takes on the config
+ schema of this object. The mapping will be applied at runtime to generate the config for
+ the graph’s constituent nodes.
+
+ If a dictionary is provided, then it will be used as the default run config for the
+ graph. This means it must conform to the config schema of the underlying nodes. Note
+ that the values provided will be viewable and editable in the Dagster UI, so be careful
+ with secrets.
+
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. If
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - metadata (Optional[RawMetadataMapping]) – Dictionary of metadata to be associated with
+ - tags (Optional[Mapping[str, str]]) – (Experimental) Tags for filtering and organizing. These tags are not
+ - owners (Optional[Sequence[str]]) – experimentalteam:,
+ - kinds (Optional[Set[str]]) – A list of strings representing the kinds of the asset. These
+ - automation_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – The AutomationCondition to use
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – The BackfillPolicy to use for this asset.
+ - code_version (Optional[str]) – Version of the code that generates this asset. In
+ - key (Optional[CoeercibleToAssetKey]) – The key for this asset. If provided, cannot specify key_prefix or name.
+
+
+ Examples:
+
+ ```python
+ @op
+ def fetch_files_from_slack(context) -> pd.DataFrame:
+ ...
+
+ @op
+ def store_files(files) -> None:
+ files.to_sql(name="slack_files", con=create_db_connection())
+
+ @graph_asset
+ def slack_files_table():
+ return store_files(fetch_files_from_slack())
+ ```
+
+
+
+
+
+
@dagster.graph_multi_asset
+
+
+ Create a combined definition of multiple assets that are computed using the same graph of
+ ops, and the same upstream assets.
+
+ Each argument to the decorated function references an upstream asset that this asset depends on.
+ The name of the argument designates the name of the upstream asset.
+
+ Parameters:
+ - name (Optional[str]) – The name of the graph.
+ - outs – (Optional[Dict[str, AssetOut]]): The AssetOuts representing the produced assets.
+ - ins (Optional[Mapping[str, [*AssetIn*](#dagster.AssetIn)]]) – A dictionary that maps input names to information
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – The backfill policy for the asset.
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. This
+ - can_subset (bool) – Whether this asset’s computation can emit a subset of the asset
+ - config (Optional[Union[[*ConfigMapping*](config.mdx#dagster.ConfigMapping)], Mapping[str, Any]) –
+
+ Describes how the graph underlying the asset is configured at runtime.
+
+ If a [`ConfigMapping`](config.mdx#dagster.ConfigMapping) object is provided, then the graph takes on the config
+ schema of this object. The mapping will be applied at runtime to generate the config for
+ the graph’s constituent nodes.
+
+ If a dictionary is provided, then it will be used as the default run config for the
+ graph. This means it must conform to the config schema of the underlying nodes. Note
+ that the values provided will be viewable and editable in the Dagster UI, so be careful
+ with secrets.
+
+ If no value is provided, then the config schema for the graph is the default (derived
+
+
+
+
+
+
+
+
+
+
+
+
+## Multi-asset definitions
+
+Refer to the [Multi-asset](https://docs.dagster.io/guides/build/assets/defining-assets#multi-asset) documentation for more information.
+
+
+
@dagster.multi_asset
+
+
+ Create a combined definition of multiple assets that are computed using the same op and same
+ upstream assets.
+
+ Each argument to the decorated function references an upstream asset that this asset depends on.
+ The name of the argument designates the name of the upstream asset.
+
+ You can set I/O managers keys, auto-materialize policies, freshness policies, group names, etc.
+ on an individual asset within the multi-asset by attaching them to the [`AssetOut`](#dagster.AssetOut)
+ corresponding to that asset in the outs parameter.
+
+ Parameters:
+ - name (Optional[str]) – The name of the op.
+ - outs – (Optional[Dict[str, AssetOut]]): The AssetOuts representing the assets materialized by
+ - ins (Optional[Mapping[str, [*AssetIn*](#dagster.AssetIn)]]) – A dictionary that maps input names to information
+ - deps (Optional[Sequence[Union[[*AssetsDefinition*](#dagster.AssetsDefinition), [*SourceAsset*](#dagster.SourceAsset), [*AssetKey*](#dagster.AssetKey), str]]]) – The assets that are upstream dependencies, but do not correspond to a parameter of the
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The configuration schema for the asset’s underlying
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the underlying op.
+ - internal_asset_deps (Optional[Mapping[str, Set[[*AssetKey*](#dagster.AssetKey)]]]) – By default, it is assumed
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – The backfill policy for the op that computes the asset.
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset.
+ - can_subset (bool) – If this asset’s computation can emit a subset of the asset
+ - resource_defs (Optional[Mapping[str, object]]) – experimental
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. This
+ - retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that computes the asset.
+ - code_version (Optional[str]) – Version of the code encapsulated by the multi-asset. If set,
+ - specs (Optional[Sequence[[*AssetSpec*](#dagster.AssetSpec)]]) – The specifications for the assets materialized
+ - check_specs (Optional[Sequence[[*AssetCheckSpec*](asset-checks.mdx#dagster.AssetCheckSpec)]]) – Specs for asset checks that
+ - pool (Optional[str]) – A string that identifies the concurrency pool that governs this
+ - non_argument_deps (Optional[Union[Set[[*AssetKey*](#dagster.AssetKey)], Set[str]]]) – deprecateddeps instead.) Deprecated, use deps instead.
+
+
+ Examples:
+
+ ```python
+ @multi_asset(
+ specs=[
+ AssetSpec("asset1", deps=["asset0"]),
+ AssetSpec("asset2", deps=["asset0"]),
+ ]
+ )
+ def my_function():
+ asset0_value = load(path="asset0")
+ asset1_result, asset2_result = do_some_transformation(asset0_value)
+ write(asset1_result, path="asset1")
+ write(asset2_result, path="asset2")
+
+ # Or use IO managers to handle I/O:
+ @multi_asset(
+ outs={
+ "asset1": AssetOut(),
+ "asset2": AssetOut(),
+ }
+ )
+ def my_function(asset0):
+ asset1_value = do_some_transformation(asset0)
+ asset2_value = do_some_other_transformation(asset0)
+ return asset1_value, asset2_value
+ ```
+
+
+
+
+
+
@dagster.multi_observable_source_asset
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Defines a set of assets that can be observed together with the same function.
+
+ Parameters:
+ - name (Optional[str]) – The name of the op.
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - can_subset (bool) – If this asset’s computation can emit a subset of the asset
+ - resource_defs (Optional[Mapping[str, object]]) – (Experimental) A mapping of resource keys to resources. These resources
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. This
+ - specs (Optional[Sequence[[*AssetSpec*](#dagster.AssetSpec)]]) – (Experimental) The specifications for the assets
+ - check_specs (Optional[Sequence[[*AssetCheckSpec*](asset-checks.mdx#dagster.AssetCheckSpec)]]) – (Experimental) Specs for asset checks that
+
+
+ Examples:
+
+ ```python
+ @multi_observable_source_asset(
+ specs=[AssetSpec("asset1"), AssetSpec("asset2")],
+ )
+ def my_function():
+ yield ObserveResult(asset_key="asset1", metadata={"foo": "bar"})
+ yield ObserveResult(asset_key="asset2", metadata={"baz": "qux"})
+ ```
+
+
+
+
+
+
class dagster.AssetOut
+
+
+ Defines one of the assets produced by a [`@multi_asset`](#dagster.multi_asset).
+
+
+
static from_spec
+
+
+ Builds an AssetOut from the passed spec.
+
+ Parameters:
+ - spec ([*AssetSpec*](#dagster.AssetSpec)) – The spec to build the AssetOut from.
+ - dagster_type (Optional[Union[Type, [*DagsterType*](types.mdx#dagster.DagsterType)]]) – The type of this output. Should only
+ - is_required (bool) – Whether the presence of this field is required. (default: True)
+ - io_manager_key (Optional[str]) – The resource key of the IO manager used for this output.
+ - backfill_policy (Optional[[*BackfillPolicy*](partitions.mdx#dagster.BackfillPolicy)]) – BackfillPolicy to apply to the specified
+
+
+ Returns: The AssetOut built from the spec.Return type: [AssetOut](#dagster.AssetOut)
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Source assets
+
+Refer to the [External asset dependencies](https://docs.dagster.io/guides/build/assets/external-assets) documentation for more information.
+
+
+
class dagster.SourceAsset
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.0.
+ Use AssetSpec instead. If using the SourceAsset io_manager_key property, use AssetSpec(...).with_io_manager_key(...)..
+
+ :::
+
+ A SourceAsset represents an asset that will be loaded by (but not updated by) Dagster.
+
+
+
key
+
+
+ The key of the asset.
+
+ Type: Union[[AssetKey](#dagster.AssetKey), Sequence[str], str]
+
+
+
+
+
+
metadata
+
+
+ Metadata associated with the asset.
+
+ Type: Mapping[str, [MetadataValue](metadata.mdx#dagster.MetadataValue)]
+
+
+
+
+
+
io_manager_key
+
+
+ The key for the IOManager that will be used to load the contents of
+ the asset when it’s used as an input to other assets inside a job.
+
+ Type: Optional[str]
+
+
+
+
+
+
io_manager_def
+
+
+ (Experimental) The definition of the IOManager that will be used to load the contents of
+ the asset when it’s used as an input to other assets inside a job.
+
+ Type: Optional[[IOManagerDefinition](io-managers.mdx#dagster.IOManagerDefinition)]
+
+
+
+
+
+
resource_defs
+
+
+ (Experimental) resource definitions that may be required by the [`dagster.IOManagerDefinition`](io-managers.mdx#dagster.IOManagerDefinition) provided in the io_manager_def argument.
+
+ Type: Optional[Mapping[str, [ResourceDefinition](resources.mdx#dagster.ResourceDefinition)]]
+
+
+
+
+
+
description
+
+
+ The description of the asset.
+
+ Type: Optional[str]
+
+
+
+
+
+
partitions_def
+
+
+ Defines the set of partition keys that
+ compose the asset.
+
+ Type: Optional[[PartitionsDefinition](partitions.mdx#dagster.PartitionsDefinition)]
+
+
+
+ A dictionary of tags for the op that computes the asset.
+ Frameworks may expect and require certain metadata to be attached to a op. Values that
+ are not strings will be json encoded and must meet the criteria that
+ json.loads(json.dumps(value)) == value.
+
+ Type: Optional[Dict[str, Any]]
+
+
+
+
+
+
auto_observe_interval_minutes
+
+
+ While the asset daemon is turned on, a run
+ of the observation function for this asset will be launched at this interval. observe_fn
+ must be provided.
+
+ Type: Optional[float]
+
+
+
+
+
+
freshness_policy
+
+
+ A constraint telling Dagster how often this asset is intended to be updated
+ with respect to its root data.
+
+ Type: FreshnessPolicy
+
+
+
+
+
+
tags
+
+
+ Tags for filtering and organizing. These tags are not
+ attached to runs of the asset.
+
+ Type: Optional[Mapping[str, str]]
+
+
+
+
+
+
property is_observable
+
+
+ Whether the asset is observable.
+
+ Type: bool
+
+
+
+
+
+
property op
+
+
+ The OpDefinition associated with the observation function of an observable
+ source asset.
+
+ Throws an error if the asset is not observable.
+
+ Type: [OpDefinition](ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
+
+
+
+
@dagster.observable_source_asset
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Create a SourceAsset with an associated observation function.
+
+ The observation function of a source asset is wrapped inside of an op and can be executed as
+ part of a job. Each execution generates an AssetObservation event associated with the source
+ asset. The source asset observation function should return a `DataVersion`,
+ a ~dagster.DataVersionsByPartition, or an [`ObserveResult`](#dagster.ObserveResult).
+
+ Parameters:
+ - name (Optional[str]) – The name of the source asset. If not provided, defaults to the name of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the source asset’s key is the
+ - metadata (Mapping[str, RawMetadataValue]) – Metadata associated with the asset.
+ - io_manager_key (Optional[str]) – The key for the IOManager that will be used to load the contents of
+ - io_manager_def (Optional[[*IOManagerDefinition*](io-managers.mdx#dagster.IOManagerDefinition)]) – (Experimental) The definition of the IOManager that will be used to load the contents of
+ - description (Optional[str]) – The description of the asset.
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided,
+ - required_resource_keys (Optional[Set[str]]) – Set of resource keys required by the observe op.
+ - resource_defs (Optional[Mapping[str, [*ResourceDefinition*](resources.mdx#dagster.ResourceDefinition)]]) – (Experimental) resource
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset.
+ - tags (Optional[Mapping[str, str]]) – Tags for filtering and organizing. These tags are not
+ - observe_fn (Optional[SourceAssetObserveFunction]) – Observation function for the source asset.
+ - automation_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – A condition describing when Dagster
+
+
+
+
+
+
+
+
class dagster.ObserveResult
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ An object representing a successful observation of an asset. These can be returned from an
+ @observable_source_asset decorated function to pass metadata.
+
+
+
asset_key
+
+
+ The asset key. Optional to include.
+
+ Type: Optional[[AssetKey](#dagster.AssetKey)]
+
+
+
+
+
+
metadata
+
+
+ Metadata to record with the corresponding
+ AssetObservation event.
+
+ Type: Optional[RawMetadataMapping]
+
+
+
+
+
+
check_results
+
+
+ Check results to record with the
+ corresponding AssetObservation event.
+
+ Type: Optional[Sequence[[AssetCheckResult](asset-checks.mdx#dagster.AssetCheckResult)]]
+
+
+
+
+
+
data_version
+
+
+ The data version of the asset that was observed.
+
+ Type: Optional[DataVersion]
+
+
+
+
+
+
tags
+
+
+ Tags to record with the corresponding AssetObservation
+ event.
+
+ Type: Optional[Mapping[str, str]]
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Dependencies
+
+
+
class dagster.AssetDep
+
+
+ Specifies a dependency on an upstream asset.
+
+
+
asset
+
+
+ The upstream asset to depend on.
+
+ Type: Union[[AssetKey](#dagster.AssetKey), str, [AssetSpec](#dagster.AssetSpec), [AssetsDefinition](#dagster.AssetsDefinition), [SourceAsset](#dagster.SourceAsset)]
+
+
+
+
+
+
partition_mapping
+
+
+ Defines what partitions to depend on in
+ the upstream asset. If not provided and the upstream asset is partitioned, defaults to
+ the default partition mapping for the partitions definition, which is typically maps
+ partition keys to the same partition keys in upstream assets.
+
+ Type: Optional[[PartitionMapping](partitions.mdx#dagster.PartitionMapping)]
+
+
+
+ If provided, the asset’s key is the
+ concatenation of the key_prefix and the input name. Only one of the “key_prefix” and
+ “key” arguments should be provided.
+
+ Type: Optional[Union[str, Sequence[str]]]
+
+
+
+
+
+
key
+
+
+ The asset’s key. Only one of the
+ “key_prefix” and “key” arguments should be provided.
+
+ Type: Optional[Union[str, Sequence[str], [AssetKey](#dagster.AssetKey)]]
+
+
+
+
+
+
metadata
+
+
+ A dict of the metadata for the input.
+ For example, if you only need a subset of columns from an upstream table, you could
+ include that in metadata and the IO manager that loads the upstream table could use the
+ metadata to determine which columns to load.
+
+ Type: Optional[Dict[str, Any]]
+
+
+
+
+
+
partition_mapping
+
+
+ Defines what partitions to depend on in
+ the upstream asset. If not provided, defaults to the default partition mapping for the
+ partitions definition, which is typically maps partition keys to the same partition keys
+ in upstream assets.
+
+ Type: Optional[[PartitionMapping](partitions.mdx#dagster.PartitionMapping)]
+
+
+
+
+
+
dagster_type
+
+
+ Allows specifying type validation functions that
+ will be executed on the input of the decorated function before it runs.
+
+ Type: [DagsterType](types.mdx#dagster.DagsterType)
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Asset jobs
+
+[Asset jobs](https://docs.dagster.io/guides/build/assets/asset-jobs) enable the automation of asset materializations. Dagster’s [asset selection syntax](https://docs.dagster.io/guides/build/assets/asset-selection-syntax) can be used to select assets and assign them to a job.
+
+
+
dagster.define_asset_job
+
+
+ Creates a definition of a job which will either materialize a selection of assets or observe
+ a selection of source assets. This will only be resolved to a JobDefinition once placed in a
+ code location.
+
+ Parameters:
+ - name (str) – The name for the job.
+ - selection (Union[str, Sequence[str], Sequence[[*AssetKey*](#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](#dagster.AssetsDefinition), [*SourceAsset*](#dagster.SourceAsset)]], [*AssetSelection*](#dagster.AssetSelection)]) –
+
+ The assets that will be materialized or observed when the job is run.
+
+ The selected assets must all be included in the assets that are passed to the assets
+ argument of the Definitions object that this job is included on.
+
+ The string “my_asset*” selects my_asset and all downstream assets within the code
+ location. A list of strings represents the union of all assets selected by strings
+ within the list.
+
+ - config –
+
+ Describes how the Job is parameterized at runtime.
+
+ If no value is provided, then the schema for the job’s run config is a standard
+ format based on its ops and resources.
+
+ If a dictionary is provided, then it must conform to the standard config schema, and
+ it will be used as the job’s run config for the job whenever the job is executed.
+ The values provided will be viewable and editable in the Dagster UI, so be
+ careful with secrets.
+
+ - tags (Optional[Mapping[str, object]]) – A set of key-value tags that annotate the job and can
+ - run_tags (Optional[Mapping[str, object]]) – A set of key-value tags that will be automatically attached to runs launched by this
+ - metadata (Optional[Mapping[str, RawMetadataValue]]) – Arbitrary metadata about the job.
+ - description (Optional[str]) – A description for the Job.
+ - executor_def (Optional[[*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]) – How this Job will be executed. Defaults to [`multi_or_in_process_executor`](execution.mdx#dagster.multi_or_in_process_executor),
+ - op_retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The default retry policy for all ops that compute assets in this job.
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – deprecated
+
+
+ Returns: The job, which can be placed inside a code location.Return type: UnresolvedAssetJobDefinition
+ Examples:
+
+ ```python
+ # A job that targets all assets in the code location:
+ @asset
+ def asset1():
+ ...
+
+ defs = Definitions(
+ assets=[asset1],
+ jobs=[define_asset_job("all_assets")],
+ )
+
+ # A job that targets a single asset
+ @asset
+ def asset1():
+ ...
+
+ defs = Definitions(
+ assets=[asset1],
+ jobs=[define_asset_job("all_assets", selection=[asset1])],
+ )
+
+ # A job that targets all the assets in a group:
+ defs = Definitions(
+ assets=assets,
+ jobs=[define_asset_job("marketing_job", selection=AssetSelection.groups("marketing"))],
+ )
+
+ @observable_source_asset
+ def source_asset():
+ ...
+
+ # A job that observes a source asset:
+ defs = Definitions(
+ assets=assets,
+ jobs=[define_asset_job("observation_job", selection=[source_asset])],
+ )
+
+ # Resources are supplied to the assets, not the job:
+ @asset(required_resource_keys={"slack_client"})
+ def asset1():
+ ...
+
+ defs = Definitions(
+ assets=[asset1],
+ jobs=[define_asset_job("all_assets")],
+ resources={"slack_client": prod_slack_client},
+ )
+ ```
+
+
+
+
+
+
class dagster.AssetSelection
+
+
+ An AssetSelection defines a query over a set of assets and asset checks, normally all that are defined in a code location.
+
+ You can use the “|”, “&”, and “-” operators to create unions, intersections, and differences of selections, respectively.
+
+ AssetSelections are typically used with [`define_asset_job()`](#dagster.define_asset_job).
+
+ By default, selecting assets will also select all of the asset checks that target those assets.
+
+ Examples:
+
+ ```python
+ # Select all assets in group "marketing":
+ AssetSelection.groups("marketing")
+
+ # Select all assets in group "marketing", as well as the asset with key "promotion":
+ AssetSelection.groups("marketing") | AssetSelection.assets("promotion")
+
+ # Select all assets in group "marketing" that are downstream of asset "leads":
+ AssetSelection.groups("marketing") & AssetSelection.assets("leads").downstream()
+
+ # Select a list of assets:
+ AssetSelection.assets(*my_assets_list)
+
+ # Select all assets except for those in group "marketing"
+ AssetSelection.all() - AssetSelection.groups("marketing")
+
+ # Select all assets which are materialized by the same op as "projections":
+ AssetSelection.assets("projections").required_multi_asset_neighbors()
+
+ # Select all assets in group "marketing" and exclude their asset checks:
+ AssetSelection.groups("marketing") - AssetSelection.all_asset_checks()
+
+ # Select all asset checks that target a list of assets:
+ AssetSelection.checks_for_assets(*my_assets_list)
+
+ # Select a specific asset check:
+ AssetSelection.checks(my_asset_check)
+ ```
+
+
static all
+
+
+ Returns a selection that includes all assets and their asset checks.
+
+ Parameters: include_sources (bool) – experimental
+
+
+
+
+
+
static all_asset_checks
+
+ Returns a selection that includes all asset checks.
+
+
+
+
+
static assets
+
+
+ Returns a selection that includes all of the provided assets and asset checks that target
+ them.
+
+ Parameters: *assets_defs (Union[[*AssetsDefinition*](#dagster.AssetsDefinition), str, Sequence[str], [*AssetKey*](#dagster.AssetKey)]) – The assets to
+ select.
+ Examples:
+
+ ```python
+ AssetSelection.assets(AssetKey(["a"]))
+
+ AssetSelection.assets("a")
+
+ AssetSelection.assets(AssetKey(["a"]), AssetKey(["b"]))
+
+ AssetSelection.assets("a", "b")
+
+ @asset
+ def asset1():
+ ...
+
+ AssetSelection.assets(asset1)
+
+ asset_key_list = [AssetKey(["a"]), AssetKey(["b"])]
+ AssetSelection.assets(*asset_key_list)
+ ```
+
+
+
+
+
+
static checks
+
+ Returns a selection that includes all of the provided asset checks or check keys.
+
+
+
+
+
static checks_for_assets
+
+
+ Returns a selection with the asset checks that target the provided assets.
+
+ Parameters: *assets_defs (Union[[*AssetsDefinition*](#dagster.AssetsDefinition), str, Sequence[str], [*AssetKey*](#dagster.AssetKey)]) – The assets to
+ select checks for.
+
+
+
+
+
+
downstream
+
+
+ Returns a selection that includes all assets that are downstream of any of the assets in
+ this selection, selecting the assets in this selection by default. Includes the asset checks targeting the returned assets. Iterates through each
+ asset in this selection and returns the union of all downstream assets.
+
+ depth (Optional[int]): If provided, then only include assets to the given depth. A depth
+ of 2 means all assets that are children or grandchildren of the assets in this
+ selection.
+
+ include_self (bool): If True, then include the assets in this selection in the result.
+ If the include_self flag is False, return each downstream asset that is not part of the
+ original selection. By default, set to True.
+
+
+
+
+
+
+
static groups
+
+
+ Returns a selection that includes materializable assets that belong to any of the
+ provided groups and all the asset checks that target them.
+
+ Parameters: include_sources (bool) – If True, then include source assets matching the group in the
+ selection.
+
+
+
+
+
+
static key_prefixes
+
+
+ Returns a selection that includes assets that match any of the provided key prefixes and all the asset checks that target them.
+
+ Parameters: include_sources (bool) – If True, then include source assets matching the key prefix(es)
+ in the selection.
+ Examples:
+
+ ```python
+ # match any asset key where the first segment is equal to "a" or "b"
+ # e.g. AssetKey(["a", "b", "c"]) would match, but AssetKey(["abc"]) would not.
+ AssetSelection.key_prefixes("a", "b")
+
+ # match any asset key where the first two segments are ["a", "b"] or ["a", "c"]
+ AssetSelection.key_prefixes(["a", "b"], ["a", "c"])
+ ```
+
+
+
+
+
+
static keys
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use AssetSelection.assets instead..
+
+ :::
+
+ Returns a selection that includes assets with any of the provided keys and all asset
+ checks that target them.
+
+ Deprecated: use AssetSelection.assets instead.
+
+ Examples:
+
+ ```python
+ AssetSelection.keys(AssetKey(["a"]))
+
+ AssetSelection.keys("a")
+
+ AssetSelection.keys(AssetKey(["a"]), AssetKey(["b"]))
+
+ AssetSelection.keys("a", "b")
+
+ asset_key_list = [AssetKey(["a"]), AssetKey(["b"])]
+ AssetSelection.keys(*asset_key_list)
+ ```
+
+
+
+
+
+
materializable
+
+ Given an asset selection, returns a new asset selection that contains all of the assets
+ that are materializable. Removes any assets which are not materializable.
+
+
+
+
+
required_multi_asset_neighbors
+
+ Given an asset selection in which some assets are output from a multi-asset compute op
+ which cannot be subset, returns a new asset selection that contains all of the assets
+ required to execute the original asset selection. Includes the asset checks targeting the returned assets.
+
+
+
+
+
roots
+
+
+ Given an asset selection, returns a new asset selection that contains all of the root
+ assets within the original asset selection. Includes the asset checks targeting the returned assets.
+
+ A root asset is an asset that has no upstream dependencies within the asset selection.
+ The root asset can have downstream dependencies outside of the asset selection.
+
+ Because mixed selections of source and materializable assets are currently not supported,
+ keys corresponding to SourceAssets will not be included as roots. To select source assets,
+ use the upstream_source_assets method.
+
+
+
+
+
+
+
sinks
+
+
+ Given an asset selection, returns a new asset selection that contains all of the sink
+ assets within the original asset selection. Includes the asset checks targeting the returned assets.
+
+ A sink asset is an asset that has no downstream dependencies within the asset selection.
+ The sink asset can have downstream dependencies outside of the asset selection.
+
+
+
+
+
+
+
sources
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use AssetSelection.roots instead..
+
+ :::
+
+ Given an asset selection, returns a new asset selection that contains all of the root
+ assets within the original asset selection. Includes the asset checks targeting the returned assets.
+
+ A root asset is a materializable asset that has no upstream dependencies within the asset
+ selection. The root asset can have downstream dependencies outside of the asset selection.
+
+ Because mixed selections of source and materializable assets are currently not supported,
+ keys corresponding to SourceAssets will not be included as roots. To select source assets,
+ use the upstream_source_assets method.
+
+
+
+
+
+
+
static tag
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a selection that includes materializable assets that have the provided tag, and
+ all the asset checks that target them.
+
+ Parameters: include_sources (bool) – If True, then include source assets matching the group in the
+ selection.
+
+
+
+
+
+
upstream
+
+
+ Returns a selection that includes all materializable assets that are upstream of any of
+ the assets in this selection, selecting the assets in this selection by default. Includes
+ the asset checks targeting the returned assets. Iterates through each asset in this
+ selection and returns the union of all upstream assets.
+
+ Because mixed selections of source and materializable assets are currently not supported,
+ keys corresponding to SourceAssets will not be included as upstream of regular assets.
+
+ Parameters:
+ - depth (Optional[int]) – If provided, then only include assets to the given depth. A depth
+ - include_self (bool) – If True, then include the assets in this selection in the result.
+
+
+
+
+
+
+
+
upstream_source_assets
+
+ Given an asset selection, returns a new asset selection that contains all of the source
+ assets that are parents of assets in the original selection. Includes the asset checks
+ targeting the returned assets.
+
+
+
+
+
without_checks
+
+ Removes all asset checks in the selection.
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Code locations
+
+Loading assets and asset jobs into a [code location](https://docs.dagster.io/guides/deploy/code-locations/) makes them available to Dagster tools like the UI, CLI, and GraphQL API.
+
+
+
dagster.load_assets_from_modules
+
+
+ Constructs a list of assets and source assets from the given modules.
+
+ Parameters:
+ - modules (Iterable[ModuleType]) – The Python modules to look for assets inside.
+ - group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies
+ - freshness_policy (Optional[FreshnessPolicy]) – FreshnessPolicy to apply to all the loaded
+ - automation_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – AutomationCondition to apply
+ - backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
+ - source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned
+
+
+ Returns: A list containing assets and source assets defined in the given modules.Return type: Sequence[Union[[AssetsDefinition](#dagster.AssetsDefinition), [SourceAsset](#dagster.SourceAsset)]]
+
+
+
+
+
+
dagster.load_assets_from_current_module
+
+
+ Constructs a list of assets, source assets, and cacheable assets from the module where
+ this function is called.
+
+ Parameters:
+ - group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies
+ - freshness_policy (Optional[FreshnessPolicy]) – FreshnessPolicy to apply to all the loaded
+ - automation_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – AutomationCondition to apply
+ - backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
+ - source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned
+
+
+ Returns: A list containing assets, source assets, and cacheable assets defined in the module.Return type: Sequence[Union[[AssetsDefinition](#dagster.AssetsDefinition), [SourceAsset](#dagster.SourceAsset), CachableAssetsDefinition]]
+
+
+
+
+
+
dagster.load_assets_from_package_module
+
+
+ Constructs a list of assets and source assets that includes all asset
+ definitions, source assets, and cacheable assets in all sub-modules of the given package module.
+
+ A package module is the result of importing a package.
+
+ Parameters:
+ - package_module (ModuleType) – The package module to looks for assets inside.
+ - group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies
+ - freshness_policy (Optional[FreshnessPolicy]) – FreshnessPolicy to apply to all the loaded
+ - automation_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – AutomationCondition to apply
+ - backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
+ - source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned
+
+
+ Returns: A list containing assets, source assets, and cacheable assets defined in the module.Return type: Sequence[Union[[AssetsDefinition](#dagster.AssetsDefinition), [SourceAsset](#dagster.SourceAsset), CacheableAssetsDefinition]]
+
+
+
+
+
+
dagster.load_assets_from_package_name
+
+
+ Constructs a list of assets, source assets, and cacheable assets that includes all asset
+ definitions and source assets in all sub-modules of the given package.
+
+ Parameters:
+ - package_name (str) – The name of a Python package to look for assets inside.
+ - group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies
+ - freshness_policy (Optional[FreshnessPolicy]) – FreshnessPolicy to apply to all the loaded
+ - auto_materialize_policy (Optional[AutoMaterializePolicy]) – AutoMaterializePolicy to apply
+ - backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
+ - source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned
+
+
+ Returns: A list containing assets, source assets, and cacheable assets defined in the module.Return type: Sequence[Union[[AssetsDefinition](#dagster.AssetsDefinition), [SourceAsset](#dagster.SourceAsset), CacheableAssetsDefinition]]
+
+
+
+
+
+
+
+
+
+
+## Observations
+
+Refer to the [Asset observation](https://docs.dagster.io/guides/build/assets/metadata-and-tags/asset-observations) documentation for more information.
+
+
+
class dagster.AssetObservation
+
+
+ Event that captures metadata about an asset at a point in time.
+
+ Parameters:
+ - asset_key (Union[str, List[str], [*AssetKey*](#dagster.AssetKey)]) – A key to identify the asset.
+ - partition (Optional[str]) – The name of a partition of the asset that the metadata
+ - tags (Optional[Mapping[str, str]]) – A mapping containing tags for the observation.
+ - metadata (Optional[Dict[str, Union[str, float, int, [*MetadataValue*](metadata.mdx#dagster.MetadataValue)]]]) – Arbitrary metadata about the asset. Keys are displayed string labels, and values are
+
+
+
+
+
+
+
+
+
+
+
+
+## Declarative Automation
+
+Refer to the [Declarative Automation](https://docs.dagster.io/guides/automate/declarative-automation/) documentation for more information.
+
+
+
class dagster.AutomationCondition
+
+
+ An AutomationCondition represents a condition of an asset that impacts whether it should be
+ automatically executed. For example, you can have a condition which becomes true whenever the
+ code version of the asset is changed, or whenever an upstream dependency is updated.
+
+ ```python
+ from dagster import AutomationCondition, asset
+
+ @asset(automation_condition=AutomationCondition.on_cron("0 0 * * *"))
+ def my_asset(): ...
+ ```
+ AutomationConditions may be combined together into expressions using a variety of operators.
+
+ ```python
+ from dagster import AssetSelection, AutomationCondition, asset
+
+ # any dependencies from the "important" group are missing
+ any_important_deps_missing = AutomationCondition.any_deps_match(
+ AutomationCondition.missing(),
+ ).allow(AssetSelection.groups("important"))
+
+ # there is a new code version for this asset since the last time it was requested
+ new_code_version = AutomationCondition.code_version_changed().since(
+ AutomationCondition.newly_requested()
+ )
+
+ # there is a new code version and no important dependencies are missing
+ my_condition = new_code_version & ~any_important_deps_missing
+
+ @asset(automation_condition=my_condition)
+ def my_asset(): ...
+ ```
+
+
static all_checks_match
+
+
+ Returns an AutomationCondition that is true for an asset partition if all of its checks
+ evaluate to True for the given condition.
+
+ Parameters:
+ - condition ([*AutomationCondition*](#dagster.AutomationCondition)) – The AutomationCondition that will be evaluated against
+ - blocking_only (bool) – Determines if this condition will only be evaluated against blocking
+
+
+
+
+
+
+
+
static all_deps_blocking_checks_passed
+
+
+ Returns an AutomationCondition that is true for any partition where all upstream
+ blocking checks have passed, or will be requested on this tick.
+
+ In-tick requests are allowed to enable creating runs that target both a parent with
+ blocking checks and a child. Even though the checks have not currently passed, if
+ they fail within the run, the run machinery will prevent the child from being
+ materialized.
+
+
+
+
+
+
+
static all_deps_match
+
+
+ Returns an AutomationCondition that is true for a if at least one partition
+ of the all of the target’s dependencies evaluate to True for the given condition.
+
+ Parameters: condition ([*AutomationCondition*](#dagster.AutomationCondition)) – The AutomationCondition that will be evaluated against
+ this target’s dependencies.
+
+
+
+
+
+
static all_deps_updated_since_cron
+
+ Returns an AutomatonCondition that is true if all of the target’s dependencies have
+ updated since the latest tick of the provided cron schedule.
+
+
+
+
+
static any_checks_match
+
+
+ Returns an AutomationCondition that is true for if at least one of the target’s
+ checks evaluate to True for the given condition.
+
+ Parameters:
+ - condition ([*AutomationCondition*](#dagster.AutomationCondition)) – The AutomationCondition that will be evaluated against
+ - blocking_only (bool) – Determines if this condition will only be evaluated against blocking
+
+
+
+
+
+
+
+
static any_deps_in_progress
+
+ Returns an AutomationCondition that is true if the target has at least one dependency
+ that is in progress.
+
+
+
+
+
static any_deps_match
+
+
+ Returns an AutomationCondition that is true for a if at least one partition
+ of the any of the target’s dependencies evaluate to True for the given condition.
+
+ Parameters: condition ([*AutomationCondition*](#dagster.AutomationCondition)) – The AutomationCondition that will be evaluated against
+ this target’s dependencies.
+
+
+
+
+
+
static any_deps_missing
+
+ Returns an AutomationCondition that is true if the target has at least one dependency
+ that is missing, and will not be requested on this tick.
+
+
+
+
+
static any_deps_updated
+
+
+ Returns an AutomationCondition that is true if the target has at least one dependency
+ that has updated since the previous tick, or will be requested on this tick.
+
+ Will ignore parent updates if the run that updated the parent also plans to update
+ the asset or check that this condition is applied to.
+
+
+
+
+
+
+
static any_downstream_conditions
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns an AutomationCondition which represents the union of all distinct downstream conditions.
+
+
+
+
+
+
+
static asset_matches
+
+ Returns an AutomationCondition that is true if this condition is true for the given key.
+
+
+
+
+
static backfill_in_progress
+
+ Returns an AutomationCondition that is true if the target is part of an in-progress backfill.
+
+
+
+
+
static check_failed
+
+ Returns an AutomationCondition that is true for an asset check if it has evaluated against
+ the latest materialization of an asset and failed.
+
+
+
+
+
static check_passed
+
+ Returns an AutomationCondition that is true for an asset check if it has evaluated against
+ the latest materialization of an asset and passed.
+
+
+
+
+
static code_version_changed
+
+ Returns an AutomationCondition that is true if the target’s code version has been changed
+ since the previous tick.
+
+
+
+
+
static cron_tick_passed
+
+ Returns an AutomationCondition that is whenever a cron tick of the provided schedule is passed.
+
+
+
+
+
static eager
+
+
+ Returns an AutomationCondition which will cause a target to be executed if any of
+ its dependencies update, and will execute missing partitions if they become missing
+ after this condition is applied to the target.
+
+ This will not execute targets that have any missing or in progress dependencies, or
+ are currently in progress.
+
+ For time partitioned assets, only the latest time partition will be considered.
+
+
+
+
+
+
+
static execution_failed
+
+ Returns an AutomationCondition that is true if the latest execution of the target failed.
+
+
+
+
+
static in_latest_time_window
+
+
+ Returns an AutomationCondition that is true when the target it is within the latest
+ time window.
+
+ Parameters: lookback_delta (Optional, datetime.timedelta) – If provided, the condition will
+ return all partitions within the provided delta of the end of the latest time window.
+ For example, if this is used on a daily-partitioned asset with a lookback_delta of
+ 48 hours, this will return the latest two partitions.
+
+
+
+
+
+
static in_progress
+
+ Returns an AutomationCondition that is true for an asset partition if it is part of an
+ in-progress run or backfill.
+
+
+
+
+
static initial_evaluation
+
+ Returns an AutomationCondition that is true on the first evaluation of the expression.
+
+
+
+
+
static missing
+
+ Returns an AutomationCondition that is true if the target has not been executed.
+
+
+
+
+
static newly_missing
+
+ Returns an AutomationCondition that is true on the tick that the target becomes missing.
+
+
+
+
+
static newly_requested
+
+ Returns an AutomationCondition that is true if the target was requested on the previous tick.
+
+
+
+
+
static newly_updated
+
+ Returns an AutomationCondition that is true if the target has been updated since the previous tick.
+
+
+
+
+
static on_cron
+
+
+ Returns an AutomationCondition which will cause a target to be executed on a given
+ cron schedule, after all of its dependencies have been updated since the latest
+ tick of that cron schedule.
+
+ For time partitioned assets, only the latest time partition will be considered.
+
+
+
+
+
+
+
static on_missing
+
+
+ Returns an AutomationCondition which will execute partitions of the target that
+ are added after this condition is applied to the asset.
+
+ This will not execute targets that have any missing dependencies.
+
+ For time partitioned assets, only the latest time partition will be considered.
+
+
+
+
+
+
+
replace
+
+
+ Replaces all instances of `old` across any sub-conditions with `new`.
+
+ If `old` is a string, then conditions with a label matching
+ that string will be replaced.
+
+ Parameters:
+ - old (Union[[*AutomationCondition*](#dagster.AutomationCondition), str]) – The condition to replace.
+ - new ([*AutomationCondition*](#dagster.AutomationCondition)) – The condition to replace with.
+
+
+
+
+
+
+
+
static run_in_progress
+
+ Returns an AutomationCondition that is true if the target is part of an in-progress run.
+
+
+
+
+
static will_be_requested
+
+ Returns an AutomationCondition that is true if the target will be requested this tick.
+
+
+
+
+
+
+
+
+
class dagster.AutomationConditionSensorDefinition
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Targets a set of assets and repeatedly evaluates all the AutomationConditions on all of
+ those assets to determine which to request runs for.
+
+ Parameters:
+ - name – The name of the sensor.
+ - target (Union[str, Sequence[str], Sequence[[*AssetKey*](#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](#dagster.AssetsDefinition), [*SourceAsset*](#dagster.SourceAsset)]], [*AssetSelection*](#dagster.AssetSelection)]) – A selection of assets to evaluate AutomationConditions of and request runs for.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - run_tags (Optional[Mapping[str, Any]]) – Tags that will be automatically attached to runs launched by this sensor.
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - minimum_interval_seconds (Optional[int]) – The frequency at which to try to evaluate the
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - emit_backfills (bool) – If set to True, will emit a backfill on any tick where more than one partition
+ - use_user_code_server (bool) – (experimental) If set to True, this sensor will be evaluated in the user
+ - default_condition (Optional[[*AutomationCondition*](#dagster.AutomationCondition)]) – (experimental) If provided, this condition will
+
+
+ Examples:
+
+ ```python
+ import dagster as dg
+
+ # automation condition sensor that defaults to running
+ defs1 = dg.Definitions(
+ assets=...,
+ sensors=[
+ dg.AutomationConditionSensorDefinition(
+ name="automation_condition_sensor",
+ target=dg.AssetSelection.all(),
+ default_status=dg.DefaultSensorStatus.RUNNING,
+ ),
+ ]
+ )
+
+ # one automation condition sensor per group
+ defs2 = dg.Definitions(
+ assets=...,
+ sensors=[
+ dg.AutomationConditionSensorDefinition(
+ name="raw_data_automation_condition_sensor",
+ target=dg.AssetSelection.groups("raw_data"),
+ ),
+ dg.AutomationConditionSensorDefinition(
+ name="ml_automation_condition_sensor",
+ target=dg.AssetSelection.groups("machine_learning"),
+ ),
+ ]
+ )
+ ```
+
+
+
+
+
+
+
+
+
+## Asset values
+
+
+
class dagster.AssetValueLoader
+
+
+ Caches resource definitions that are used to load asset values across multiple load
+ invocations.
+
+ Should not be instantiated directly. Instead, use
+ [`get_asset_value_loader()`](repositories.mdx#dagster.RepositoryDefinition.get_asset_value_loader).
+
+
+
load_asset_value
+
+
+ Loads the contents of an asset as a Python object.
+
+ Invokes load_input on the [`IOManager`](io-managers.mdx#dagster.IOManager) associated with the asset.
+
+ Parameters:
+ - asset_key (Union[[*AssetKey*](#dagster.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.
+ - input_definition_metadata (Optional[Dict[str, Any]]) – Input metadata to pass to the [`IOManager`](io-managers.mdx#dagster.IOManager)
+ - resource_config (Optional[Any]) – A dictionary of resource configurations to be passed
+
+
+ Returns: The contents of an asset as a Python object.
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/cli.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/cli.mdx
new file mode 100644
index 0000000000000..e90997e4ac7f0
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/cli.mdx
@@ -0,0 +1,1871 @@
+---
+title: 'dagster cli'
+title_meta: 'dagster cli API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dagster cli Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Dagster CLI
+
+
+
+
+## dagster asset
+
+Commands for working with Dagster assets.
+
+ ```shell
+ dagster asset [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
list
+
+ List assets
+
+
+
+
+
materialize
+
+ Execute a run to materialize a selection…
+
+
+
+
+
wipe
+
+ Eliminate asset key indexes from event logs.
+
+
+
+
+
wipe-partitions-status-cache
+
+ Clears the asset partitions status cache,…
+
+
+
+
+
+
+
+
+
+## dagster debug
+
+Commands for helping debug Dagster issues by dumping or loading artifacts from specific runs.
+
+This can be used to send a file to someone like the Dagster team who doesn’t have direct access
+to your instance to allow them to view the events and details of a specific run.
+
+Debug files can be viewed using dagster-webserver-debug cli.
+Debug files can also be downloaded from the Dagster UI.
+
+ ```shell
+ dagster debug [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
export
+
+ Export the relevant artifacts for a job…
+
+
+
+
+
import
+
+ Import the relevant artifacts from debug…
+
+
+
+
+
+
+
+## dagster definitions validate
+
+The dagster definitions validate command loads and validate your Dagster definitions using a Dagster instance.
+
+This command indicates which code locations contain errors, and which ones can be successfully loaded.
+Code locations containing errors are considered invalid, otherwise valid.
+
+When running, this command sets the environment variable DAGSTER_IS_DEFS_VALIDATION_CLI=1.
+This environment variable can be used to control the behavior of your code in validation mode.
+
+This command returns an exit code 1 when errors are found, otherwise an exit code 0.
+
+This command should be run in a Python environment where the dagster package is installed.
+
+ ```shell
+ dagster definitions validate [OPTIONS]
+ ```
+Options:
+
+
+
--log-format \
+
+
+ Format of the logs for dagster services
+
+ Default: `'colored'`Options: colored | json | rich
+
+
+
+
+
+
--log-level \
+
+
+ Set the log level for dagster services.
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
-w, --workspace \
+
+ Path to workspace file. Argument can be provided multiple times.
+
+
+
+
+
-f, --python-file \
+
+ Specify python file or files (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each file as a code location in the current python environment.
+
+
+
+
+
-m, --module-name \
+
+ Specify module or modules (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each module as a code location in the current python environment.
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-definitions-validate-f)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-definitions-validate-m)
+
+
+
+
+
+
+
+
+
+
+
+## dagster dev
+
+Start a local deployment of Dagster, including dagster-webserver running on localhost and the dagster-daemon running in the background
+
+ ```shell
+ dagster dev [OPTIONS]
+ ```
+Options:
+
+
+
-d, --working-directory \
+
+ Specify working directory to use when loading the repository or job
+
+
+
+
+
-m, --module-name \
+
+ Specify module or modules (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each module as a code location in the current python environment.
+
+
+
+
+
-f, --python-file \
+
+ Specify python file or files (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each file as a code location in the current python environment.
+
+
+
+
+
-w, --workspace \
+
+ Path to workspace file. Argument can be provided multiple times.
+
+
+
+
+
--code-server-log-level \
+
+
+ Set the log level for code servers spun up by dagster services.
+
+ Default: `'warning'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-level \
+
+
+ Set the log level for dagster services.
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-format \
+
+
+ Format of the logs for dagster services
+
+ Default: `'colored'`Options: colored | json | rich
+
+
+
+
+
+
-p, --port, --dagit-port \
+
+ Port to use for the Dagster webserver.
+
+
+
+
+
-h, --host, --dagit-host \
+
+ Host to use for the Dagster webserver.
+
+
+
+
+
--live-data-poll-rate \
+
+
+ Rate at which the dagster UI polls for updated asset data (in milliseconds)
+
+ Default: `'2000'`
+
+
+
+
+
+
--use-legacy-code-server-behavior
+
+ Use the legacy behavior of the daemon and webserver each starting up their own code server
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--working-directory`](#cmdoption-dagster-dev-d)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-dev-m)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-dev-f)
+
+
+
+
+
+
+
+
+
+
+
+
+
+## dagster instance
+
+Commands for working with the current Dagster instance.
+
+ ```shell
+ dagster instance [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
concurrency
+
+ Commands for working with the…
+
+
+
+
+
info
+
+ List the information about the current…
+
+
+
+
+
migrate
+
+ Automatically migrate an out of date…
+
+
+
+
+
reindex
+
+ Rebuild index over historical runs for…
+
+
+
+
+
+
+
+
+
+## dagster job
+
+Commands for working with Dagster jobs.
+
+ ```shell
+ dagster job [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
backfill
+
+ Backfill a partitioned job.
+
+
+
+
+
execute
+
+ Execute a job.
+
+
+
+
+
launch
+
+ Launch a job using the run launcher…
+
+
+
+
+
list
+
+ List the jobs in a repository.
+
+
+
+
+
print
+
+ Print a job.
+
+
+
+
+
scaffold_config
+
+ Scaffold the config for a job.
+
+
+
+
+
+
+
+
+
+## dagster run
+
+Commands for working with Dagster job runs.
+
+ ```shell
+ dagster run [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
delete
+
+ Delete a run by id and its associated…
+
+
+
+
+
list
+
+ List the runs in the current Dagster…
+
+
+
+
+
migrate-repository
+
+ Migrate the run history for a job from a…
+
+
+
+
+
wipe
+
+ Eliminate all run history and event logs.
+
+
+
+
+
+
+
+
+
+## dagster schedule
+
+Commands for working with Dagster schedules.
+
+ ```shell
+ dagster schedule [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
debug
+
+ Debug information about the scheduler.
+
+
+
+
+
list
+
+ List all schedules that correspond to a…
+
+
+
+
+
logs
+
+ Get logs for a schedule.
+
+
+
+
+
preview
+
+ Preview changes that will be performed by…
+
+
+
+
+
restart
+
+ Restart a running schedule.
+
+
+
+
+
start
+
+ Start an existing schedule.
+
+
+
+
+
stop
+
+ Stop an existing schedule.
+
+
+
+
+
wipe
+
+ Delete the schedule history and turn off…
+
+
+
+
+
+
+
+
+
+## dagster sensor
+
+Commands for working with Dagster sensors.
+
+ ```shell
+ dagster sensor [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
cursor
+
+ Set the cursor value for an existing sensor.
+
+
+
+
+
list
+
+ List all sensors that correspond to a…
+
+
+
+
+
preview
+
+ Preview an existing sensor execution.
+
+
+
+
+
start
+
+ Start an existing sensor.
+
+
+
+
+
stop
+
+ Stop an existing sensor.
+
+
+
+
+
+
+
+
+
+## dagster project
+
+Commands for bootstrapping new Dagster projects and code locations.
+
+ ```shell
+ dagster project [OPTIONS] COMMAND [ARGS]...
+ ```
+Commands:
+
+
+
from-example
+
+ Download one of the official Dagster examples to the current directory. This CLI enables you to quickly bootstrap your project with an officially maintained example.
+
+
+
+
+
list-examples
+
+ List the examples that available to bootstrap with.
+
+
+
+
+
scaffold
+
+ Create a folder structure with a single Dagster code location and other files such as pyproject.toml. This CLI enables you to quickly start building a new Dagster project with everything set up.
+
+
+
+
+
scaffold-code-location
+
+ (DEPRECATED; Use dagster project scaffold –excludes README.md instead) Create a folder structure with a single Dagster code location, in the current directory. This CLI helps you to scaffold a new Dagster code location within a folder structure that includes multiple Dagster code locations.
+
+
+
+
+
scaffold-repository
+
+ (DEPRECATED; Use dagster project scaffold –excludes README.md instead) Create a folder structure with a single Dagster repository, in the current directory. This CLI helps you to scaffold a new Dagster repository within a folder structure that includes multiple Dagster repositories
+
+
+
+
+
+
+
+## dagster-graphql
+
+Run a GraphQL query against the dagster interface to a specified repository or pipeline/job.
+
+Can only use ONE of –workspace/-w, –python-file/-f, –module-name/-m, –grpc-port, –grpc-socket.
+
+Examples:
+
+1. dagster-graphql
+2. dagster-graphql -y path/to/workspace.yaml
+3. dagster-graphql -f path/to/file.py -a define_repo
+4. dagster-graphql -m some_module -a define_repo
+5. dagster-graphql -f path/to/file.py -a define_pipeline
+6. dagster-graphql -m some_module -a define_pipeline
+ ```shell
+ dagster-graphql [OPTIONS]
+ ```
+Options:
+
+
+
--version
+
+ Show the version and exit.
+
+
+
+
+
-t, --text \
+
+ GraphQL document to execute passed as a string
+
+
+
+
+
-f, --file \
+
+ GraphQL document to execute passed as a file
+
+
+
+
+
-p, --predefined \
+
+
+ GraphQL document to execute, from a predefined set provided by dagster-graphql.
+
+ Options: launchPipelineExecution
+
+
+
+
+
+
-v, --variables \
+
+ A JSON encoded string containing the variables for GraphQL execution.
+
+
+
+
+
-r, --remote \
+
+ A URL for a remote instance running dagster-webserver to send the GraphQL request to.
+
+
+
+
+
-o, --output \
+
+ A file path to store the GraphQL response to. This flag is useful when making pipeline/job execution queries, since pipeline/job execution causes logs to print to stdout and stderr.
+
+
+
+
+
--ephemeral-instance
+
+ Use an ephemeral DagsterInstance instead of resolving via DAGSTER_HOME
+
+
+
+
+
--empty-workspace
+
+ Allow an empty workspace
+
+
+
+
+
-w, --workspace \
+
+ Path to workspace file. Argument can be provided multiple times.
+
+
+
+
+
-d, --working-directory \
+
+ Specify working directory to use when loading the repository or job
+
+
+
+
+
-f, --python-file \
+
+ Specify python file or files (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each file as a code location in the current python environment.
+
+
+
+
+
-m, --module-name \
+
+ Specify module or modules (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each module as a code location in the current python environment.
+
+
+
+
+
--package-name \
+
+ Specify Python package where repository or job function lives
+
+
+
+
+
-a, --attribute \
+
+ Attribute that is either a 1) repository or job or 2) a function that returns a repository or job
+
+
+
+
+
--grpc-port \
+
+ Port to use to connect to gRPC server
+
+
+
+
+
--grpc-socket \
+
+ Named socket to use to connect to gRPC server
+
+
+
+
+
--grpc-host \
+
+ Host to use to connect to gRPC server, defaults to localhost
+
+
+
+
+
--use-ssl
+
+ Use a secure channel when connecting to the gRPC server
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--working-directory`](#cmdoption-dagster-graphql-d)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-graphql-0)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-graphql-m)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PACKAGE_NAME
+
+ >
+
+ Provide a default for [`--package-name`](#cmdoption-dagster-graphql-package-name)
+
+
+
+
+
+
+
+
+
+
DAGSTER_ATTRIBUTE
+
+ >
+
+ Provide a default for [`--attribute`](#cmdoption-dagster-graphql-a)
+
+
+
+
+
+
+
+
+
+
+
+## dagster-webserver
+
+Run dagster-webserver. Loads a code location.
+
+Can only use ONE of –workspace/-w, –python-file/-f, –module-name/-m, –grpc-port, –grpc-socket.
+
+Examples:
+
+1. dagster-webserver (works if ./workspace.yaml exists)
+2. dagster-webserver -w path/to/workspace.yaml
+3. dagster-webserver -f path/to/file.py
+4. dagster-webserver -f path/to/file.py -d path/to/working_directory
+5. dagster-webserver -m some_module
+6. dagster-webserver -f path/to/file.py -a define_repo
+7. dagster-webserver -m some_module -a define_repo
+8. dagster-webserver -p 3333
+Options can also provide arguments via environment variables prefixed with DAGSTER_WEBSERVER.
+
+For example, DAGSTER_WEBSERVER_PORT=3333 dagster-webserver
+
+ ```shell
+ dagster-webserver [OPTIONS]
+ ```
+Options:
+
+
+
--use-ssl
+
+ Use a secure channel when connecting to the gRPC server
+
+
+
+
+
--grpc-host \
+
+ Host to use to connect to gRPC server, defaults to localhost
+
+
+
+
+
--grpc-socket \
+
+ Named socket to use to connect to gRPC server
+
+
+
+
+
--grpc-port \
+
+ Port to use to connect to gRPC server
+
+
+
+
+
-a, --attribute \
+
+ Attribute that is either a 1) repository or job or 2) a function that returns a repository or job
+
+
+
+
+
--package-name \
+
+ Specify Python package where repository or job function lives
+
+
+
+
+
-m, --module-name \
+
+ Specify module or modules (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each module as a code location in the current python environment.
+
+
+
+
+
-f, --python-file \
+
+ Specify python file or files (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each file as a code location in the current python environment.
+
+
+
+
+
-d, --working-directory \
+
+ Specify working directory to use when loading the repository or job
+
+
+
+
+
-w, --workspace \
+
+ Path to workspace file. Argument can be provided multiple times.
+
+
+
+
+
--empty-workspace
+
+ Allow an empty workspace
+
+
+
+
+
-h, --host \
+
+
+ Host to run server on
+
+ Default: `'127.0.0.1'`
+
+
+
+
+
+
-p, --port \
+
+ Port to run server on - defaults to 3000
+
+
+
+
+
-l, --path-prefix \
+
+
+ The path prefix where server will be hosted (eg: /dagster-webserver)
+
+ Default: `''`
+
+
+
+
+
+
--db-statement-timeout \
+
+
+ The timeout in milliseconds to set on database statements sent to the DagsterInstance. Not respected in all configurations.
+
+ Default: `15000`
+
+
+
+
+
+
--db-pool-recycle \
+
+
+ The maximum age of a connection to use from the sqlalchemy pool without connection recycling. Set to -1 to disable. Not respected in all configurations.
+
+ Default: `3600`
+
+
+
+
+
+
--read-only
+
+ Start server in read-only mode, where all mutations such as launching runs and turning schedules on/off are turned off.
+
+
+
+
+
--suppress-warnings
+
+ Filter all warnings when hosting server.
+
+
+
+
+
--uvicorn-log-level, --log-level \
+
+
+ Set the log level for the uvicorn web server.
+
+ Default: `'warning'`Options: critical | error | warning | info | debug | trace
+
+
+
+
+
+
--dagster-log-level \
+
+
+ Set the log level for dagster log events.
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-format \
+
+
+ Format of the log output from the webserver
+
+ Default: `'colored'`Options: colored | json | rich
+
+
+
+
+
+
--code-server-log-level \
+
+
+ Set the log level for any code servers spun up by the webserver.
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--live-data-poll-rate \
+
+
+ Rate at which the dagster UI polls for updated asset data (in milliseconds)
+
+ Default: `2000`
+
+
+
+
+
+
--version
+
+ Show the version and exit.
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_ATTRIBUTE
+
+ >
+
+ Provide a default for [`--attribute`](#cmdoption-dagster-webserver-a)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PACKAGE_NAME
+
+ >
+
+ Provide a default for [`--package-name`](#cmdoption-dagster-webserver-package-name)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-webserver-m)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-webserver-f)
+
+
+
+
+
+
+
+
+
+
DAGSTER_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--working-directory`](#cmdoption-dagster-webserver-d)
+
+
+
+
+
+
+
+
+
+
DAGSTER_WEBSERVER_LOG_LEVEL
+
+ >
+
+ Provide a default for [`--dagster-log-level`](#cmdoption-dagster-webserver-dagster-log-level)
+
+
+
+
+
+
+
+
+
+
+
+## dagster-daemon run
+
+Run any daemons configured on the DagsterInstance.
+
+ ```shell
+ dagster-daemon run [OPTIONS]
+ ```
+Options:
+
+
+
--code-server-log-level \
+
+
+ Set the log level for any code servers spun up by the daemon.
+
+ Default: `'warning'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-level \
+
+
+ Set the log level for any code servers spun up by the daemon.
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-format \
+
+
+ Format of the log output from the webserver
+
+ Default: `'colored'`Options: colored | json | rich
+
+
+
+
+
+
--use-ssl
+
+ Use a secure channel when connecting to the gRPC server
+
+
+
+
+
--grpc-host \
+
+ Host to use to connect to gRPC server, defaults to localhost
+
+
+
+
+
--grpc-socket \
+
+ Named socket to use to connect to gRPC server
+
+
+
+
+
--grpc-port \
+
+ Port to use to connect to gRPC server
+
+
+
+
+
-a, --attribute \
+
+ Attribute that is either a 1) repository or job or 2) a function that returns a repository or job
+
+
+
+
+
--package-name \
+
+ Specify Python package where repository or job function lives
+
+
+
+
+
-m, --module-name \
+
+ Specify module or modules (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each module as a code location in the current python environment.
+
+
+
+
+
-f, --python-file \
+
+ Specify python file or files (flag can be used multiple times) where dagster definitions reside as top-level symbols/variables and load each file as a code location in the current python environment.
+
+
+
+
+
-d, --working-directory \
+
+ Specify working directory to use when loading the repository or job
+
+
+
+
+
-w, --workspace \
+
+ Path to workspace file. Argument can be provided multiple times.
+
+
+
+
+
--empty-workspace
+
+ Allow an empty workspace
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_DAEMON_LOG_LEVEL
+
+ >
+
+ Provide a default for [`--log-level`](#cmdoption-dagster-daemon-run-log-level)
+
+
+
+
+
+
+
+
+
+
DAGSTER_ATTRIBUTE
+
+ >
+
+ Provide a default for [`--attribute`](#cmdoption-dagster-daemon-run-a)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PACKAGE_NAME
+
+ >
+
+ Provide a default for [`--package-name`](#cmdoption-dagster-daemon-run-package-name)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-daemon-run-m)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-daemon-run-f)
+
+
+
+
+
+
+
+
+
+
DAGSTER_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--working-directory`](#cmdoption-dagster-daemon-run-d)
+
+
+
+
+
+## dagster api grpc
+
+Serve the Dagster inter-process API over GRPC
+
+ ```shell
+ dagster api grpc [OPTIONS]
+ ```
+Options:
+
+
+
-p, --port \
+
+ Port over which to serve. You must pass one and only one of –port/-p or –socket/-s.
+
+
+
+
+
-s, --socket \
+
+ Serve over a UDS socket. You must pass one and only one of –port/-p or –socket/-s.
+
+
+
+
+
-h, --host \
+
+ Hostname at which to serve. Default is localhost.
+
+
+
+
+
-n, --max-workers, --max_workers \
+
+ Maximum number of (threaded) workers to use in the GRPC server
+
+
+
+
+
--heartbeat
+
+ If set, the GRPC server will shut itself down when it fails to receive a heartbeat after a timeout configurable with –heartbeat-timeout.
+
+
+
+
+
--heartbeat-timeout \
+
+ Timeout after which to shutdown if –heartbeat is set and a heartbeat is not received
+
+
+
+
+
--lazy-load-user-code
+
+ Wait until the first LoadRepositories call to actually load the repositories, instead of waiting to load them when the server is launched. Useful for surfacing errors when the server is managed directly from the Dagster UI.
+
+
+
+
+
-a, --attribute \
+
+ Attribute that is either a 1) repository or job or 2) a function that returns a repository or job
+
+
+
+
+
--package-name \
+
+ Specify Python package where repository or job function lives
+
+
+
+
+
-m, --module-name \
+
+ Specify module where dagster definitions reside as top-level symbols/variables and load the module as a code location in the current python environment.
+
+
+
+
+
-f, --python-file \
+
+ Specify python file where dagster definitions reside as top-level symbols/variables and load the file as a code location in the current python environment.
+
+
+
+
+
-d, --working-directory \
+
+ Specify working directory to use when loading the repository or job
+
+
+
+
+
--use-python-environment-entry-point
+
+ If this flag is set, the server will signal to clients that they should launch dagster commands using \ -m dagster, instead of the default dagster entry point. This is useful when there are multiple Python environments running in the same machine, so a single dagster entry point is not enough to uniquely determine the environment.
+
+
+
+
+
--empty-working-directory
+
+ Indicates that the working directory should be empty and should not set to the current directory as a default
+
+
+
+
+
--fixed-server-id \
+
+ [INTERNAL] This option should generally not be used by users. Internal param used by dagster to spawn a gRPC server with the specified server id.
+
+
+
+
+
--log-level \
+
+
+ Level at which to log output from the code server process
+
+ Default: `'info'`Options: critical | error | warning | info | debug
+
+
+
+
+
+
--log-format \
+
+
+ Format of the log output from the code server process
+
+ Default: `'colored'`Options: colored | json | rich
+
+
+
+
+
+
--container-image \
+
+ Container image to use to run code from this server.
+
+
+
+
+
--container-context \
+
+ Serialized JSON with configuration for any containers created to run the code from this server.
+
+
+
+
+
--inject-env-vars-from-instance
+
+ Whether to load env vars from the instance and inject them into the environment.
+
+
+
+
+
--location-name \
+
+ Name of the code location this server corresponds to.
+
+
+
+
+
--instance-ref \
+
+ [INTERNAL] Serialized InstanceRef to use for accessing the instance
+
+
+
+
+
--enable-metrics
+
+ [INTERNAL] Retrieves current utilization metrics from GRPC server.
+
+
+
+Environment variables:
+
+
+
+
DAGSTER_GRPC_PORT
+
+ >
+
+ Provide a default for [`--port`](#cmdoption-dagster-api-grpc-p)
+
+
+
+
+
+
+
+
+
+
DAGSTER_GRPC_SOCKET
+
+ >
+
+ Provide a default for [`--socket`](#cmdoption-dagster-api-grpc-s)
+
+
+
+
+
+
+
+
+
+
DAGSTER_GRPC_HOST
+
+ >
+
+ Provide a default for [`--host`](#cmdoption-dagster-api-grpc-h)
+
+
+
+
+
+
+
+
+
+
DAGSTER_GRPC_MAX_WORKERS
+
+ >
+
+ Provide a default for [`--max-workers`](#cmdoption-dagster-api-grpc-n)
+
+
+
+
+
+
+
+
+
+
DAGSTER_LAZY_LOAD_USER_CODE
+
+ >
+
+ Provide a default for [`--lazy-load-user-code`](#cmdoption-dagster-api-grpc-lazy-load-user-code)
+
+
+
+
+
+
+
+
+
+
DAGSTER_ATTRIBUTE
+
+ >
+
+ Provide a default for [`--attribute`](#cmdoption-dagster-api-grpc-a)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PACKAGE_NAME
+
+ >
+
+ Provide a default for [`--package-name`](#cmdoption-dagster-api-grpc-package-name)
+
+
+
+
+
+
+
+
+
+
DAGSTER_MODULE_NAME
+
+ >
+
+ Provide a default for [`--module-name`](#cmdoption-dagster-api-grpc-m)
+
+
+
+
+
+
+
+
+
+
DAGSTER_PYTHON_FILE
+
+ >
+
+ Provide a default for [`--python-file`](#cmdoption-dagster-api-grpc-f)
+
+
+
+
+
+
+
+
+
+
DAGSTER_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--working-directory`](#cmdoption-dagster-api-grpc-d)
+
+
+
+
+
+
+
+
+
+
DAGSTER_USE_PYTHON_ENVIRONMENT_ENTRY_POINT
+
+ >
+
+ Provide a default for [`--use-python-environment-entry-point`](#cmdoption-dagster-api-grpc-use-python-environment-entry-point)
+
+
+
+
+
+
+
+
+
+
DAGSTER_EMPTY_WORKING_DIRECTORY
+
+ >
+
+ Provide a default for [`--empty-working-directory`](#cmdoption-dagster-api-grpc-empty-working-directory)
+
+
+
+
+
+
+
+
+
+
DAGSTER_CONTAINER_IMAGE
+
+ >
+
+ Provide a default for [`--container-image`](#cmdoption-dagster-api-grpc-container-image)
+
+
+
+
+
+
+
+
+
+
DAGSTER_CONTAINER_CONTEXT
+
+ >
+
+ Provide a default for [`--container-context`](#cmdoption-dagster-api-grpc-container-context)
+
+
+
+
+
+
+
+
+
+
DAGSTER_INJECT_ENV_VARS_FROM_INSTANCE
+
+ >
+
+ Provide a default for [`--inject-env-vars-from-instance`](#cmdoption-dagster-api-grpc-inject-env-vars-from-instance)
+
+
+
+
+
+
+
+
+
+
DAGSTER_LOCATION_NAME
+
+ >
+
+ Provide a default for [`--location-name`](#cmdoption-dagster-api-grpc-location-name)
+
+
+
+
+
+
+
+
+
+
DAGSTER_INSTANCE_REF
+
+ >
+
+ Provide a default for [`--instance-ref`](#cmdoption-dagster-api-grpc-instance-ref)
+
+
+
+
+
+
+
+
+
+
DAGSTER_ENABLE_SERVER_METRICS
+
+ >
+
+ Provide a default for [`--enable-metrics`](#cmdoption-dagster-api-grpc-enable-metrics)
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/config.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/config.mdx
new file mode 100644
index 0000000000000..a826f016526e1
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/config.mdx
@@ -0,0 +1,764 @@
+---
+title: 'config'
+title_meta: 'config API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'config Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Config
+
+
+
+
+## Pythonic config system
+
+The following classes are used as part of the new [Pythonic config system](https://docs.dagster.io/guides/operate/configuration/advanced-config-types). They are used in conjunction with builtin types.
+
+
+
class dagster.Config
+
+
+ Base class for Dagster configuration models, used to specify config schema for
+ ops and assets. Subclasses `pydantic.BaseModel`.
+
+ Example definition:
+
+ ```python
+ from pydantic import Field
+
+ class MyAssetConfig(Config):
+ my_str: str = "my_default_string"
+ my_int_list: List[int]
+ my_bool_with_metadata: bool = Field(default=False, description="A bool field")
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def asset_with_config(config: MyAssetConfig):
+ assert config.my_str == "my_default_string"
+ assert config.my_int_list == [1, 2, 3]
+ assert config.my_bool_with_metadata == False
+
+ asset_with_config(MyAssetConfig(my_int_list=[1, 2, 3], my_bool_with_metadata=True))
+ ```
+
+
+
+
+
+
class dagster.PermissiveConfig
+
+
+ Subclass of [`Config`](#dagster.Config) that allows arbitrary extra fields. This is useful for
+ config classes which may have open-ended inputs.
+
+ Example definition:
+
+ ```python
+ class MyPermissiveOpConfig(PermissiveConfig):
+ my_explicit_parameter: bool
+ my_other_explicit_parameter: str
+ ```
+ Example usage:
+
+ ```python
+ @op
+ def op_with_config(config: MyPermissiveOpConfig):
+ assert config.my_explicit_parameter == True
+ assert config.my_other_explicit_parameter == "foo"
+ assert config.dict().get("my_implicit_parameter") == "bar"
+
+ op_with_config(
+ MyPermissiveOpConfig(
+ my_explicit_parameter=True,
+ my_other_explicit_parameter="foo",
+ my_implicit_parameter="bar"
+ )
+ )
+ ```
+
+
+
+
+
+
class dagster.RunConfig
+
+
+ Container for all the configuration that can be passed to a run. Accepts Pythonic definitions
+ for op and asset config and resources and converts them under the hood to the appropriate config dictionaries.
+
+ Example usage:
+
+ ```python
+ class MyAssetConfig(Config):
+ a_str: str
+
+ @asset
+ def my_asset(config: MyAssetConfig):
+ assert config.a_str == "foo"
+
+ materialize(
+ [my_asset],
+ run_config=RunConfig(
+ ops={"my_asset": MyAssetConfig(a_str="foo")}
+ )
+ )
+ ```
+
+
to_config_dict
+
+
+ Converts the RunConfig to a dictionary representation.
+
+ Returns: The dictionary representation of the RunConfig.Return type: Dict[str, Any]
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Legacy Dagster config types
+
+The following types are used as part of the legacy [Dagster config system](https://docs.dagster.io/guides/operate/configuration/). They are used in conjunction with builtin types.
+
+
+
class dagster.ConfigSchema
+
+
+ Placeholder type for config schemas.
+
+ Any time that it appears in documentation, it means that any of the following types are
+ acceptable:
+
+ 1. A Python scalar type that resolves to a Dagster config type
+ (`python:int`, `python:float`, `python:bool`,
+ or `python:str`). For example:
+
+ - `@op(config_schema=int)`
+ - `@op(config_schema=str)`
+
+
+ 2. A built-in python collection (`python:list`, or `python:dict`).
+ `python:list` is exactly equivalent to [`Array`](#dagster.Array) [
+ `Any` ] and `python:dict` is equivalent to
+ [`Permissive`](#dagster.Permissive). For example:
+
+ - `@op(config_schema=list)`
+ - `@op(config_schema=dict)`
+
+
+ 3. A Dagster config type:
+
+ - `Any`
+ - [`Array`](#dagster.Array)
+ - `Bool`
+ - [`Enum`](#dagster.Enum)
+ - `Float`
+ - `Int`
+ - [`IntSource`](#dagster.IntSource)
+ - [`Noneable`](#dagster.Noneable)
+ - [`Permissive`](#dagster.Permissive)
+ - [`Map`](#dagster.Map)
+ - [`ScalarUnion`](#dagster.ScalarUnion)
+ - [`Selector`](#dagster.Selector)
+ - [`Shape`](#dagster.Shape)
+ - `String`
+ - [`StringSource`](#dagster.StringSource)
+
+
+ 4. A bare python dictionary, which will be automatically wrapped in
+ [`Shape`](#dagster.Shape). Values of the dictionary are resolved recursively
+ according to the same rules. For example:
+
+ - `\{'some_config': str}` is equivalent to `Shape(\{'some_config: str})`.
+ - `\{'some_config1': \{'some_config2': str}}` is equivalent to
+
+
+ 5. A bare python list of length one, whose single element will be wrapped in a
+ [`Array`](#dagster.Array) is resolved recursively according to the same
+ rules. For example:
+
+ - `[str]` is equivalent to `Array[str]`.
+ - `[[str]]` is equivalent to `Array[Array[str]]`.
+ - `[\{'some_config': str}]` is equivalent to `Array(Shape(\{'some_config: str}))`.
+
+
+ 6. An instance of [`Field`](#dagster.Field).
+
+
+
+
+
+
class dagster.Field
+
+
+ Defines the schema for a configuration field.
+
+ Fields are used in config schema instead of bare types when one wants to add a description,
+ a default value, or to mark it as not required.
+
+ Config fields are parsed according to their schemas in order to yield values available at
+ job execution time through the config system. Config fields can be set on ops, on
+ loaders for custom, and on other pluggable components of the system, such as resources, loggers,
+ and executors.
+
+ Parameters:
+ - config (Any) –
+
+ The schema for the config. This value can be any of:
+
+ 1. A Python primitive type that resolves to a Dagster config type
+ (`python:int`, `python:float`, `python:bool`,
+ `python:str`, or `python:list`).
+ 2. A Dagster config type:
+
+ - `Any`
+ - [`Array`](#dagster.Array)
+ - `Bool`
+ - [`Enum`](#dagster.Enum)
+ - `Float`
+ - `Int`
+ - [`IntSource`](#dagster.IntSource)
+ - [`Noneable`](#dagster.Noneable)
+ - [`Permissive`](#dagster.Permissive)
+ - [`ScalarUnion`](#dagster.ScalarUnion)
+ - [`Selector`](#dagster.Selector)
+ - [`Shape`](#dagster.Shape)
+ - `String`
+ - [`StringSource`](#dagster.StringSource)
+
+
+ 3. A bare python dictionary, which will be automatically wrapped in
+ [`Shape`](#dagster.Shape). Values of the dictionary are resolved recursively
+ according to the same rules.
+ 4. A bare python list of length one which itself is config type.
+ - default_value (Any) –
+
+ A default value for this field, conformant to the schema set by the `dagster_type`
+ argument. If a default value is provided, `is_required` should be `False`.
+
+ - is_required (bool) – Whether the presence of this field is required. Defaults to true. If `is_required`
+ - description (str) – A human-readable description of this config field.
+
+
+ Examples:
+
+ ```python
+ @op(
+ config_schema={
+ 'word': Field(str, description='I am a word.'),
+ 'repeats': Field(Int, default_value=1, is_required=False),
+ }
+ )
+ def repeat_word(context):
+ return context.op_config['word'] * context.op_config['repeats']
+ ```
+
+
property default_provided
+
+
+ Was a default value provided.
+
+ Returns: Yes or noReturn type: bool
+
+
+
+
+
+
property default_value
+
+
+ The default value for the field.
+
+ Raises an exception if no default value was provided.
+
+
+
+
+
+
+
property description
+
+ A human-readable description of this config field, if provided.
+
+
+
+
+
property is_required
+
+
+ Whether a value for this field must be provided at runtime.
+
+ Cannot be True if a default value is provided.
+
+
+
+
+
+
+
+
+
+
+
class dagster.Selector
+
+
+ Define a config field requiring the user to select one option.
+
+ Selectors are used when you want to be able to present several different options in config but
+ allow only one to be selected. For example, a single input might be read in from either a csv
+ file or a parquet file, but not both at once.
+
+ Note that in some other type systems this might be called an ‘input union’.
+
+ Functionally, a selector is like a `Dict`, except that only one key from the dict can
+ be specified in valid config.
+
+ Parameters: fields (Dict[str, [*Field*](#dagster.Field)]) – The fields from which the user must select.
+ Examples:
+
+ ```python
+ @op(
+ config_schema=Field(
+ Selector(
+ {
+ 'haw': {'whom': Field(String, default_value='honua', is_required=False)},
+ 'cn': {'whom': Field(String, default_value='世界', is_required=False)},
+ 'en': {'whom': Field(String, default_value='world', is_required=False)},
+ }
+ ),
+ is_required=False,
+ default_value={'en': {'whom': 'world'}},
+ )
+ )
+ def hello_world_with_default(context):
+ if 'haw' in context.op_config:
+ return 'Aloha {whom}!'.format(whom=context.op_config['haw']['whom'])
+ if 'cn' in context.op_config:
+ return '你好, {whom}!'.format(whom=context.op_config['cn']['whom'])
+ if 'en' in context.op_config:
+ return 'Hello, {whom}!'.format(whom=context.op_config['en']['whom'])
+ ```
+
+
+
+
+
+
class dagster.Permissive
+
+
+ Defines a config dict with a partially specified schema.
+
+ A permissive dict allows partial specification of the config schema. Any fields with a
+ specified schema will be type checked. Other fields will be allowed, but will be ignored by
+ the type checker.
+
+ Parameters: fields (Dict[str, [*Field*](#dagster.Field)]) – The partial specification of the config dict.
+ Examples:
+
+ ```python
+ @op(config_schema=Field(Permissive({'required': Field(String)})))
+ def map_config_op(context) -> List:
+ return sorted(list(context.op_config.items()))
+ ```
+
+
+
+
+
+
class dagster.Shape
+
+
+ Schema for configuration data with string keys and typed values via [`Field`](#dagster.Field).
+
+ Unlike [`Permissive`](#dagster.Permissive), unspecified fields are not allowed and will throw a
+ [`DagsterInvalidConfigError`](errors.mdx#dagster.DagsterInvalidConfigError).
+
+ Parameters:
+ - fields (Dict[str, [*Field*](#dagster.Field)]) – The specification of the config dict.
+ - field_aliases (Dict[str, str]) – Maps a string key to an alias that can be used instead of the original key. For example,
+
+
+
+
+
+
+
+
class dagster.Map
+
+
+ Defines a config dict with arbitrary scalar keys and typed values.
+
+ A map can contrain arbitrary keys of the specified scalar type, each of which has
+ type checked values. Unlike [`Shape`](#dagster.Shape) and [`Permissive`](#dagster.Permissive), scalar
+ keys other than strings can be used, and unlike [`Permissive`](#dagster.Permissive), all
+ values are type checked.
+
+ Parameters:
+ - key_type (type) – The type of keys this map can contain. Must be a scalar type.
+ - inner_type (type) – The type of the values that this map type can contain.
+ - key_label_name (string) – Optional name which describes the role of keys in the map.
+
+
+ Examples:
+
+ ```python
+ @op(config_schema=Field(Map({str: int})))
+ def partially_specified_config(context) -> List:
+ return sorted(list(context.op_config.items()))
+ ```
+
+
property key_label_name
+
+ Name which describes the role of keys in the map, if provided.
+
+
+
+
+
+
+
+
+
class dagster.Array
+
+
+ Defines an array (list) configuration type that contains values of type `inner_type`.
+
+ Parameters: inner_type (type) – The type of the values that this configuration type can contain.
+
+
property description
+
+ A human-readable description of this Array type.
+
+
+
+
+
+
+
+
+
class dagster.Noneable
+
+
+ Defines a configuration type that is the union of `NoneType` and the type `inner_type`.
+
+ Parameters: inner_type (type) – The type of the values that this configuration type can contain.
+ Examples:
+
+ ```python
+ config_schema={"name": Noneable(str)}
+
+ config={"name": "Hello"} # Ok
+ config={"name": None} # Ok
+ config={} # Error
+ ```
+
+
+
+
+
+
class dagster.Enum
+
+
+ Defines a enum configuration type that allows one of a defined set of possible values.
+
+ Parameters:
+ - name (str) – The name of the enum configuration type.
+ - enum_values (List[[*EnumValue*](#dagster.EnumValue)]) – The set of possible values for the enum configuration type.
+
+
+ Examples:
+
+ ```python
+ @op(
+ config_schema=Field(
+ Enum(
+ 'CowboyType',
+ [
+ EnumValue('good'),
+ EnumValue('bad'),
+ EnumValue('ugly'),
+ ]
+ )
+ )
+ )
+ def resolve_standoff(context):
+ # ...
+ ```
+
+
+
+
+
+
class dagster.EnumValue
+
+
+ Define an entry in a [`Enum`](#dagster.Enum).
+
+ Parameters:
+ - config_value (str) – The string representation of the config to accept when passed.
+ - python_value (Optional[Any]) – The python value to convert the enum entry in to. Defaults to the `config_value`.
+ - description (Optional[str]) – A human-readable description of the enum entry.
+
+
+
+
+
+
+
+
class dagster.ScalarUnion
+
+
+ Defines a configuration type that accepts a scalar value OR a non-scalar value like a
+ `List`, `Dict`, or [`Selector`](#dagster.Selector).
+
+ This allows runtime scalars to be configured without a dictionary with the key `value` and
+ instead just use the scalar value directly. However this still leaves the option to
+ load scalars from a json or pickle file.
+
+ Parameters:
+ - scalar_type (type) – The scalar type of values that this configuration type can hold. For example,
+ - non_scalar_schema ([*ConfigSchema*](#dagster.ConfigSchema)) – The schema of a non-scalar Dagster configuration type. For example, `List`,
+ - key (Optional[str]) – The configuation type’s unique key. If not set, then the key will be set to
+
+
+ Examples:
+
+ ```yaml
+ graph:
+ transform_word:
+ inputs:
+ word:
+ value: foobar
+ ```
+ becomes, optionally,
+
+ ```yaml
+ graph:
+ transform_word:
+ inputs:
+ word: foobar
+ ```
+
+
+
+
+
+
dagster.StringSource
+
+
+ Use this type when you want to read a string config value from an environment variable. The value
+ passed to a config field of this type may either be a string literal, or a selector describing
+ how to look up the value from the executing process’s environment variables.
+
+ Examples:
+
+ ```python
+ from dagster import job, op, StringSource
+
+ @op(config_schema=StringSource)
+ def secret_op(context) -> str:
+ return context.op_config
+
+ @job
+ def secret_job():
+ secret_op()
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_op': {'config': 'test_value'}}
+ }
+ )
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_op': {'config': {'env': 'VERY_SECRET_ENV_VARIABLE'}}}
+ }
+ )
+ ```
+
+
+
+
+
+
dagster.IntSource
+
+
+ Use this type when you want to read an integer config value from an environment variable. The
+ value passed to a config field of this type may either be a integer literal, or a selector
+ describing how to look up the value from the executing process’s environment variables.
+
+ Examples:
+
+ ```python
+ from dagster import job, op, IntSource
+
+ @op(config_schema=IntSource)
+ def secret_int_op(context) -> int:
+ return context.op_config
+
+ @job
+ def secret_job():
+ secret_int_op()
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_int_op': {'config': 1234}}
+ }
+ )
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_int_op': {'config': {'env': 'VERY_SECRET_ENV_VARIABLE_INT'}}}
+ }
+ )
+ ```
+
+
+
+
+
+
dagster.BoolSource
+
+
+ Use this type when you want to read an boolean config value from an environment variable. The
+ value passed to a config field of this type may either be a boolean literal, or a selector
+ describing how to look up the value from the executing process’s environment variables. Set the
+ value of the corresponding environment variable to `""` to indicate `False`.
+
+ Examples:
+
+ ```python
+ from dagster import job, op, BoolSource
+
+ @op(config_schema=BoolSource)
+ def secret_bool_op(context) -> bool:
+ return context.op_config
+
+ @job
+ def secret_job():
+ secret_bool_op()
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_bool_op': {'config': False}}
+ }
+ )
+
+ secret_job.execute_in_process(
+ run_config={
+ 'ops': {'secret_bool_op': {'config': {'env': 'VERY_SECRET_ENV_VARIABLE_BOOL'}}}
+ }
+ )
+ ```
+
+
+
+
+
+
+
+
+
+## Config Utilities
+
+
+
class dagster.ConfigMapping
+
+
+ Defines a config mapping for a graph (or job).
+
+ By specifying a config mapping function, you can override the configuration for the child
+ ops and graphs contained within a graph.
+
+ Config mappings require the configuration schema to be specified as `config_schema`, which will
+ be exposed as the configuration schema for the graph, as well as a configuration mapping
+ function, `config_fn`, which maps the config provided to the graph to the config
+ that will be provided to the child nodes.
+
+ Parameters:
+ - config_fn (Callable[[dict], dict]) – The function that will be called
+ - config_schema ([*ConfigSchema*](#dagster.ConfigSchema)) – The schema of the graph config.
+ - receive_processed_config_values (Optional[bool]) – If true, config values provided to the config_fn
+
+
+
+
+
+
+
+
@dagster.configured
+
+
+ A decorator that makes it easy to create a function-configured version of an object.
+
+ The following definition types can be configured using this function:
+
+ - [`GraphDefinition`](graphs.mdx#dagster.GraphDefinition)
+ - [`ExecutorDefinition`](internals.mdx#dagster.ExecutorDefinition)
+ - [`LoggerDefinition`](loggers.mdx#dagster.LoggerDefinition)
+ - [`ResourceDefinition`](resources.mdx#dagster.ResourceDefinition)
+ - [`OpDefinition`](ops.mdx#dagster.OpDefinition)
+
+
+ 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.
+
+ If the config that will be supplied to the object is constant, you may alternatively invoke this
+ and call the result with a dict of config values to be curried. Examples of both strategies
+ below.
+
+ Parameters:
+ - configurable (ConfigurableDefinition) – An object that can be configured.
+ - config_schema ([*ConfigSchema*](#dagster.ConfigSchema)) – The config schema that the inputs to the decorated function
+ - **kwargs – Arbitrary keyword arguments that will be passed to the initializer of the returned
+
+
+ Returns: (Callable[[Union[Any, Callable[[Any], Any]]], ConfigurableDefinition])
+ Examples:
+
+ ```python
+ class GreetingConfig(Config):
+ message: str
+
+ @op
+ def greeting_op(config: GreetingConfig):
+ print(config.message)
+
+ class HelloConfig(Config):
+ name: str
+
+ @configured(greeting_op)
+ def hello_op(config: HelloConfig):
+ return GreetingConfig(message=f"Hello, {config.name}!")
+ ```
+ ```python
+ dev_s3 = configured(S3Resource, name="dev_s3")({'bucket': 'dev'})
+
+ @configured(S3Resource)
+ def dev_s3(_):
+ return {'bucket': 'dev'}
+
+ @configured(S3Resource, {'bucket_prefix', str})
+ def dev_s3(config):
+ return {'bucket': config['bucket_prefix'] + 'dev'}
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/definitions.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/definitions.mdx
new file mode 100644
index 0000000000000..dacdde1256dd9
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/definitions.mdx
@@ -0,0 +1,350 @@
+---
+title: 'definitions'
+title_meta: 'definitions API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'definitions Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Definitions
+
+
+
class dagster.Definitions
+
+
+ A set of definitions explicitly available and loadable by Dagster tools.
+
+ Parameters:
+ - assets (Optional[Iterable[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset), CacheableAssetsDefinition]]]) – A list of assets. Assets can be created by annotating
+ - asset_checks (Optional[Iterable[[*AssetChecksDefinition*](asset-checks.mdx#dagster.AssetChecksDefinition)]]) – A list of asset checks.
+ - schedules (Optional[Iterable[Union[[*ScheduleDefinition*](schedules-sensors.mdx#dagster.ScheduleDefinition), UnresolvedPartitionedAssetScheduleDefinition]]]) – List of schedules.
+ - sensors (Optional[Iterable[[*SensorDefinition*](schedules-sensors.mdx#dagster.SensorDefinition)]]) – List of sensors, typically created with [`@sensor`](schedules-sensors.mdx#dagster.sensor).
+ - jobs (Optional[Iterable[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]]) – List of jobs. Typically created with [`define_asset_job`](assets.mdx#dagster.define_asset_job)
+ - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets.
+ - executor (Optional[Union[[*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition), [*Executor*](internals.mdx#dagster.Executor)]]) – Default executor for jobs. Individual jobs can override this and define their own executors
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.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:
+
+ ```python
+ 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`](#dagster.Definitions).
+
+
+
get_all_asset_specs
+
+
+ :::warning[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`](io-managers.mdx#dagster.IOManager) associated with the assets. Avoids
+ spinning up resources separately for each asset.
+
+ Usage:
+
+ ```python
+ 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 of [`define_asset_job()`](assets.mdx#dagster.define_asset_job)) it will be resolved to a [`JobDefinition`](jobs.mdx#dagster.JobDefinition) when returned
+ from this function, with all resource dependencies fully resolved.
+
+
+
+
+
get_schedule_def
+
+ Get a [`ScheduleDefinition`](schedules-sensors.mdx#dagster.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`](schedules-sensors.mdx#dagster.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`](io-managers.mdx#dagster.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()`](#dagster.Definitions.get_asset_value_loader), which avoids spinning up
+ resources separately for each asset.
+
+ Parameters:
+ - asset_key (Union[[*AssetKey*](assets.mdx#dagster.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`](io-managers.mdx#dagster.IOManager)
+
+
+ Returns: The contents of an asset as a Python object.
+
+
+
+
+
+
static merge
+
+
+ :::warning[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:
+
+ ```python
+ import submodule1
+ import submodule2
+
+ defs = Definitions.merge(submodule1.defs, submodule2.defs)
+ ```
+ Returns: The merged definitions.Return type: [Definitions](#dagster.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
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Create a named repository using the same arguments as [`Definitions`](#dagster.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`](#dagster.Definitions) API and 2) but still
+ want multiple logical groups of assets in the same code location, we have introduced this function.
+
+ Example usage:
+
+ ```python
+ 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
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs the [`dagster.Definitions`](#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`](#dagster.Definitions).
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – Default loggers for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+ - executor (Optional[Union[[*Executor*](internals.mdx#dagster.Executor), [*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]]) – Default executor for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+
+
+ Returns: The [`dagster.Definitions`](#dagster.Definitions) defined in the current module.Return type: [Definitions](#dagster.Definitions)
+
+
+
+
+
+
dagster.load_definitions_from_module
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs the [`dagster.Definitions`](#dagster.Definitions) from the given module.
+
+ Parameters:
+ - module (ModuleType) – The Python module to look for [`dagster.Definitions`](#dagster.Definitions) inside.
+ - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded [`dagster.Definitions`](#dagster.Definitions).
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – Default loggers for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+ - executor (Optional[Union[[*Executor*](internals.mdx#dagster.Executor), [*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]]) – Default executor for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+
+
+ Returns: The [`dagster.Definitions`](#dagster.Definitions) defined in the given module.Return type: [Definitions](#dagster.Definitions)
+
+
+
+
+
+
dagster.load_definitions_from_modules
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs the [`dagster.Definitions`](#dagster.Definitions) from the given modules.
+
+ Parameters:
+ - modules (Iterable[ModuleType]) – The Python modules to look for [`dagster.Definitions`](#dagster.Definitions) inside.
+ - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded [`dagster.Definitions`](#dagster.Definitions).
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – Default loggers for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+ - executor (Optional[Union[[*Executor*](internals.mdx#dagster.Executor), [*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]]) – Default executor for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+
+
+ Returns: The [`dagster.Definitions`](#dagster.Definitions) defined in the given modules.Return type: [Definitions](#dagster.Definitions)
+
+
+
+
+
+
dagster.load_definitions_from_package_module
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs the [`dagster.Definitions`](#dagster.Definitions) from the given package module.
+
+ Parameters:
+ - package_module (ModuleType) – The package module to look for [`dagster.Definitions`](#dagster.Definitions) inside.
+ - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded [`dagster.Definitions`](#dagster.Definitions).
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – Default loggers for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+ - executor (Optional[Union[[*Executor*](internals.mdx#dagster.Executor), [*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]]) – Default executor for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+
+
+ Returns: The [`dagster.Definitions`](#dagster.Definitions) defined in the given package module.Return type: [Definitions](#dagster.Definitions)
+
+
+
+
+
+
dagster.load_definitions_from_package_name
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Constructs the [`dagster.Definitions`](#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`](#dagster.Definitions) inside.
+ - resources (Optional[Mapping[str, Any]]) – Dictionary of resources to bind to assets in the loaded [`dagster.Definitions`](#dagster.Definitions).
+ - loggers (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – Default loggers for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+ - executor (Optional[Union[[*Executor*](internals.mdx#dagster.Executor), [*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]]) – Default executor for jobs in the loaded [`dagster.Definitions`](#dagster.Definitions). Individual jobs
+
+
+ Returns: The [`dagster.Definitions`](#dagster.Definitions) defined in the package module for the given package name.Return type: [Definitions](#dagster.Definitions)
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/dynamic.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/dynamic.mdx
new file mode 100644
index 0000000000000..84769596f5df6
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/dynamic.mdx
@@ -0,0 +1,105 @@
+---
+title: 'dynamic mapping & collect'
+title_meta: 'dynamic mapping & collect API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dynamic mapping & collect Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+# Dynamic Mapping & Collect
+
+These APIs provide the means for a simple kind of dynamic orchestration — where the work to be orchestrated is determined not at job definition time but at runtime, dependent on data that’s observed as part of job execution.
+
+
+
+
class dagster.DynamicOut
+
+
+ Variant of [`Out`](ops.mdx#dagster.Out) for an output that will dynamically alter the graph at
+ runtime.
+
+ When using in a composition function such as [`@graph`](graphs.mdx#dagster.graph),
+ dynamic outputs must be used with either
+
+ - `map` - clone downstream ops for each separate [`DynamicOut`](#dagster.DynamicOut)
+ - `collect` - gather across all [`DynamicOut`](#dagster.DynamicOut) in to a list
+
+
+ Uses the same constructor as [`Out`](ops.mdx#dagster.Out)
+
+ >
+
+ ```python
+ @op(
+ config_schema={
+ "path": Field(str, default_value=file_relative_path(__file__, "sample"))
+ },
+ out=DynamicOut(str),
+ )
+ def files_in_directory(context):
+ path = context.op_config["path"]
+ dirname, _, filenames = next(os.walk(path))
+ for file in filenames:
+ yield DynamicOutput(os.path.join(dirname, file), mapping_key=_clean(file))
+
+ @job
+ def process_directory():
+ files = files_in_directory()
+
+ # use map to invoke an op on each dynamic output
+ file_results = files.map(process_file)
+
+ # use collect to gather the results in to a list
+ summarize_directory(file_results.collect())
+ ```
+
+
+
+
+
+
+
+
class dagster.DynamicOutput
+
+
+ Variant of [`Output`](ops.mdx#dagster.Output) used to support
+ dynamic mapping & collect. Each `DynamicOutput` produced by an op represents
+ one item in a set that can be processed individually with `map` or gathered
+ with `collect`.
+
+ Each `DynamicOutput` must have a unique `mapping_key` to distinguish it with it’s set.
+
+ Parameters:
+ - value (Any) – The value returned by the compute function.
+ - mapping_key (str) – The key that uniquely identifies this dynamic value relative to its peers.
+ - output_name (Optional[str]) – Name of the corresponding [`DynamicOut`](#dagster.DynamicOut) defined on the op.
+ - metadata (Optional[Dict[str, Union[str, float, int, [*MetadataValue*](metadata.mdx#dagster.MetadataValue)]]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
+
+
+
+
property mapping_key
+
+ The mapping_key that was set for this DynamicOutput at instantiation.
+
+
+
+
+
property output_name
+
+ Name of the [`DynamicOut`](#dagster.DynamicOut) defined on the op that this DynamicOut is associated with.
+
+
+
+
+
property value
+
+ The value that is returned by the compute function for this DynamicOut.
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/errors.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/errors.mdx
new file mode 100644
index 0000000000000..89d15986d8643
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/errors.mdx
@@ -0,0 +1,234 @@
+---
+title: 'errors'
+title_meta: 'errors API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'errors Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Errors
+
+Core Dagster error classes.
+
+All errors thrown by the Dagster framework inherit from [`DagsterError`](#dagster.DagsterError). Users
+should not subclass this base class for their own exceptions.
+
+There is another exception base class, [`DagsterUserCodeExecutionError`](#dagster.DagsterUserCodeExecutionError), which is
+used by the framework in concert with the [`user_code_error_boundary()`](internals.mdx#dagster._core.errors.user_code_error_boundary).
+
+Dagster uses this construct to wrap user code into which it calls. User code can perform arbitrary
+computations and may itself throw exceptions. The error boundary catches these user code-generated
+exceptions, and then reraises them wrapped in a subclass of
+[`DagsterUserCodeExecutionError`](#dagster.DagsterUserCodeExecutionError).
+
+The wrapped exceptions include additional context for the original exceptions, injected by the
+Dagster runtime.
+
+
+
exception dagster.DagsterError
+
+
+ Base class for all errors thrown by the Dagster framework.
+
+ Users should not subclass this base class for their own exceptions.
+
+
+
property is_user_code_error
+
+ Returns true if this error is attributable to user code.
+
+ Indicates that an unexpected error occurred while executing the body of a config mapping
+ function defined in a [`JobDefinition`](jobs.mdx#dagster.JobDefinition) or ~dagster.GraphDefinition during
+ config parsing.
+
+
+
+
+
exception dagster.DagsterEventLogInvalidForRun
+
+ Raised when the event logs for a historical run are malformed or invalid.
+
+
+ Indicates that you have attempted to construct a config with an invalid value.
+
+ Acceptable values for config types are any of:
+ 1. A Python primitive type that resolves to a Dagster config type
+ (`python:int`, `python:float`, `python:bool`,
+ `python:str`, or `python:list`).
+
+ 2. A Dagster config type: `Int`, `Float`,
+ `Bool`, `String`,
+ [`StringSource`](config.mdx#dagster.StringSource), `Any`,
+ [`Array`](config.mdx#dagster.Array), [`Noneable`](config.mdx#dagster.Noneable), [`Enum`](config.mdx#dagster.Enum),
+ [`Selector`](config.mdx#dagster.Selector), [`Shape`](config.mdx#dagster.Shape), or
+ [`Permissive`](config.mdx#dagster.Permissive).
+
+ 3. A bare python dictionary, which will be automatically wrapped in
+ [`Shape`](config.mdx#dagster.Shape). Values of the dictionary are resolved recursively
+ according to the same rules.
+
+ 4. A bare python list of length one which itself is config type.
+ Becomes [`Array`](config.mdx#dagster.Array) with list element as an argument.
+
+ 5. An instance of [`Field`](config.mdx#dagster.Field).
+
+
+
+
+
+
exception dagster.DagsterInvalidDefinitionError
+
+ Indicates that the rules for a definition have been violated by the user.
+
+
+
+
+
exception dagster.DagsterInvalidSubsetError
+
+ Indicates that a subset of a pipeline is invalid because either:
+ - One or more ops in the specified subset do not exist on the job.’
+ - The subset produces an invalid job.
+
+
+
+
+
exception dagster.DagsterInvariantViolationError
+
+ Indicates the user has violated a well-defined invariant that can only be enforced
+ at runtime.
+
+
+
+
+
exception dagster.DagsterResourceFunctionError
+
+ Indicates an error occurred while executing the body of the `resource_fn` in a
+ [`ResourceDefinition`](resources.mdx#dagster.ResourceDefinition) during resource initialization.
+
+
+
+
+
exception dagster.DagsterRunNotFoundError
+
+ Thrown when a run cannot be found in run storage.
+
+
+
+
+
exception dagster.DagsterStepOutputNotFoundError
+
+ Indicates that previous step outputs required for an execution step to proceed are not
+ available.
+
+
+
+
+
exception dagster.DagsterSubprocessError
+
+ An exception has occurred in one or more of the child processes dagster manages.
+ This error forwards the message and stack trace for all of the collected errors.
+
+
+
+
+
exception dagster.DagsterTypeCheckDidNotPass
+
+
+ Indicates that a type check failed.
+
+ This is raised when `raise_on_error` is `True` in calls to the synchronous job and
+ graph execution APIs (e.g. graph.execute_in_process(), job.execute_in_process() – typically
+ within a test), and a [`DagsterType`](types.mdx#dagster.DagsterType)’s type check fails by returning either
+ `False` or an instance of [`TypeCheck`](ops.mdx#dagster.TypeCheck) whose `success` member is `False`.
+
+
+
+
+
+
+
exception dagster.DagsterTypeCheckError
+
+ Indicates an error in the op type system at runtime. E.g. a op receives an
+ unexpected input, or produces an output that does not match the type of the output definition.
+
+
+
+
+
exception dagster.DagsterUnknownResourceError
+
+ Indicates that an unknown resource was accessed in the body of an execution step. May often
+ happen by accessing a resource in the compute function of an op without first supplying the
+ op with the correct required_resource_keys argument.
+
+ Indicates the resolved executor is incompatible with the state of other systems
+ such as the [`DagsterInstance`](internals.mdx#dagster.DagsterInstance) or system storage configuration.
+
+
+
+
+
exception dagster.DagsterUserCodeExecutionError
+
+
+ This is the base class for any exception that is meant to wrap an
+ `python:Exception` thrown by user code. It wraps that existing user code.
+ The `original_exc_info` argument to the constructor is meant to be a tuple of the type
+ returned by `sys.exc_info` at the call site of the constructor.
+
+ Users should not subclass this base class for their own exceptions and should instead throw
+ freely from user code. User exceptions will be automatically wrapped and rethrown.
+
+
+
property is_user_code_error
+
+ Returns true if this error is attributable to user code.
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/execution.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/execution.mdx
new file mode 100644
index 0000000000000..e871596744f60
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/execution.mdx
@@ -0,0 +1,3797 @@
+---
+title: 'execution'
+title_meta: 'execution API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'execution Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Execution
+
+
+
+
+## Materializing Assets
+
+
+
dagster.materialize
+
+
+ Executes a single-threaded, in-process run which materializes provided assets.
+
+ By default, will materialize assets to the local filesystem.
+
+ Parameters:
+ - assets (Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*AssetSpec*](assets.mdx#dagster.AssetSpec), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]]) –
+
+ The assets to materialize.
+
+ Unless you’re using deps or non_argument_deps, you must also include all assets that are
+ upstream of the assets that you want to materialize. This is because those upstream
+ asset definitions have information that is needed to load their contents while
+ materializing the downstream assets.
+
+ - resources (Optional[Mapping[str, object]]) – The resources needed for execution. Can provide resource instances
+ - run_config (Optional[Any]) – The run config to use for the run that materializes the assets.
+ - partition_key – (Optional[str])
+ - tags (Optional[Mapping[str, str]]) – Tags for the run.
+ - selection (Optional[Union[str, Sequence[str], Sequence[[*AssetKey*](assets.mdx#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]], [*AssetSelection*](assets.mdx#dagster.AssetSelection)]]) –
+
+ A sub-selection of assets to materialize.
+
+ If not provided, then all assets will be materialized.
+
+
+
+ Returns: The result of the execution.Return type: [ExecuteInProcessResult](#dagster.ExecuteInProcessResult)
+ Examples:
+
+ ```python
+ @asset
+ def asset1():
+ ...
+
+ @asset
+ def asset2(asset1):
+ ...
+
+ # executes a run that materializes asset1 and then asset2
+ materialize([asset1, asset2])
+
+ # executes a run that materializes just asset2, loading its input from asset1
+ materialize([asset1, asset2], selection=[asset2])
+ ```
+
+
+
+
+
+
dagster.materialize_to_memory
+
+
+ Executes a single-threaded, in-process run which materializes provided assets in memory.
+
+ Will explicitly use [`mem_io_manager()`](io-managers.mdx#dagster.mem_io_manager) for all required io manager
+ keys. If any io managers are directly provided using the resources
+ argument, a [`DagsterInvariantViolationError`](errors.mdx#dagster.DagsterInvariantViolationError) will be thrown.
+
+ Parameters:
+ - assets (Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*AssetSpec*](assets.mdx#dagster.AssetSpec), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]]) – The assets to materialize. Can also provide [`SourceAsset`](assets.mdx#dagster.SourceAsset) objects to fill dependencies for asset defs.
+ - run_config (Optional[Any]) – The run config to use for the run that materializes the assets.
+ - resources (Optional[Mapping[str, object]]) – The resources needed for execution. Can provide resource instances
+ - partition_key – (Optional[str])
+ - tags (Optional[Mapping[str, str]]) – Tags for the run.
+ - selection (Optional[Union[str, Sequence[str], Sequence[[*AssetKey*](assets.mdx#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]], [*AssetSelection*](assets.mdx#dagster.AssetSelection)]]) –
+
+ A sub-selection of assets to materialize.
+
+ If not provided, then all assets will be materialized.
+
+
+
+ Returns: The result of the execution.Return type: [ExecuteInProcessResult](#dagster.ExecuteInProcessResult)
+ Examples:
+
+ ```python
+ @asset
+ def asset1():
+ ...
+
+ @asset
+ def asset2(asset1):
+ ...
+
+ # executes a run that materializes asset1 and then asset2
+ materialize([asset1, asset2])
+
+ # executes a run that materializes just asset1
+ materialize([asset1, asset2], selection=[asset1])
+ ```
+
+
+
+
+
+
+
+
+
+
+## Executing Jobs
+
+
+
class dagster.JobDefinition
+
+
+ Defines a Dagster job.
+
+
+
execute_in_process
+
+
+ Execute the Job in-process, gathering results in-memory.
+
+ The executor_def on the Job will be ignored, and replaced with the in-process executor.
+ If using the default io_manager, it will switch from filesystem to in-memory.
+
+ Parameters:
+ - (Optional[Mapping[str (run_config) – The configuration for the run
+ - Any]] – The configuration for the run
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The instance to execute against, an ephemeral one will be used if none provided.
+ - partition_key – (Optional[str])
+ - raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur.
+ - op_selection (Optional[Sequence[str]]) – A list of op selection queries (including single op
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of the job. Input
+ - resources (Optional[Mapping[str, Any]]) – The resources needed if any are required. Can provide resource instances directly,
+
+
+ Returns: [`ExecuteInProcessResult`](#dagster.ExecuteInProcessResult)
+
+
+
+
+
+
run_request_for_partition
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.0.
+ Directly instantiate `RunRequest(partition_key=...)` instead..
+
+ :::
+
+ Creates a RunRequest object for a run that processes the given partition.
+
+ Parameters:
+ - partition_key – The key of the partition to request a run for.
+ - run_key (Optional[str]) – A string key to identify this launched run. For sensors, ensures that
+ - tags (Optional[Dict[str, str]]) – A dictionary of tags (string key-value pairs) to attach
+ - (Optional[Mapping[str (run_config) – Configuration for the run. If the job has
+ - Any]] – Configuration for the run. If the job has
+ - current_time (Optional[datetime]) – Used to determine which time-partitions exist.
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: an object that requests a run to process the given partition.Return type: [RunRequest](schedules-sensors.mdx#dagster.RunRequest)
+
+
+
+
+
+
with_hooks
+
+ Apply a set of hooks to all op instances within the job.
+
+
+
+
+
with_top_level_resources
+
+ Apply a set of resources to all op instances within the job.
+
+
+
+
+
property config_mapping
+
+
+ The config mapping for the job, if it has one.
+
+ A config mapping defines a way to map a top-level config schema to run config for the job.
+
+
+
+
+
+
+
property executor_def
+
+
+ Returns the default [`ExecutorDefinition`](internals.mdx#dagster.ExecutorDefinition) for the job.
+
+ If the user has not specified an executor definition, then this will default to the
+ [`multi_or_in_process_executor()`](#dagster.multi_or_in_process_executor). If a default is specified on the
+ [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to, then that will be used instead.
+
+
+
+
+
+
+
property has_specified_executor
+
+ Returns True if this job has explicitly specified an executor, and False if the executor
+ was inherited through defaults or the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to.
+
+
+
+
+
property has_specified_loggers
+
+ Returns true if the job explicitly set loggers, and False if loggers were inherited
+ through defaults or the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to.
+
+
+
+
+
property loggers
+
+
+ Returns the set of LoggerDefinition objects specified on the job.
+
+ If the user has not specified a mapping of [`LoggerDefinition`](loggers.mdx#dagster.LoggerDefinition) objects, then this
+ will default to the `colored_console_logger()` under the key console. If a default
+ is specified on the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to, then that will
+ be used instead.
+
+
+
+
+
+
+
property partitioned_config
+
+
+ The partitioned config for the job, if it has one.
+
+ A partitioned config defines a way to map partition keys to run config for the job.
+
+
+
+
+
+
+
property partitions_def
+
+
+ Returns the [`PartitionsDefinition`](partitions.mdx#dagster.PartitionsDefinition) for the job, if it has one.
+
+ A partitions definition defines the set of partition keys the job operates on.
+
+
+
+
+
+
+
property resource_defs
+
+
+ Returns the set of ResourceDefinition objects specified on the job.
+
+ This may not be the complete set of resources required by the job, since those can also be
+ provided on the [`Definitions`](definitions.mdx#dagster.Definitions) object the job may be provided to.
+
+
+
+
+
+
+
+
+
+
+
dagster.execute_job
+
+
+ Execute a job synchronously.
+
+ This API represents dagster’s python entrypoint for out-of-process
+ execution. For most testing purposes, `
+ execute_in_process()` will be more suitable, but when wanting to run
+ execution using an out-of-process executor (such as `dagster.
+ multiprocess_executor`), then execute_job is suitable.
+
+ execute_job expects a persistent [`DagsterInstance`](internals.mdx#dagster.DagsterInstance) for
+ execution, meaning the $DAGSTER_HOME environment variable must be set.
+ It also expects a reconstructable pointer to a [`JobDefinition`](jobs.mdx#dagster.JobDefinition) so
+ that it can be reconstructed in separate processes. This can be done by
+ wrapping the `JobDefinition` in a call to `dagster.
+ reconstructable()`.
+
+ ```python
+ from dagster import DagsterInstance, execute_job, job, reconstructable
+
+ @job
+ def the_job():
+ ...
+
+ instance = DagsterInstance.get()
+ result = execute_job(reconstructable(the_job), instance=instance)
+ assert result.success
+ ```
+ If using the [`to_job()`](graphs.mdx#dagster.GraphDefinition.to_job) method to
+ construct the `JobDefinition`, then the invocation must be wrapped in a
+ module-scope function, which can be passed to `reconstructable`.
+
+ ```python
+ from dagster import graph, reconstructable
+
+ @graph
+ def the_graph():
+ ...
+
+ def define_job():
+ return the_graph.to_job(...)
+
+ result = execute_job(reconstructable(define_job), ...)
+ ```
+ Since execute_job is potentially executing outside of the current
+ process, output objects need to be retrieved by use of the provided job’s
+ io managers. Output objects can be retrieved by opening the result of
+ execute_job as a context manager.
+
+ ```python
+ from dagster import execute_job
+
+ with execute_job(...) as result:
+ output_obj = result.output_for_node("some_op")
+ ```
+ `execute_job` can also be used to reexecute a run, by providing a [`ReexecutionOptions`](#dagster.ReexecutionOptions) object.
+
+ ```python
+ from dagster import ReexecutionOptions, execute_job
+
+ instance = DagsterInstance.get()
+
+ options = ReexecutionOptions.from_failure(run_id=failed_run_id, instance)
+ execute_job(reconstructable(job), instance, reexecution_options=options)
+ ```
+ Parameters:
+ - job (ReconstructableJob) – A reconstructable pointer to a [`JobDefinition`](jobs.mdx#dagster.JobDefinition).
+ - instance ([*DagsterInstance*](internals.mdx#dagster.DagsterInstance)) – The instance to execute against.
+ - run_config (Optional[dict]) – The configuration that parametrizes this run, as a dict.
+ - tags (Optional[Dict[str, Any]]) – Arbitrary key-value pairs that will be added to run logs.
+ - raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur.
+ - op_selection (Optional[List[str]]) –
+
+ A list of op selection queries (including single
+ op names) to execute. For example:
+
+ - `['some_op']`: selects `some_op` itself.
+ - `['*some_op']`: select `some_op` and all its ancestors (upstream dependencies).
+ - `['*some_op+++']`: select `some_op`, all its ancestors, and its descendants
+ - `['*some_op', 'other_op_a', 'other_op_b+']`: select `some_op` and all its
+ - reexecution_options (Optional[[*ReexecutionOptions*](#dagster.ReexecutionOptions)]) – Reexecution options to provide to the run, if this run is
+
+
+ Returns: The result of job execution.Return type: [`JobExecutionResult`](#dagster.JobExecutionResult)
+
+
+
+
+
+
class dagster.ReexecutionOptions
+
+
+ Reexecution options for python-based execution in Dagster.
+
+ Parameters:
+ - parent_run_id (str) – The run_id of the run to reexecute.
+ - step_selection (Sequence[str]) –
+
+ The list of step selections to reexecute. Must be a subset or match of the
+ set of steps executed in the original run. For example:
+
+ - `['some_op']`: selects `some_op` itself.
+ - `['*some_op']`: select `some_op` and all its ancestors (upstream dependencies).
+ - `['*some_op+++']`: select `some_op`, all its ancestors, and its descendants
+ - `['*some_op', 'other_op_a', 'other_op_b+']`: select `some_op` and all its
+
+
+
+
+
+
+
+
dagster.instance_for_test
+
+
+ Creates a persistent [`DagsterInstance`](internals.mdx#dagster.DagsterInstance) available within a context manager.
+
+ When a context manager is opened, if no temp_dir parameter is set, a new
+ temporary directory will be created for the duration of the context
+ manager’s opening. If the set_dagster_home parameter is set to True
+ (True by default), the $DAGSTER_HOME environment variable will be
+ overridden to be this directory (or the directory passed in by temp_dir)
+ for the duration of the context manager being open.
+
+ Parameters:
+ - overrides (Optional[Mapping[str, Any]]) – Config to provide to instance (config format follows that typically found in an instance.yaml file).
+ - set_dagster_home (Optional[bool]) – If set to True, the $DAGSTER_HOME environment variable will be
+ - temp_dir (Optional[str]) – The directory to use for storing local artifacts produced by the
+
+
+
+
+
+
+
+
+
+
+
+
+## Executing Graphs
+
+
+
class dagster.GraphDefinition
+
+
+ Defines a Dagster op graph.
+
+ An op graph is made up of
+
+ - Nodes, which can either be an op (the functional unit of computation), or another graph.
+ - Dependencies, which determine how the values produced by nodes as outputs flow from
+
+
+ End users should prefer the [`@graph`](graphs.mdx#dagster.graph) decorator. GraphDefinition is generally
+ intended to be used by framework authors or for programatically generated graphs.
+
+ Parameters:
+ - name (str) – The name of the graph. Must be unique within any [`GraphDefinition`](graphs.mdx#dagster.GraphDefinition)
+ - description (Optional[str]) – A human-readable description of the job.
+ - node_defs (Optional[Sequence[NodeDefinition]]) – The set of ops / graphs used in this graph.
+ - dependencies (Optional[Dict[Union[str, [*NodeInvocation*](graphs.mdx#dagster.NodeInvocation)], Dict[str, [*DependencyDefinition*](graphs.mdx#dagster.DependencyDefinition)]]]) – A structure that declares the dependencies of each op’s inputs on the outputs of other
+ - input_mappings (Optional[Sequence[[*InputMapping*](graphs.mdx#dagster.InputMapping)]]) – Defines the inputs to the nested graph, and
+ - output_mappings (Optional[Sequence[[*OutputMapping*](graphs.mdx#dagster.OutputMapping)]]) – Defines the outputs of the nested graph,
+ - config (Optional[[*ConfigMapping*](config.mdx#dagster.ConfigMapping)]) – Defines the config of the graph, and how its schema maps
+ - tags (Optional[Dict[str, Any]]) – Arbitrary metadata for any execution of the graph.
+ - composition_fn (Optional[Callable]) – The function that defines this graph. Used to generate
+
+
+ Examples:
+
+ ```python
+ @op
+ def return_one():
+ return 1
+
+ @op
+ def add_one(num):
+ return num + 1
+
+ graph_def = GraphDefinition(
+ name='basic',
+ node_defs=[return_one, add_one],
+ dependencies={'add_one': {'num': DependencyDefinition('return_one')}},
+ )
+ ```
+
+
alias
+
+
+ Aliases the graph with a new name.
+
+ Can only be used in the context of a [`@graph`](graphs.mdx#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.alias("my_graph_alias")
+ ```
+
+
+
+
+
+
execute_in_process
+
+
+ Execute this graph in-process, collecting results in-memory.
+
+ Parameters:
+ - run_config (Optional[Mapping[str, Any]]) – Run config to provide to execution. The configuration for the underlying graph
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The instance to execute against, an ephemeral one will be used if none provided.
+ - resources (Optional[Mapping[str, Any]]) – The resources needed if any are required. Can provide resource instances directly,
+ - raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur.
+ - op_selection (Optional[List[str]]) – A list of op selection queries (including single op
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of the graph.
+
+
+ Returns: [`ExecuteInProcessResult`](#dagster.ExecuteInProcessResult)
+
+
+
+
+
+
tag
+
+
+ Attaches the provided tags to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](graphs.mdx#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.tag({"my_tag": "my_value"})
+ ```
+
+
+
+
+
+
to_job
+
+
+ Make this graph in to an executable Job by providing remaining components required for execution.
+
+ Parameters:
+ - name (Optional[str]) – The name for the Job. Defaults to the name of the this graph.
+ - resource_defs (Optional[Mapping [str, object]]) – Resources that are required by this graph for execution.
+ - config –
+
+ Describes how the job is parameterized at runtime.
+
+ If no value is provided, then the schema for the job’s run config is a standard
+ format based on its ops and resources.
+
+ If a dictionary is provided, then it must conform to the standard config schema, and
+ it will be used as the job’s run config for the job whenever the job is executed.
+ The values provided will be viewable and editable in the Dagster UI, so be
+ careful with secrets.
+
+ If a [`ConfigMapping`](config.mdx#dagster.ConfigMapping) object is provided, then the schema for the job’s run config is
+ determined by the config mapping, and the ConfigMapping, which should return
+ configuration in the standard format to configure the job.
+
+ - tags (Optional[Mapping[str, object]]) – A set of key-value tags that annotate the job and can
+ - run_tags (Optional[Mapping[str, object]]) – A set of key-value tags that will be automatically attached to runs launched by this
+ - metadata (Optional[Mapping[str, RawMetadataValue]]) – Arbitrary information that will be attached to the JobDefinition and be viewable in the Dagster UI.
+ - logger_defs (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – A dictionary of string logger identifiers to their implementations.
+ - executor_def (Optional[[*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]) – How this Job will be executed. Defaults to [`multi_or_in_process_executor`](#dagster.multi_or_in_process_executor),
+ - op_retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The default retry policy for all ops in this job.
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines a discrete set of partition
+ - asset_layer (Optional[AssetLayer]) – Top level information about the assets this job
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of a job.
+
+
+ Returns: JobDefinition
+
+
+
+
+
+
with_hooks
+
+
+ Attaches the provided hooks to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](graphs.mdx#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.with_hooks({my_hook})
+ ```
+
+
+
+
+
+
with_retry_policy
+
+
+ Attaches the provided retry policy to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](graphs.mdx#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.with_retry_policy(RetryPolicy(max_retries=5))
+ ```
+
+
+
+
+
+
property config_mapping
+
+
+ The config mapping for the graph, if present.
+
+ By specifying a config mapping function, you can override the configuration for the child nodes contained within a graph.
+
+
+
+
+
+
+
property input_mappings
+
+
+ Input mappings for the graph.
+
+ An input mapping is a mapping from an input of the graph to an input of a child node.
+
+
+
+
+
+
+
property name
+
+ The name of the graph.
+
+
+
+
+
property output_mappings
+
+
+ Output mappings for the graph.
+
+ An output mapping is a mapping from an output of the graph to an output of a child node.
+
+
+
+
+
+
+
property tags
+
+ The tags associated with the graph.
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Execution results
+
+
+
class dagster.ExecuteInProcessResult
+
+
+ Result object returned by in-process testing APIs.
+
+ Users should not instantiate this object directly. Used for retrieving run success, events, and outputs from execution methods that return this object.
+
+ This object is returned by:
+ - [`dagster.GraphDefinition.execute_in_process()`](graphs.mdx#dagster.GraphDefinition.execute_in_process)
+ - [`dagster.JobDefinition.execute_in_process()`](jobs.mdx#dagster.JobDefinition.execute_in_process)
+ - [`dagster.materialize_to_memory()`](#dagster.materialize_to_memory)
+ - [`dagster.materialize()`](#dagster.materialize)
+
+
+
asset_value
+
+
+ Retrieves the value of an asset that was materialized during the execution of the job.
+
+ Parameters: asset_key (CoercibleToAssetKey) – The key of the asset to retrieve.Returns: The value of the retrieved asset.Return type: Any
+
+
+
+
+
+
output_for_node
+
+
+ Retrieves output value with a particular name from the in-process run of the job.
+
+ Parameters:
+ - node_str (str) – Name of the op/graph whose output should be retrieved. If the intended
+ - output_name (Optional[str]) – Name of the output on the op/graph to retrieve. Defaults to
+
+
+ Returns: The value of the retrieved output.Return type: Any
+
+
+
+
+
+
output_value
+
+
+ Retrieves output of top-level job, if an output is returned.
+
+ Parameters: output_name (Optional[str]) – The name of the output to retrieve. Defaults to result,
+ the default output name in dagster.Returns: The value of the retrieved output.Return type: Any
+
+
+
+
+
+
property all_events
+
+
+ All dagster events emitted during execution.
+
+ Type: List[[DagsterEvent](#dagster.DagsterEvent)]
+
+
+
+
+
+
property dagster_run
+
+
+ The Dagster run that was executed.
+
+ Type: [DagsterRun](internals.mdx#dagster.DagsterRun)
+
+
+
+
+
+
property job_def
+
+
+ The job definition that was executed.
+
+ Type: [JobDefinition](jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
property run_id
+
+
+ The run ID of the executed [`DagsterRun`](internals.mdx#dagster.DagsterRun).
+
+ Type: str
+
+
+
+
+
+
+
+
+
+
class dagster.JobExecutionResult
+
+
+ Result object returned by [`dagster.execute_job()`](#dagster.execute_job).
+
+ Used for retrieving run success, events, and outputs from execute_job.
+ Users should not directly instantiate this class.
+
+ Events and run information can be retrieved off of the object directly. In
+ order to access outputs, the ExecuteJobResult object needs to be opened
+ as a context manager, which will re-initialize the resources from
+ execution.
+
+
+
output_for_node
+
+
+ Retrieves output value with a particular name from the run of the job.
+
+ In order to use this method, the ExecuteJobResult object must be opened as a context manager. If this method is used without opening the context manager, it will result in a [`DagsterInvariantViolationError`](errors.mdx#dagster.DagsterInvariantViolationError).
+
+ Parameters:
+ - node_str (str) – Name of the op/graph whose output should be retrieved. If the intended
+ - output_name (Optional[str]) – Name of the output on the op/graph to retrieve. Defaults to
+
+
+ Returns: The value of the retrieved output.Return type: Any
+
+
+
+
+
+
output_value
+
+
+ Retrieves output of top-level job, if an output is returned.
+
+ In order to use this method, the ExecuteJobResult object must be opened as a context manager. If this method is used without opening the context manager, it will result in a [`DagsterInvariantViolationError`](errors.mdx#dagster.DagsterInvariantViolationError). If the top-level job has no output, calling this method will also result in a [`DagsterInvariantViolationError`](errors.mdx#dagster.DagsterInvariantViolationError).
+
+ Parameters: output_name (Optional[str]) – The name of the output to retrieve. Defaults to result,
+ the default output name in dagster.Returns: The value of the retrieved output.Return type: Any
+
+
+
+
+
+
property all_events
+
+
+ List of all events yielded by the job execution.
+
+ Type: Sequence[[DagsterEvent](#dagster.DagsterEvent)]
+
+
+
+
+
+
property dagster_run
+
+
+ The Dagster run that was executed.
+
+ Type: [DagsterRun](internals.mdx#dagster.DagsterRun)
+
+
+
+
+
+
property job_def
+
+
+ The job definition that was executed.
+
+ Type: [JobDefinition](jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
property run_id
+
+
+ The id of the Dagster run that was executed.
+
+ Type: str
+
+
+
+
+
+
+
+
+
+
class dagster.DagsterEvent
+
+
+ Events yielded by op and job execution.
+
+ Users should not instantiate this class.
+
+
+
event_type_value
+
+
+ Value for a DagsterEventType.
+
+ Type: str
+
+
+
+
+
+
job_name
+
+
+ Type: str
+
+
+
+
+
+
node_handle
+
+
+ Type: NodeHandle
+
+
+
+
+
+
step_kind_value
+
+
+ Value for a StepKind.
+
+ Type: str
+
+
+
+
+
+
logging_tags
+
+
+ Type: Dict[str, str]
+
+
+
+
+
+
event_specific_data
+
+
+ Type must correspond to event_type_value.
+
+ Type: Any
+
+
+
+
+
+
message
+
+
+ Type: str
+
+
+
+
+
+
pid
+
+
+ Type: int
+
+
+
+
+
+
step_key
+
+
+ DEPRECATED
+
+ Type: Optional[str]
+
+
+
+
+
+
property asset_key
+
+
+ For events that correspond to a specific asset_key / partition
+ (ASSET_MATERIALIZTION, ASSET_OBSERVATION, ASSET_MATERIALIZATION_PLANNED), returns that
+ asset key. Otherwise, returns None.
+
+ Type: Optional[[AssetKey](assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
property event_type
+
+
+ The type of this event.
+
+ Type: [DagsterEventType](#dagster.DagsterEventType)
+
+
+
+
+
+
property is_asset_materialization_planned
+
+
+ If this event is of type ASSET_MATERIALIZATION_PLANNED.
+
+ Type: bool
+
+
+
+
+
+
property is_asset_observation
+
+
+ If this event is of type ASSET_OBSERVATION.
+
+ Type: bool
+
+
+
+
+
+
property is_engine_event
+
+
+ If this event is of type ENGINE_EVENT.
+
+ Type: bool
+
+
+
+
+
+
property is_expectation_result
+
+
+ If this event is of type STEP_EXPECTATION_RESULT.
+
+ Type: bool
+
+
+
+
+
+
property is_failure
+
+
+ If this event represents the failure of a run or step.
+
+ Type: bool
+
+
+
+
+
+
property is_handled_output
+
+
+ If this event is of type HANDLED_OUTPUT.
+
+ Type: bool
+
+
+
+
+
+
property is_hook_event
+
+
+ If this event relates to the execution of a hook.
+
+ Type: bool
+
+
+
+
+
+
property is_loaded_input
+
+
+ If this event is of type LOADED_INPUT.
+
+ Type: bool
+
+
+
+
+
+
property is_resource_init_failure
+
+
+ If this event is of type RESOURCE_INIT_FAILURE.
+
+ Type: bool
+
+
+
+
+
+
property is_step_event
+
+
+ If this event relates to a specific step.
+
+ Type: bool
+
+
+
+
+
+
property is_step_failure
+
+
+ If this event is of type STEP_FAILURE.
+
+ Type: bool
+
+
+
+
+
+
property is_step_materialization
+
+
+ If this event is of type ASSET_MATERIALIZATION.
+
+ Type: bool
+
+
+
+
+
+
property is_step_restarted
+
+
+ If this event is of type STEP_RESTARTED.
+
+ Type: bool
+
+
+
+
+
+
property is_step_skipped
+
+
+ If this event is of type STEP_SKIPPED.
+
+ Type: bool
+
+
+
+
+
+
property is_step_start
+
+
+ If this event is of type STEP_START.
+
+ Type: bool
+
+
+
+
+
+
property is_step_success
+
+
+ If this event is of type STEP_SUCCESS.
+
+ Type: bool
+
+
+
+
+
+
property is_step_up_for_retry
+
+
+ If this event is of type STEP_UP_FOR_RETRY.
+
+ Type: bool
+
+
+
+
+
+
property is_successful_output
+
+
+ If this event is of type STEP_OUTPUT.
+
+ Type: bool
+
+
+
+
+
+
property partition
+
+
+ For events that correspond to a specific asset_key / partition
+ (ASSET_MATERIALIZTION, ASSET_OBSERVATION, ASSET_MATERIALIZATION_PLANNED), returns that
+ partition. Otherwise, returns None.
+
+ Type: Optional[[AssetKey](assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
+
+
+
+
class dagster.DagsterEventType
+
+ The types of events that may be yielded by op and job execution.
+
+
+
+
+
+
+
+
+
+## Reconstructable jobs
+
+
+
class dagster.reconstructable
+
+
+ Create a `ReconstructableJob` from a
+ function that returns a [`JobDefinition`](jobs.mdx#dagster.JobDefinition)/[`JobDefinition`](jobs.mdx#dagster.JobDefinition),
+ or a function decorated with [`@job`](jobs.mdx#dagster.job).
+
+ When your job must cross process boundaries, e.g., for execution on multiple nodes or
+ in different systems (like `dagstermill`), Dagster must know how to reconstruct the job
+ on the other side of the process boundary.
+
+ Passing a job created with `~dagster.GraphDefinition.to_job` to `reconstructable()`,
+ requires you to wrap that job’s definition in a module-scoped function, and pass that function
+ instead:
+
+ ```python
+ from dagster import graph, reconstructable
+
+ @graph
+ def my_graph():
+ ...
+
+ def define_my_job():
+ return my_graph.to_job()
+
+ reconstructable(define_my_job)
+ ```
+ This function implements a very conservative strategy for reconstruction, so that its behavior
+ is easy to predict, but as a consequence it is not able to reconstruct certain kinds of jobs
+ or jobs, such as those defined by lambdas, in nested scopes (e.g., dynamically within a method
+ call), or in interactive environments such as the Python REPL or Jupyter notebooks.
+
+ If you need to reconstruct objects constructed in these ways, you should use
+ `build_reconstructable_job()` instead, which allows you to
+ specify your own reconstruction strategy.
+
+ Examples:
+
+ ```python
+ from dagster import job, reconstructable
+
+ @job
+ def foo_job():
+ ...
+
+ reconstructable_foo_job = reconstructable(foo_job)
+
+
+ @graph
+ def foo():
+ ...
+
+ def make_bar_job():
+ return foo.to_job()
+
+ reconstructable_bar_job = reconstructable(make_bar_job)
+ ```
+
+
+
+ The default executor for a job.
+
+ This is the executor available by default on a [`JobDefinition`](jobs.mdx#dagster.JobDefinition)
+ that does not provide custom executors. This executor has a multiprocessing-enabled mode, and a
+ single-process mode. By default, multiprocessing mode is enabled. Switching between multiprocess
+ mode and in-process mode can be achieved via config.
+
+ ```yaml
+ execution:
+ config:
+ multiprocess:
+
+
+ execution:
+ config:
+ in_process:
+ ```
+ When using the multiprocess mode, `max_concurrent` and `retries` can also be configured.
+
+ ```yaml
+ execution:
+ config:
+ multiprocess:
+ max_concurrent: 4
+ retries:
+ enabled:
+ ```
+ The `max_concurrent` arg is optional and tells the execution engine how many processes may run
+ concurrently. By default, or if you set `max_concurrent` to be 0, this is the return value of
+ `python:multiprocessing.cpu_count()`.
+
+ When using the in_process mode, then only retries can be configured.
+
+ Execution priority can be configured using the `dagster/priority` tag via op metadata,
+ where the higher the number the higher the priority. 0 is the default and both positive
+ and negative numbers can be used.
+
+
+
+
+
+
+
dagster.in_process_executor ExecutorDefinition
+
+
+ The in-process executor executes all steps in a single process.
+
+ To select it, include the following top-level fragment in config:
+
+ ```yaml
+ execution:
+ in_process:
+ ```
+ Execution priority can be configured using the `dagster/priority` tag via op metadata,
+ where the higher the number the higher the priority. 0 is the default and both positive
+ and negative numbers can be used.
+
+
+
+
+
+
+
dagster.multiprocess_executor ExecutorDefinition
+
+
+ The multiprocess executor executes each step in an individual process.
+
+ Any job that does not specify custom executors will use the multiprocess_executor by default.
+ To configure the multiprocess executor, include a fragment such as the following in your run
+ config:
+
+ ```yaml
+ execution:
+ config:
+ multiprocess:
+ max_concurrent: 4
+ ```
+ The `max_concurrent` arg is optional and tells the execution engine how many processes may run
+ concurrently. By default, or if you set `max_concurrent` to be None or 0, this is the return value of
+ `python:multiprocessing.cpu_count()`.
+
+ Execution priority can be configured using the `dagster/priority` tag via op metadata,
+ where the higher the number the higher the priority. 0 is the default and both positive
+ and negative numbers can be used.
+
+
+
+
+
+
+
+
+
+
+
+## Contexts
+
+
+
class dagster.AssetExecutionContext
+
+
+
+
add_asset_metadata
+
+
+ Add metadata to an asset materialization event. This metadata will be
+ available in the Dagster UI.
+
+ Parameters:
+ - metadata (Mapping[str, Any]) – The metadata to add to the asset
+ - asset_key (Optional[CoercibleToAssetKey]) – The asset key to add metadata to.
+ - partition_key (Optional[str]) – The partition key to add metadata to, if
+
+
+ Examples:
+
+ Adding metadata to the asset materialization event for a single asset:
+
+ ```python
+ import dagster as dg
+
+ @dg.asset
+ def my_asset(context):
+ # Add metadata
+ context.add_asset_metadata({"key": "value"})
+ ```
+ Adding metadata to the asset materialization event for a particular partition of a partitioned asset:
+
+ ```python
+ import dagster as dg
+
+ @dg.asset(partitions_def=dg.StaticPartitionsDefinition(["a", "b"]))
+ def my_asset(context):
+ # Adds metadata to all partitions currently being materialized, since no
+ # partition is specified.
+ context.add_asset_metadata({"key": "value"})
+
+ for partition_key in context.partition_keys:
+ # Add metadata only to the event for partition "a"
+ if partition_key == "a":
+ context.add_asset_metadata({"key": "value"}, partition_key=partition_key)
+ ```
+ Adding metadata to the asset materialization event for a particular asset in a multi-asset.
+
+ ```python
+ import dagster as dg
+
+ @dg.multi_asset(specs=[dg.AssetSpec("asset1"), dg.AssetSpec("asset2")])
+ def my_multi_asset(context):
+ # Add metadata to the materialization event for "asset1"
+ context.add_asset_metadata({"key": "value"}, asset_key="asset1")
+
+ # THIS line will fail since asset key is not specified:
+ context.add_asset_metadata({"key": "value"})
+ ```
+
+
+
+
+
+
add_output_metadata
+
+
+ Add metadata to one of the outputs of an op.
+
+ This can be invoked multiple times per output in the body of an op. If the same key is
+ passed multiple times, the value associated with the last call will be used.
+
+ Parameters:
+ - metadata (Mapping[str, Any]) – The metadata to attach to the output
+ - output_name (Optional[str]) – The name of the output to attach metadata to. If there is only one output on the op, then this argument does not need to be provided. The metadata will automatically be attached to the only output.
+ - mapping_key (Optional[str]) – The mapping key of the output to attach metadata to. If the
+
+
+ Examples:
+
+ ```python
+ from dagster import Out, op
+ from typing import Tuple
+
+ @op
+ def add_metadata(context):
+ context.add_output_metadata({"foo", "bar"})
+ return 5 # Since the default output is called "result", metadata will be attached to the output "result".
+
+ @op(out={"a": Out(), "b": Out()})
+ def add_metadata_two_outputs(context) -> Tuple[str, int]:
+ context.add_output_metadata({"foo": "bar"}, output_name="b")
+ context.add_output_metadata({"baz": "bat"}, output_name="a")
+
+ return ("dog", 5)
+ ```
+
+
+
+
+
+
asset_key_for_input
+
+ Return the AssetKey for the corresponding input.
+
+
+
+
+
asset_key_for_output
+
+ Return the AssetKey for the corresponding output.
+
+
+
+
+
asset_partition_key_for_input
+
+
+ Returns the partition key of the upstream asset corresponding to the given input.
+
+ Parameters: input_name (str) – The name of the input to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_for_input("upstream_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_for_input("self_dependent_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-20"
+ ```
+
+
+
+
+
+
asset_partition_key_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method asset_partition_key_for_output on AssetExecutionContext. Use context.partition_key instead..
+
+ :::
+
+ Returns the asset partition key for the given output.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_for_output())
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_for_output("first_asset"))
+ context.log.info(context.asset_partition_key_for_output("second_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ # "2023-08-21"
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ ```
+
+
+
+
+
+
asset_partition_key_range_for_input
+
+
+ Return the PartitionKeyRange for the corresponding input. Errors if the asset depends on a
+ non-contiguous chunk of the input.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_key_range_for_input` to get the range of partitions keys of the input that
+ are relevant to that backfill.
+
+ Parameters: input_name (str) – The name of the input to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_range_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_range_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-20", end="2023-08-24")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_range_for_input("self_dependent_asset"))
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-20", end="2023-08-24")
+ ```
+
+
+
+
+
+
asset_partition_key_range_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method asset_partition_key_range_for_output on AssetExecutionContext. Use context.partition_key_range instead..
+
+ :::
+
+ Return the PartitionKeyRange for the corresponding output. Errors if the run is not partitioned.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_key_range_for_output` to get all of the partitions being materialized
+ by the backfill.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition key range for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_range_for_output())
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_range_for_output("first_asset"))
+ context.log.info(context.asset_partition_key_range_for_output("second_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_range_for_output())
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ ```
+
+
+
+
+
+
asset_partition_keys_for_input
+
+
+ Returns a list of the partition keys of the upstream asset corresponding to the
+ given input.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_keys_for_input` to get all of the partition keys of the input that
+ are relevant to that backfill.
+
+ Parameters: input_name (str) – The name of the input to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_keys_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_keys_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-20", "2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24"]
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_keys_for_input("self_dependent_asset"))
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-20", "2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24"]
+ ```
+
+
+
+
+
+
asset_partition_keys_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method asset_partition_keys_for_output on AssetExecutionContext. Use context.partition_keys instead..
+
+ :::
+
+ Returns a list of the partition keys for the given output.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_keys_for_output` to get all of the partitions being materialized
+ by the backfill.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition keys for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_keys_for_output())
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_keys_for_output("first_asset"))
+ context.log.info(context.asset_partition_keys_for_output("second_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_keys_for_output())
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ ```
+
+
+
+
+
+
asset_partitions_def_for_input
+
+
+ The PartitionsDefinition on the upstream asset corresponding to this input.
+
+ Parameters: input_name (str) – The name of the input to get the PartitionsDefinition for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_def_for_input("upstream_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+ ```
+
+
+
+
+
+
asset_partitions_def_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method asset_partitions_def_for_output on AssetExecutionContext. Use context.assets_def.partitions_def instead..
+
+ :::
+
+ The PartitionsDefinition on the asset corresponding to this output.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the PartitionsDefinition for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_def_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_def_for_output("first_asset"))
+ context.log.info(context.asset_partitions_def_for_output("second_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+ # DailyPartitionsDefinition("2023-08-20")
+ ```
+
+
+
+
+
+
asset_partitions_time_window_for_input
+
+
+ The time window for the partitions of the input asset.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partitions_time_window_for_input` to get the time window of the input that
+ are relevant to that backfill.
+
+ Raises an error if either of the following are true:
+ - The input asset has no partitioning.
+ - The input asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+ Parameters: input_name (str) – The name of the input to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("upstream_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("upstream_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-21")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("self_dependent_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-21")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-25")
+ ```
+
+
+
+
+
+
asset_partitions_time_window_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method asset_partitions_time_window_for_output on AssetExecutionContext. Use context.partition_time_window instead..
+
+ :::
+
+ The time window for the partitions of the output asset.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partitions_time_window_for_output` to get the TimeWindow of all of the partitions
+ being materialized by the backfill.
+
+ Raises an error if either of the following are true:
+ - The output asset has no partitioning.
+ - The output asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_time_window_for_output())
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_time_window_for_output("first_asset"))
+ context.log.info(context.asset_partitions_time_window_for_output("second_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partitions_time_window_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+ ```
+
+
+
+
+
+
get_asset_provenance
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Return the provenance information for the most recent materialization of an asset.
+
+ Parameters: asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – Key of the asset for which to retrieve provenance.Returns:
+ Provenance information for the most recent
+ materialization of the asset. Returns None if the asset was never materialized or
+ the materialization record is too old to contain provenance information.
+
+ Return type: Optional[DataProvenance]
+
+
+
+
+
+
get_mapping_key
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method get_mapping_key on AssetExecutionContext. Use context.op_execution_context.get_mapping_key instead..
+
+ :::
+
+ Which mapping_key this execution is for if downstream of a DynamicOutput, otherwise None.
+
+
+
+
+
+
+
get_tag
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method get_tag on AssetExecutionContext. Use context.run.tags.get(key) instead..
+
+ :::
+
+ Get a logging tag.
+
+ Parameters: key (tag) – The tag to get.Returns: The value of the tag, if present.Return type: Optional[str]
+
+
+
+
+
+
has_tag
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method has_tag on AssetExecutionContext. Use key in context.run.tags instead..
+
+ :::
+
+ Check if a logging tag is set.
+
+ Parameters: key (str) – The tag to check.Returns: Whether the tag is set.Return type: bool
+
+
+
+
+
+
log_event
+
+
+ Log an AssetMaterialization, AssetObservation, or ExpectationResult from within the body of an op.
+
+ Events logged with this method will appear in the list of DagsterEvents, as well as the event log.
+
+ Parameters: event (Union[[*AssetMaterialization*](ops.mdx#dagster.AssetMaterialization), [*AssetObservation*](assets.mdx#dagster.AssetObservation), [*ExpectationResult*](ops.mdx#dagster.ExpectationResult)]) – The event to log.
+ Examples:
+
+ ```python
+ from dagster import op, AssetMaterialization
+
+ @op
+ def log_materialization(context):
+ context.log_event(AssetMaterialization("foo"))
+ ```
+
+
+
+
+
+
output_for_asset_key
+
+ Return the output name for the corresponding asset key.
+
+
+
+
+
property asset_key
+
+ The AssetKey for the current asset. In a multi_asset, use asset_key_for_output instead.
+
+
+
+
+
property asset_partition_key_range
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_key_range` instead..
+
+ :::
+
+ The range of partition keys for the current run.
+
+ If run is for a single partition key, return a PartitionKeyRange with the same start and
+ end. Raises an error if the current run is not a partitioned run.
+
+
+
+
+
+
+
property assets_def
+
+ The backing AssetsDefinition for what is currently executing, errors if not available.
+
+
+
+
+
property has_assets_def
+
+ If there is a backing AssetsDefinition for what is currently executing.
+
+
+
+
+
property has_partition_key
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property has_partition_key_range
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property instance
+
+
+ The current Dagster instance.
+
+ Type: [DagsterInstance](internals.mdx#dagster.DagsterInstance)
+
+
+
+
+
+
property job_def
+
+ The definition for the currently executing job. Information like the job name, and job tags
+ can be found on the JobDefinition.
+ Returns: JobDefinition.
+
+
+
+
+
property job_name
+
+
+ The name of the currently executing pipeline.
+
+ Type: str
+
+
+
+
+
+
property log
+
+
+ The log manager available in the execution context. Logs will be viewable in the Dagster UI.
+ Returns: DagsterLogManager.
+
+ Example:
+
+ ```python
+ @asset
+ def logger(context):
+ context.log.info("Info level message")
+ ```
+
+
+
+
+
+
property op_config
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method op_config on AssetExecutionContext. Use context.op_execution_context.op_config instead..
+
+ :::
+
+ The parsed config specific to this op.
+
+ Type: Any
+
+
+
+
+
+
property op_def
+
+
+ The current op definition.
+
+ Type: [OpDefinition](ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
property partition_key
+
+
+ The partition key for the current run.
+
+ Raises an error if the current run is not a partitioned run. Or if the current run is operating
+ over a range of partitions (ie. a backfill of several partitions executed in a single run).
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_key)
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ ```
+
+
+
+
+
+
property partition_key_range
+
+
+ The range of partition keys for the current run.
+
+ If run is for a single partition key, returns a PartitionKeyRange with the same start and
+ end. Raises an error if the current run is not a partitioned run.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_key_range)
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ ```
+
+
+
+
+
+
property partition_keys
+
+
+ Returns a list of the partition keys for the current run.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `partition_keys` to get all of the partitions being materialized
+ by the backfill.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(partitions_def=partitions_def)
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_keys)
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ ```
+
+
+
+
+
+
property partition_time_window
+
+
+ The partition time window for the current run.
+
+ Raises an error if the current run is not a partitioned run, or if the job’s partition
+ definition is not a TimeWindowPartitionsDefinition.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_time_window)
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+ ```
+
+
+
+
+
+
property pdb
+
+
+ Gives access to pdb debugging from within the asset. Materializing the asset via the
+ Dagster UI or CLI will enter the pdb debugging context in the process used to launch the UI or
+ run the CLI.
+
+ Returns: dagster.utils.forked_pdb.ForkedPdb
+
+ Example:
+
+ ```python
+ @asset
+ def debug(context):
+ context.pdb.set_trace()
+ ```
+
+
+
+
+
+
property resources
+
+
+ The currently available resources.
+
+ Type: Resources
+
+
+
+
+
+
property selected_asset_check_keys
+
+ Get the asset check keys that correspond to the current selection of assets this execution is expected to materialize.
+
+
+
+
+
property selected_asset_keys
+
+ Get the set of AssetKeys this execution is expected to materialize.
+
+
+
+
+
property selected_output_names
+
+
+ :::danger[deprecated]
+ This API will be removed in version a future release.
+ You have called the deprecated method selected_output_names on AssetExecutionContext. Use context.op_execution_context.selected_output_names instead..
+
+ :::
+
+ Get the output names that correspond to the current selection of assets this execution is expected to materialize.
+
+
+
+
+
+
+
+
+
+
+
class dagster.OpExecutionContext
+
+
+ The `context` object that can be made available as the first argument to the function
+ used for computing an op or asset.
+
+ This context object provides system information such as resources, config, and logging.
+
+ To construct an execution context for testing purposes, use [`dagster.build_op_context()`](#dagster.build_op_context).
+
+ Example:
+
+ ```python
+ from dagster import op, OpExecutionContext
+
+ @op
+ def hello_world(context: OpExecutionContext):
+ context.log.info("Hello, world!")
+ ```
+
+
add_output_metadata
+
+
+ Add metadata to one of the outputs of an op.
+
+ This can be invoked multiple times per output in the body of an op. If the same key is
+ passed multiple times, the value associated with the last call will be used.
+
+ Parameters:
+ - metadata (Mapping[str, Any]) – The metadata to attach to the output
+ - output_name (Optional[str]) – The name of the output to attach metadata to. If there is only one output on the op, then this argument does not need to be provided. The metadata will automatically be attached to the only output.
+ - mapping_key (Optional[str]) – The mapping key of the output to attach metadata to. If the
+
+
+ Examples:
+
+ ```python
+ from dagster import Out, op
+ from typing import Tuple
+
+ @op
+ def add_metadata(context):
+ context.add_output_metadata({"foo", "bar"})
+ return 5 # Since the default output is called "result", metadata will be attached to the output "result".
+
+ @op(out={"a": Out(), "b": Out()})
+ def add_metadata_two_outputs(context) -> Tuple[str, int]:
+ context.add_output_metadata({"foo": "bar"}, output_name="b")
+ context.add_output_metadata({"baz": "bat"}, output_name="a")
+
+ return ("dog", 5)
+ ```
+
+
+
+
+
+
asset_key_for_input
+
+ Return the AssetKey for the corresponding input.
+
+
+
+
+
asset_key_for_output
+
+ Return the AssetKey for the corresponding output.
+
+
+
+
+
asset_partition_key_for_input
+
+
+ Returns the partition key of the upstream asset corresponding to the given input.
+
+ Parameters: input_name (str) – The name of the input to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_for_input("upstream_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_for_input("self_dependent_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-20"
+ ```
+
+
+
+
+
+
asset_partition_key_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_key` instead..
+
+ :::
+
+ Returns the asset partition key for the given output.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_for_output())
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_for_output("first_asset"))
+ context.log.info(context.asset_partition_key_for_output("second_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ # "2023-08-21"
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ ```
+
+
+
+
+
+
asset_partition_key_range_for_input
+
+
+ Return the PartitionKeyRange for the corresponding input. Errors if the asset depends on a
+ non-contiguous chunk of the input.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_key_range_for_input` to get the range of partitions keys of the input that
+ are relevant to that backfill.
+
+ Parameters: input_name (str) – The name of the input to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_range_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_key_range_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-20", end="2023-08-24")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_range_for_input("self_dependent_asset"))
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-20", end="2023-08-24")
+ ```
+
+
+
+
+
+
asset_partition_key_range_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_key_range` instead..
+
+ :::
+
+ Return the PartitionKeyRange for the corresponding output. Errors if the run is not partitioned.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_key_range_for_output` to get all of the partitions being materialized
+ by the backfill.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition key range for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_range_for_output())
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_key_range_for_output("first_asset"))
+ context.log.info(context.asset_partition_key_range_for_output("second_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_key_range_for_output())
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ ```
+
+
+
+
+
+
asset_partition_keys_for_input
+
+
+ Returns a list of the partition keys of the upstream asset corresponding to the
+ given input.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_keys_for_input` to get all of the partition keys of the input that
+ are relevant to that backfill.
+
+ Parameters: input_name (str) – The name of the input to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_keys_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partition_keys_for_input("upstream_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-20", "2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24"]
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_keys_for_input("self_dependent_asset"))
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-20", "2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24"]
+ ```
+
+
+
+
+
+
asset_partition_keys_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_keys` instead..
+
+ :::
+
+ Returns a list of the partition keys for the given output.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partition_keys_for_output` to get all of the partitions being materialized
+ by the backfill.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the partition keys for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_keys_for_output())
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partition_keys_for_output("first_asset"))
+ context.log.info(context.asset_partition_keys_for_output("second_asset"))
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partition_keys_for_output())
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ ```
+
+
+
+
+
+
asset_partitions_def_for_input
+
+
+ The PartitionsDefinition on the upstream asset corresponding to this input.
+
+ Parameters: input_name (str) – The name of the input to get the PartitionsDefinition for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_def_for_input("upstream_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+ ```
+
+
+
+
+
+
asset_partitions_def_for_output
+
+
+ The PartitionsDefinition on the asset corresponding to this output.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the PartitionsDefinition for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_def_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_def_for_output("first_asset"))
+ context.log.info(context.asset_partitions_def_for_output("second_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # DailyPartitionsDefinition("2023-08-20")
+ # DailyPartitionsDefinition("2023-08-20")
+ ```
+
+
+
+
+
+
asset_partitions_time_window_for_input
+
+
+ The time window for the partitions of the input asset.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partitions_time_window_for_input` to get the time window of the input that
+ are relevant to that backfill.
+
+ Raises an error if either of the following are true:
+ - The input asset has no partitioning.
+ - The input asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+ Parameters: input_name (str) – The name of the input to get the partition key for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def upstream_asset():
+ ...
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("upstream_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ ins={
+ "upstream_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ partitions_def=partitions_def,
+ )
+ def another_asset(context: AssetExecutionContext, upstream_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("upstream_asset"))
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-21")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partitions_time_window_for_input("self_dependent_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-21")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-20", "2023-08-25")
+ ```
+
+
+
+
+
+
asset_partitions_time_window_for_output
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_time_window` instead..
+
+ :::
+
+ The time window for the partitions of the output asset.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `asset_partitions_time_window_for_output` to get the TimeWindow of all of the partitions
+ being materialized by the backfill.
+
+ Raises an error if either of the following are true:
+ - The output asset has no partitioning.
+ - The output asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+ Parameters: output_name (str) – For assets defined with the `@asset` decorator, the name of the output
+ will be automatically provided. For assets defined with `@multi_asset`, `output_name`
+ should be the op output associated with the asset key (as determined by AssetOut)
+ to get the time window for.
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_time_window_for_output())
+
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+ @multi_asset(
+ outs={
+ "first_asset": AssetOut(key=["my_assets", "first_asset"]),
+ "second_asset": AssetOut(key=["my_assets", "second_asset"])
+ }
+ partitions_def=partitions_def,
+ )
+ def a_multi_asset(context: AssetExecutionContext):
+ context.log.info(context.asset_partitions_time_window_for_output("first_asset"))
+ context.log.info(context.asset_partitions_time_window_for_output("second_asset"))
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+ # TimeWindow("2023-08-21", "2023-08-26")
+
+
+ @asset(
+ partitions_def=partitions_def,
+ ins={
+ "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1))
+ }
+ )
+ def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset):
+ context.log.info(context.asset_partitions_time_window_for_output())
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-26")
+ ```
+
+
+
+
+
+
get_asset_provenance
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Return the provenance information for the most recent materialization of an asset.
+
+ Parameters: asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – Key of the asset for which to retrieve provenance.Returns:
+ Provenance information for the most recent
+ materialization of the asset. Returns None if the asset was never materialized or
+ the materialization record is too old to contain provenance information.
+
+ Return type: Optional[DataProvenance]
+
+
+
+
+
+
get_mapping_key
+
+ Which mapping_key this execution is for if downstream of a DynamicOutput, otherwise None.
+
+
+
+
+
get_tag
+
+
+ Get a logging tag.
+
+ Parameters: key (tag) – The tag to get.Returns: The value of the tag, if present.Return type: Optional[str]
+
+
+
+
+
+
has_tag
+
+
+ Check if a logging tag is set.
+
+ Parameters: key (str) – The tag to check.Returns: Whether the tag is set.Return type: bool
+
+
+
+
+
+
log_event
+
+
+ Log an AssetMaterialization, AssetObservation, or ExpectationResult from within the body of an op.
+
+ Events logged with this method will appear in the list of DagsterEvents, as well as the event log.
+
+ Parameters: event (Union[[*AssetMaterialization*](ops.mdx#dagster.AssetMaterialization), [*AssetObservation*](assets.mdx#dagster.AssetObservation), [*ExpectationResult*](ops.mdx#dagster.ExpectationResult)]) – The event to log.
+ Examples:
+
+ ```python
+ from dagster import op, AssetMaterialization
+
+ @op
+ def log_materialization(context):
+ context.log_event(AssetMaterialization("foo"))
+ ```
+
+
+
+
+
+
output_for_asset_key
+
+ Return the output name for the corresponding asset key.
+
+
+
+
+
property asset_key
+
+ The AssetKey for the current asset. In a multi_asset, use asset_key_for_output instead.
+
+
+
+
+
property asset_partition_key_range
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `partition_key_range` instead..
+
+ :::
+
+ The range of partition keys for the current run.
+
+ If run is for a single partition key, return a PartitionKeyRange with the same start and
+ end. Raises an error if the current run is not a partitioned run.
+
+
+
+
+
+
+
property assets_def
+
+ The backing AssetsDefinition for what is currently executing, errors if not available.
+
+
+
+
+
property has_assets_def
+
+ If there is a backing AssetsDefinition for what is currently executing.
+
+
+
+
+
property has_partition_key
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property has_partition_key_range
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property instance
+
+
+ The current Dagster instance.
+
+ Type: [DagsterInstance](internals.mdx#dagster.DagsterInstance)
+
+
+
+
+
+
property job_def
+
+
+ The currently executing job.
+
+ Type: [JobDefinition](jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
property job_name
+
+
+ The name of the currently executing pipeline.
+
+ Type: str
+
+
+
+
+
+
property log
+
+
+ The log manager available in the execution context.
+
+ Type: [DagsterLogManager](loggers.mdx#dagster.DagsterLogManager)
+
+
+
+
+
+
property op_config
+
+
+ The parsed config specific to this op.
+
+ Type: Any
+
+
+
+
+
+
property op_def
+
+
+ The current op definition.
+
+ Type: [OpDefinition](ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
property partition_key
+
+
+ The partition key for the current run.
+
+ Raises an error if the current run is not a partitioned run. Or if the current run is operating
+ over a range of partitions (ie. a backfill of several partitions executed in a single run).
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_key)
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # "2023-08-21"
+ ```
+
+
+
+
+
+
property partition_key_range
+
+
+ The range of partition keys for the current run.
+
+ If run is for a single partition key, returns a PartitionKeyRange with the same start and
+ end. Raises an error if the current run is not a partitioned run.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_key_range)
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # PartitionKeyRange(start="2023-08-21", end="2023-08-25")
+ ```
+
+
+
+
+
+
property partition_keys
+
+
+ Returns a list of the partition keys for the current run.
+
+ If you want to write your asset to support running a backfill of several partitions in a single run,
+ you can use `partition_keys` to get all of the partitions being materialized
+ by the backfill.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(partitions_def=partitions_def)
+ def an_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_keys)
+
+
+ # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log:
+ # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"]
+ ```
+
+
+
+
+
+
property partition_time_window
+
+
+ The partition time window for the current run.
+
+ Raises an error if the current run is not a partitioned run, or if the job’s partition
+ definition is not a TimeWindowPartitionsDefinition.
+
+ Examples:
+
+ ```python
+ partitions_def = DailyPartitionsDefinition("2023-08-20")
+
+ @asset(
+ partitions_def=partitions_def
+ )
+ def my_asset(context: AssetExecutionContext):
+ context.log.info(context.partition_time_window)
+
+ # materializing the 2023-08-21 partition of this asset will log:
+ # TimeWindow("2023-08-21", "2023-08-22")
+ ```
+
+
+
+
+
+
property pdb
+
+
+ Gives access to pdb debugging from within the op.
+
+ Example:
+
+ ```python
+ @op
+ def debug(context):
+ context.pdb.set_trace()
+ ```
+ Type: dagster.utils.forked_pdb.ForkedPdb
+
+
+
+
+
+
property resources
+
+
+ The currently available resources.
+
+ Type: Resources
+
+
+
+
+
+
property retry_number
+
+ Which retry attempt is currently executing i.e. 0 for initial attempt, 1 for first retry, etc.
+
+
+
+
+
property run
+
+
+ The current run.
+
+ Type: [DagsterRun](internals.mdx#dagster.DagsterRun)
+
+
+
+
+
+
property run_config
+
+
+ The run config for the current execution.
+
+ Type: dict
+
+
+
+
+
+
property run_id
+
+
+ The id of the current execution’s run.
+
+ Type: str
+
+
+
+
+
+
property selected_asset_check_keys
+
+ Get the asset check keys that correspond to the current selection of assets this execution is expected to materialize.
+
+
+
+
+
property selected_asset_keys
+
+ Get the set of AssetKeys this execution is expected to materialize.
+
+
+
+
+
property selected_output_names
+
+ Get the output names that correspond to the current selection of assets this execution is expected to materialize.
+
+
+
+
+
+
+
+
+
dagster.build_op_context
+
+
+ Builds op execution context from provided parameters.
+
+ `build_op_context` can be used as either a function or context manager. If there is a
+ provided resource that is a context manager, then `build_op_context` must be used as a
+ context manager. This function can be used to provide the context argument when directly
+ invoking a op.
+
+ Parameters:
+ - resources (Optional[Dict[str, Any]]) – The resources to provide to the context. These can be
+ - op_config (Optional[Mapping[str, Any]]) – The config to provide to the op.
+ - resources_config (Optional[Mapping[str, Any]]) – The config to provide to the resources.
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The dagster instance configured for the context.
+ - mapping_key (Optional[str]) – A key representing the mapping key from an upstream dynamic
+ - partition_key (Optional[str]) – String value representing partition key to execute with.
+ - partition_key_range (Optional[[*PartitionKeyRange*](partitions.mdx#dagster.PartitionKeyRange)]) – Partition key range to execute with.
+ - run_tags – Optional[Mapping[str, str]]: The tags for the executing run.
+
+
+ Examples:
+
+ ```python
+ context = build_op_context()
+ op_to_invoke(context)
+
+ with build_op_context(resources={"foo": context_manager_resource}) as context:
+ op_to_invoke(context)
+ ```
+
+
+
+
+
+
dagster.build_asset_context
+
+
+ Builds asset execution context from provided parameters.
+
+ `build_asset_context` can be used as either a function or context manager. If there is a
+ provided resource that is a context manager, then `build_asset_context` must be used as a
+ context manager. This function can be used to provide the context argument when directly
+ invoking an asset.
+
+ Parameters:
+ - resources (Optional[Dict[str, Any]]) – The resources to provide to the context. These can be
+ - resources_config (Optional[Mapping[str, Any]]) – The config to provide to the resources.
+ - asset_config (Optional[Mapping[str, Any]]) – The config to provide to the asset.
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The dagster instance configured for the context.
+ - partition_key (Optional[str]) – String value representing partition key to execute with.
+ - partition_key_range (Optional[[*PartitionKeyRange*](partitions.mdx#dagster.PartitionKeyRange)]) – Partition key range to execute with.
+ - run_tags – Optional[Mapping[str, str]]: The tags for the executing run.
+
+
+ Examples:
+
+ ```python
+ context = build_asset_context()
+ asset_to_invoke(context)
+
+ with build_asset_context(resources={"foo": context_manager_resource}) as context:
+ asset_to_invoke(context)
+ ```
+
+
+
+
+
+
class dagster.TypeCheckContext
+
+
+ The `context` object available to a type check function on a DagsterType.
+
+
+
property log
+
+ Centralized log dispatch from user code.
+
+
+
+
+
property resources
+
+ An object whose attributes contain the resources available to this op.
+
+
+
+
+
property run_id
+
+ The id of this job run.
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Job configuration
+
+
+
dagster.validate_run_config
+
+
+ Function to validate a provided run config blob against a given job.
+
+ If validation is successful, this function will return a dictionary representation of the
+ validated config actually used during execution.
+
+ Parameters:
+ - job_def ([*JobDefinition*](jobs.mdx#dagster.JobDefinition)) – The job definition to validate run
+ - run_config (Optional[Dict[str, Any]]) – The run config to validate
+
+
+ Returns: A dictionary representation of the validated config.Return type: Dict[str, Any]
+
+
+
+
+
+
+
+### Run Config Schema
+
+>
+
+The `run_config` used for jobs has the following schema:
+
+ ```default
+ {
+ # configuration for execution, required if executors require config
+ execution: {
+ # the name of one, and only one available executor, typically 'in_process' or 'multiprocess'
+ __executor_name__: {
+ # executor-specific config, if required or permitted
+ config: {
+ ...
+ }
+ }
+ },
+
+ # configuration for loggers, required if loggers require config
+ loggers: {
+ # the name of an available logger
+ __logger_name__: {
+ # logger-specific config, if required or permitted
+ config: {
+ ...
+ }
+ },
+ ...
+ },
+
+ # configuration for resources, required if resources require config
+ resources: {
+ # the name of a resource
+ __resource_name__: {
+ # resource-specific config, if required or permitted
+ config: {
+ ...
+ }
+ },
+ ...
+ },
+
+ # configuration for underlying ops, required if ops require config
+ ops: {
+
+ # these keys align with the names of the ops, or their alias in this job
+ __op_name__: {
+
+ # pass any data that was defined via config_field
+ config: ...,
+
+ # configurably specify input values, keyed by input name
+ inputs: {
+ __input_name__: {
+ # if an dagster_type_loader is specified, that schema must be satisfied here;
+ # scalar, built-in types will generally allow their values to be specified directly:
+ value: ...
+ }
+ },
+
+ }
+ },
+
+ }
+ ```
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/external-assets.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/external-assets.mdx
new file mode 100644
index 0000000000000..a13569848c064
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/external-assets.mdx
@@ -0,0 +1,54 @@
+---
+title: 'external assets (experimental)'
+title_meta: 'external assets (experimental) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'external assets (experimental) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# External assets (Experimental)
+
+As Dagster doesn’t control scheduling or materializing [external assets](https://docs.dagster.io/guides/build/assets/external-assets), it’s up to you to keep their metadata updated. The APIs in this reference can be used to keep external assets updated in Dagster.
+
+
+
+
+## Instance API
+
+External asset events can be recorded using `DagsterInstance.report_runless_asset_event()` on `DagsterInstance`.
+
+Example: Reporting an asset materialization:
+
+ ```python
+ from dagster import DagsterInstance, AssetMaterialization, AssetKey
+
+ instance = DagsterInstance.get()
+ instance.report_runless_asset_event(AssetMaterialization(AssetKey("example_asset")))
+ ```
+Example: Reporting an asset check evaluation:
+
+ ```python
+ from dagster import DagsterInstance, AssetCheckEvaluation, AssetCheckKey
+
+ instance = DagsterInstance.get()
+ instance.report_runless_asset_event(
+ AssetCheckEvaluation(
+ asset_key=AssetKey("example_asset"),
+ check_name="example_check",
+ passed=True
+ )
+ )
+ ```
+
+
+
+
+
+## REST API
+
+Refer to the [External assets REST API reference](https://docs.dagster.io/apidocs/external-assets-rest) for information and examples on the available APIs.
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/graphs.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/graphs.mdx
new file mode 100644
index 0000000000000..12fe20021870a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/graphs.mdx
@@ -0,0 +1,545 @@
+---
+title: 'graphs'
+title_meta: 'graphs API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'graphs Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Graphs
+
+The core of a job is a _graph_ of ops - connected via data dependencies.
+
+
+
@dagster.graph
+
+
+ Create an op graph with the specified parameters from the decorated composition function.
+
+ Using this decorator allows you to build up a dependency graph by writing a
+ function that invokes ops (or other graphs) and passes the output to subsequent invocations.
+
+ Parameters:
+ - name (Optional[str]) – The name of the op graph. Must be unique within any [`RepositoryDefinition`](repositories.mdx#dagster.RepositoryDefinition) containing the graph.
+ - description (Optional[str]) – A human-readable description of the graph.
+ - input_defs (Optional[List[InputDefinition]]) –
+
+ Information about the inputs that this graph maps. Information provided here
+ will be combined with what can be inferred from the function signature, with these
+ explicit InputDefinitions taking precedence.
+
+ - output_defs (Optional[List[OutputDefinition]]) –
+
+ Output definitions for the graph. If not provided explicitly, these will be inferred from typehints.
+
+ Uses of these outputs in the body of the decorated composition function, as well as the
+ return value of the decorated function, will be used to infer the appropriate set of
+ [`OutputMappings`](#dagster.OutputMapping) for the underlying
+ [`GraphDefinition`](#dagster.GraphDefinition).
+
+ - ins (Optional[Dict[str, [*GraphIn*](#dagster.GraphIn)]]) – Information about the inputs that this graph maps. Information provided here
+ - out –
+
+ Information about the outputs that this graph maps. Information provided here will be
+ combined with what can be inferred from the return type signature if the function does
+ not use yield.
+
+
+
+
+
+
+
+
+
class dagster.GraphDefinition
+
+
+ Defines a Dagster op graph.
+
+ An op graph is made up of
+
+ - Nodes, which can either be an op (the functional unit of computation), or another graph.
+ - Dependencies, which determine how the values produced by nodes as outputs flow from
+
+
+ End users should prefer the [`@graph`](#dagster.graph) decorator. GraphDefinition is generally
+ intended to be used by framework authors or for programatically generated graphs.
+
+ Parameters:
+ - name (str) – The name of the graph. Must be unique within any [`GraphDefinition`](#dagster.GraphDefinition)
+ - description (Optional[str]) – A human-readable description of the job.
+ - node_defs (Optional[Sequence[NodeDefinition]]) – The set of ops / graphs used in this graph.
+ - dependencies (Optional[Dict[Union[str, [*NodeInvocation*](#dagster.NodeInvocation)], Dict[str, [*DependencyDefinition*](#dagster.DependencyDefinition)]]]) – A structure that declares the dependencies of each op’s inputs on the outputs of other
+ - input_mappings (Optional[Sequence[[*InputMapping*](#dagster.InputMapping)]]) – Defines the inputs to the nested graph, and
+ - output_mappings (Optional[Sequence[[*OutputMapping*](#dagster.OutputMapping)]]) – Defines the outputs of the nested graph,
+ - config (Optional[[*ConfigMapping*](config.mdx#dagster.ConfigMapping)]) – Defines the config of the graph, and how its schema maps
+ - tags (Optional[Dict[str, Any]]) – Arbitrary metadata for any execution of the graph.
+ - composition_fn (Optional[Callable]) – The function that defines this graph. Used to generate
+
+
+ Examples:
+
+ ```python
+ @op
+ def return_one():
+ return 1
+
+ @op
+ def add_one(num):
+ return num + 1
+
+ graph_def = GraphDefinition(
+ name='basic',
+ node_defs=[return_one, add_one],
+ dependencies={'add_one': {'num': DependencyDefinition('return_one')}},
+ )
+ ```
+
+
alias
+
+
+ Aliases the graph with a new name.
+
+ Can only be used in the context of a [`@graph`](#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.alias("my_graph_alias")
+ ```
+
+
+
+
+
+
execute_in_process
+
+
+ Execute this graph in-process, collecting results in-memory.
+
+ Parameters:
+ - run_config (Optional[Mapping[str, Any]]) – Run config to provide to execution. The configuration for the underlying graph
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The instance to execute against, an ephemeral one will be used if none provided.
+ - resources (Optional[Mapping[str, Any]]) – The resources needed if any are required. Can provide resource instances directly,
+ - raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur.
+ - op_selection (Optional[List[str]]) – A list of op selection queries (including single op
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of the graph.
+
+
+ Returns: [`ExecuteInProcessResult`](execution.mdx#dagster.ExecuteInProcessResult)
+
+
+
+
+
+
tag
+
+
+ Attaches the provided tags to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.tag({"my_tag": "my_value"})
+ ```
+
+
+
+
+
+
to_job
+
+
+ Make this graph in to an executable Job by providing remaining components required for execution.
+
+ Parameters:
+ - name (Optional[str]) – The name for the Job. Defaults to the name of the this graph.
+ - resource_defs (Optional[Mapping [str, object]]) – Resources that are required by this graph for execution.
+ - config –
+
+ Describes how the job is parameterized at runtime.
+
+ If no value is provided, then the schema for the job’s run config is a standard
+ format based on its ops and resources.
+
+ If a dictionary is provided, then it must conform to the standard config schema, and
+ it will be used as the job’s run config for the job whenever the job is executed.
+ The values provided will be viewable and editable in the Dagster UI, so be
+ careful with secrets.
+
+ If a [`ConfigMapping`](config.mdx#dagster.ConfigMapping) object is provided, then the schema for the job’s run config is
+ determined by the config mapping, and the ConfigMapping, which should return
+ configuration in the standard format to configure the job.
+
+ - tags (Optional[Mapping[str, object]]) – A set of key-value tags that annotate the job and can
+ - run_tags (Optional[Mapping[str, object]]) – A set of key-value tags that will be automatically attached to runs launched by this
+ - metadata (Optional[Mapping[str, RawMetadataValue]]) – Arbitrary information that will be attached to the JobDefinition and be viewable in the Dagster UI.
+ - logger_defs (Optional[Mapping[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – A dictionary of string logger identifiers to their implementations.
+ - executor_def (Optional[[*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]) – How this Job will be executed. Defaults to [`multi_or_in_process_executor`](execution.mdx#dagster.multi_or_in_process_executor),
+ - op_retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The default retry policy for all ops in this job.
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines a discrete set of partition
+ - asset_layer (Optional[AssetLayer]) – Top level information about the assets this job
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of a job.
+
+
+ Returns: JobDefinition
+
+
+
+
+
+
with_hooks
+
+
+ Attaches the provided hooks to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.with_hooks({my_hook})
+ ```
+
+
+
+
+
+
with_retry_policy
+
+
+ Attaches the provided retry policy to the graph immutably.
+
+ Can only be used in the context of a [`@graph`](#dagster.graph), [`@job`](jobs.mdx#dagster.job), or `@asset_graph` decorated function.
+
+ Examples:
+ ```python
+ @job
+ def do_it_all():
+ my_graph.with_retry_policy(RetryPolicy(max_retries=5))
+ ```
+
+
+
+
+
+
property config_mapping
+
+
+ The config mapping for the graph, if present.
+
+ By specifying a config mapping function, you can override the configuration for the child nodes contained within a graph.
+
+
+
+
+
+
+
property input_mappings
+
+
+ Input mappings for the graph.
+
+ An input mapping is a mapping from an input of the graph to an input of a child node.
+
+
+
+
+
+
+
property name
+
+ The name of the graph.
+
+
+
+
+
property output_mappings
+
+
+ Output mappings for the graph.
+
+ An output mapping is a mapping from an output of the graph to an output of a child node.
+
+
+
+
+
+
+
property tags
+
+ The tags associated with the graph.
+
+
+
+
+
+
+
+
+
class dagster.GraphIn
+
+
+ Represents information about an input that a graph maps.
+
+ Parameters: description (Optional[str]) – Human-readable description of the input.
+
+
+
+
+
+
class dagster.GraphOut
+
+
+ Represents information about the outputs that a graph maps.
+
+ Parameters: description (Optional[str]) – Human-readable description of the output.
+
+
+
+
+
+
+## Explicit dependencies
+
+
+
class dagster.DependencyDefinition
+
+
+ Represents an edge in the DAG of nodes (ops or graphs) forming a job.
+
+ This object is used at the leaves of a dictionary structure that represents the complete
+ dependency structure of a job whose keys represent the dependent node and dependent
+ input, so this object only contains information about the dependee.
+
+ Concretely, if the input named ‘input’ of op_b depends on the output named ‘result’ of
+ op_a, and the output named ‘other_result’ of graph_a, the structure will look as follows:
+
+ ```python
+ dependency_structure = {
+ 'my_downstream_op': {
+ 'input': DependencyDefinition('my_upstream_op', 'result')
+ }
+ 'my_downstream_op': {
+ 'input': DependencyDefinition('my_upstream_graph', 'result')
+ }
+ }
+ ```
+ In general, users should prefer not to construct this class directly or use the
+ [`JobDefinition`](jobs.mdx#dagster.JobDefinition) API that requires instances of this class. Instead, use the
+ [`@job`](jobs.mdx#dagster.job) API:
+
+ ```python
+ @job
+ def the_job():
+ node_b(node_a())
+ ```
+ Parameters:
+ - node (str) – The name of the node (op or graph) that is depended on, that is, from which the value
+ - output (Optional[str]) – The name of the output that is depended on. (default: “result”)
+ - description (Optional[str]) – Human-readable description of this dependency.
+
+
+
+
is_fan_in
+
+ Return True if the dependency is fan-in (always False for DependencyDefinition).
+
+
+
+
+
+
+
+
+
class dagster.MultiDependencyDefinition
+
+
+ Represents a fan-in edge in the DAG of op instances forming a job.
+
+ This object is used only when an input of type `List[T]` is assembled by fanning-in multiple
+ upstream outputs of type `T`.
+
+ This object is used at the leaves of a dictionary structure that represents the complete
+ dependency structure of a job whose keys represent the dependent ops or graphs and dependent
+ input, so this object only contains information about the dependee.
+
+ Concretely, if the input named ‘input’ of op_c depends on the outputs named ‘result’ of
+ op_a and op_b, this structure will look as follows:
+
+ ```python
+ dependency_structure = {
+ 'op_c': {
+ 'input': MultiDependencyDefinition(
+ [
+ DependencyDefinition('op_a', 'result'),
+ DependencyDefinition('op_b', 'result')
+ ]
+ )
+ }
+ }
+ ```
+ In general, users should prefer not to construct this class directly or use the
+ [`JobDefinition`](jobs.mdx#dagster.JobDefinition) API that requires instances of this class. Instead, use the
+ [`@job`](jobs.mdx#dagster.job) API:
+
+ ```python
+ @job
+ def the_job():
+ op_c(op_a(), op_b())
+ ```
+ Parameters: dependencies (List[Union[[*DependencyDefinition*](#dagster.DependencyDefinition), Type[MappedInputPlaceHolder]]]) – List of
+ upstream dependencies fanned in to this input.
+
+
get_dependencies_and_mappings
+
+ Return the combined list of dependencies contained by this object, inculding of [`DependencyDefinition`](#dagster.DependencyDefinition) and `MappedInputPlaceholder` objects.
+
+
+
+
+
get_node_dependencies
+
+ Return the list of [`DependencyDefinition`](#dagster.DependencyDefinition) contained by this object.
+
+
+
+
+
is_fan_in
+
+ Return True if the dependency is fan-in (always True for MultiDependencyDefinition).
+
+
+
+
+
+
+
+
+
class dagster.NodeInvocation
+
+
+ Identifies an instance of a node in a graph dependency structure.
+
+ Parameters:
+ - name (str) – Name of the node of which this is an instance.
+ - alias (Optional[str]) – Name specific to this instance of the node. Necessary when there are
+ - tags (Optional[Dict[str, Any]]) – Optional tags values to extend or override those
+ - hook_defs (Optional[AbstractSet[[*HookDefinition*](hooks.mdx#dagster.HookDefinition)]]) – A set of hook definitions applied to the
+
+
+ Examples:
+ In general, users should prefer not to construct this class directly or use the
+ [`JobDefinition`](jobs.mdx#dagster.JobDefinition) API that requires instances of this class. Instead, use the
+ [`@job`](jobs.mdx#dagster.job) API:
+
+ ```python
+ from dagster import job
+
+ @job
+ def my_job():
+ other_name = some_op.alias('other_name')
+ some_graph(other_name(some_op))
+ ```
+
+
+
+
+
+
class dagster.OutputMapping
+
+
+ Defines an output mapping for a graph.
+
+ Parameters:
+ - graph_output_name (str) – Name of the output in the graph being mapped to.
+ - mapped_node_name (str) – Named of the node (op/graph) that the output is being mapped from.
+ - mapped_node_output_name (str) – Name of the output in the node (op/graph) that is being mapped from.
+ - graph_output_description (Optional[str]) – A description of the output in the graph being mapped from.
+ - from_dynamic_mapping (bool) – Set to true if the node being mapped to is a mapped dynamic node.
+ - dagster_type (Optional[[*DagsterType*](types.mdx#dagster.DagsterType)]) – deprecateddagster_type should come from the underlying op Output.) The dagster type of the graph’s output being mapped to.
+
+
+ Examples:
+
+ ```python
+ from dagster import OutputMapping, GraphDefinition, op, graph, GraphOut
+
+ @op
+ def emit_five(x):
+ return 5
+
+ # The following two graph definitions are equivalent
+ GraphDefinition(
+ name="the_graph",
+ node_defs=[emit_five],
+ output_mappings=[
+ OutputMapping(
+ graph_output_name="result", # Default output name
+ mapped_node_name="emit_five",
+ mapped_node_output_name="result"
+ )
+ ]
+ )
+
+ @graph(out=GraphOut())
+ def the_graph:
+ return emit_five()
+ ```
+
+
+
+
+
+
class dagster.InputMapping
+
+
+ Defines an input mapping for a graph.
+
+ Parameters:
+ - graph_input_name (str) – Name of the input in the graph being mapped from.
+ - mapped_node_name (str) – Named of the node (op/graph) that the input is being mapped to.
+ - mapped_node_input_name (str) – Name of the input in the node (op/graph) that is being mapped to.
+ - fan_in_index (Optional[int]) – The index in to a fanned input, otherwise None.
+ - graph_input_description (Optional[str]) – A description of the input in the graph being mapped from.
+ - dagster_type (Optional[[*DagsterType*](types.mdx#dagster.DagsterType)]) – deprecateddagster_type should come from the upstream op Output.) The dagster type of the graph’s input
+
+
+ Examples:
+
+ ```python
+ from dagster import InputMapping, GraphDefinition, op, graph
+
+ @op
+ def needs_input(x):
+ return x + 1
+
+ # The following two graph definitions are equivalent
+ GraphDefinition(
+ name="the_graph",
+ node_defs=[needs_input],
+ input_mappings=[
+ InputMapping(
+ graph_input_name="maps_x", mapped_node_name="needs_input",
+ mapped_node_input_name="x"
+ )
+ ]
+ )
+
+ @graph
+ def the_graph(maps_x):
+ needs_input(maps_x)
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/hooks.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/hooks.mdx
new file mode 100644
index 0000000000000..2f81ec791a68d
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/hooks.mdx
@@ -0,0 +1,227 @@
+---
+title: 'hooks'
+title_meta: 'hooks API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'hooks Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Hooks
+
+
+
@dagster.success_hook
+
+
+ Create a hook on step success events with the specified parameters from the decorated function.
+
+ Parameters:
+ - name (Optional[str]) – The name of this hook.
+ - required_resource_keys (Optional[AbstractSet[str]]) – Keys for the resources required by the
+
+
+ Examples:
+
+ ```python
+ @success_hook(required_resource_keys={'slack'})
+ def slack_message_on_success(context):
+ message = 'op {} succeeded'.format(context.op.name)
+ context.resources.slack.send_message(message)
+
+ @success_hook
+ def do_something_on_success(context):
+ do_something()
+ ```
+
+
+
+
+
+
@dagster.failure_hook
+
+
+ Create a hook on step failure events with the specified parameters from the decorated function.
+
+ Parameters:
+ - name (Optional[str]) – The name of this hook.
+ - required_resource_keys (Optional[AbstractSet[str]]) – Keys for the resources required by the
+
+
+ Examples:
+
+ ```python
+ @failure_hook(required_resource_keys={'slack'})
+ def slack_message_on_failure(context):
+ message = 'op {} failed'.format(context.op.name)
+ context.resources.slack.send_message(message)
+
+ @failure_hook
+ def do_something_on_failure(context):
+ do_something()
+ ```
+
+
+
+
+
+
class dagster.HookDefinition
+
+
+ Define a hook which can be triggered during a op execution (e.g. a callback on the step
+ execution failure event during a op execution).
+
+ Parameters:
+ - name (str) – The name of this hook.
+ - hook_fn (Callable) – The callback function that will be triggered.
+ - required_resource_keys (Optional[AbstractSet[str]]) – Keys for the resources required by the
+
+
+
+
+
+
+
+
class dagster.HookContext
+
+
+ The `context` object available to a hook function on an DagsterEvent.
+
+
+
property hook_def
+
+ The hook that the context object belongs to.
+
+
+
+
+
property instance
+
+ The instance configured to run the current job.
+
+
+
+
+
property job_name
+
+ The name of the job where this hook is being triggered.
+
+
+
+
+
property log
+
+ Centralized log dispatch from user code.
+
+
+
+
+
property op_config
+
+ The parsed config specific to this op.
+
+
+
+
+
property op_exception
+
+ The thrown exception in a failed op.
+
+
+
+
+
property op_output_metadata
+
+
+ The applied output metadata.
+
+ Returns a dictionary where keys are output names and the values are:
+ - the applied output metadata in the normal case
+ - a dictionary from mapping key to corresponding metadata in the mapped case
+
+
+
+
+
+
+
+
property op_output_values
+
+
+ The computed output values.
+
+ Returns a dictionary where keys are output names and the values are:
+ - the output values in the normal case
+ - a dictionary from mapping key to corresponding value in the mapped case
+
+
+
+
+
+
+
+
property required_resource_keys
+
+ Resources required by this hook.
+
+
+
+
+
property resources
+
+ Resources available in the hook context.
+
+
+
+
+
property run_id
+
+ The id of the run where this hook is being triggered.
+
+
+
+
+
property step_key
+
+ The key for the step where this hook is being triggered.
+
+
+
+
+
+
+
+
+
dagster.build_hook_context
+
+
+ Builds hook context from provided parameters.
+
+ `build_hook_context` can be used as either a function or a context manager. If there is a
+ provided resource to `build_hook_context` that is a context manager, then it must be used as a
+ context manager. This function can be used to provide the context argument to the invocation of
+ a hook definition.
+
+ Parameters:
+ - resources (Optional[Dict[str, Any]]) – The resources to provide to the context. These can
+ - op (Optional[[*OpDefinition*](ops.mdx#dagster.OpDefinition), PendingNodeInvocation]) – The op definition which the
+ - run_id (Optional[str]) – The id of the run in which the hook is invoked (provided for mocking purposes).
+ - job_name (Optional[str]) – The name of the job in which the hook is used (provided for mocking purposes).
+ - op_exception (Optional[Exception]) – The exception that caused the hook to be triggered.
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The Dagster instance configured to run the hook.
+
+
+ Examples:
+
+ ```python
+ context = build_hook_context()
+ hook_to_invoke(context)
+
+ with build_hook_context(resources={"foo": context_manager_resource}) as context:
+ hook_to_invoke(context)
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/index.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/index.mdx
new file mode 100644
index 0000000000000..005473f027d33
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/index.mdx
@@ -0,0 +1,8 @@
+---
+sidebar_class_name: hidden
+title: API reference
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/internals.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/internals.mdx
new file mode 100644
index 0000000000000..23856c4c4c97a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/internals.mdx
@@ -0,0 +1,1568 @@
+---
+title: 'internals'
+title_meta: 'internals API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'internals Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# 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](https://join.slack.com/t/dagster/shared_invite/enQtNjEyNjkzNTA2OTkzLTI0MzdlNjU0ODVhZjQyOTMyMGM1ZDUwZDQ1YjJmYjI3YzExZGViMDI1ZDlkNTY5OThmYWVlOWM1MWVjN2I3NjU).
+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`](#dagster.InitExecutorContext) and return an instance
+ of [`Executor`](#dagster.Executor).
+
+ Parameters:
+ - name (Optional[str]) – The name of the executor.
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – 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[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data
+ - requirements (Optional[List[ExecutorRequirement]]) – Any requirements that must
+ - executor_creation_fn (Optional[Callable]) – Should accept an [`InitExecutorContext`](#dagster.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[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – 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`](#dagster.InitExecutorContext) and returns an instance of
+ [`Executor`](#dagster.Executor).
+
+
+
+
+
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](#dagster.ExecutorDefinition)
+
+
+
+
+
+
executor_config
+
+
+ The parsed config passed to the executor.
+
+ Type: dict
+
+
+
+
+
+
instance
+
+
+ The current instance.
+
+ Type: [DagsterInstance](#dagster.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()`](#dagster._core.storage.file_manager.FileManager.read),
+ [`read_data()`](#dagster._core.storage.file_manager.FileManager.read_data),
+ [`write()`](#dagster._core.storage.file_manager.FileManager.write), and
+ [`write_data()`](#dagster._core.storage.file_manager.FileManager.write_data) methods.
+
+ Parameters: file_handle ([*FileHandle*](#dagster.FileHandle)) – 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()`](#dagster._core.storage.file_manager.FileManager.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 ([*FileHandle*](#dagster.FileHandle)) – 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 ([*FileHandle*](#dagster.FileHandle)) – 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](#dagster.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.FileHandle)
+
+
+
+
+
+
+
+
+
+
dagster.local_file_manager ResourceDefinition
+
+
+ FileManager that provides abstract access to a local filesystem.
+
+ By default, files will be stored in \/storage/file_manager where
+ \ can be configured the `dagster.yaml` file in `$DAGSTER_HOME`.
+
+ Implements the [`FileManager`](#dagster._core.storage.file_manager.FileManager) API.
+
+ Examples:
+
+ ```python
+ 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:
+
+ ```python
+ @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 a `dagster.yaml` such as the
+ following:
+
+ dagster.yaml
+
+ ```YAML
+ storage:
+ postgres:
+ postgres_db:
+ username: my_username
+ password: my_password
+ hostname: my_hostname
+ db_name: my_database
+ port: 5432
+ ```
+ Parameters:
+ - instance_type (InstanceType) – Indicates whether the instance is ephemeral or persistent.
+ - local_artifact_storage ([*LocalArtifactStorage*](#dagster._core.storage.root.LocalArtifactStorage)) – The local artifact storage is used to
+ - run_storage ([*RunStorage*](#dagster._core.storage.runs.RunStorage)) – The run storage is used to store metadata about ongoing and past
+ - event_storage ([*EventLogStorage*](#dagster._core.storage.event_log.EventLogStorage)) – Used to store the structured event logs generated by
+ - compute_log_manager (Optional[[*ComputeLogManager*](#dagster._core.storage.compute_log_manager.ComputeLogManager)]) – The compute log manager handles stdout
+ - run_coordinator (Optional[RunCoordinator]) – A runs coordinator may be used to manage the execution
+ - run_launcher (Optional[[*RunLauncher*](#dagster._core.launcher.RunLauncher)]) – Optionally, a run launcher may be used to enable
+ - settings (Optional[Dict]) – Specifies certain per-instance settings,
+ - ref (Optional[[*InstanceRef*](#dagster._core.instance.InstanceRef)]) – Used by internal machinery to pass instances across process
+
+
+
+
add_dynamic_partitions
+
+
+ Add partitions to the specified [`DynamicPartitionsDefinition`](partitions.mdx#dagster.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`](partitions.mdx#dagster.DynamicPartitionsDefinition).
+ If the partition does not exist, exits silently.
+
+ Parameters:
+ - partitions_def_name (str) – The name of the DynamicPartitionsDefinition.
+ - partition_key (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](#dagster.DagsterInstance)
+
+
+
+
+
+
fetch_materializations
+
+
+ Return a list of materialization records stored in the event log storage.
+
+ Parameters:
+ - records_filter (Union[[*AssetKey*](assets.mdx#dagster.AssetKey), 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[[*AssetKey*](assets.mdx#dagster.AssetKey), 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[[*DagsterEventType*](execution.mdx#dagster.DagsterEventType), 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](#dagster.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](assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
get_asset_records
+
+
+ Return an AssetRecord for each of the given asset keys.
+
+ Parameters: asset_keys (Optional[Sequence[[*AssetKey*](assets.mdx#dagster.AssetKey)]]) – List of asset keys to retrieve records for.Returns: List of asset records.Return type: Sequence[[AssetRecord](#dagster._core.storage.event_log.AssetRecord)]
+
+
+
+
+
+
get_dynamic_partitions
+
+
+ Get the set of partition keys for the specified [`DynamicPartitionsDefinition`](partitions.mdx#dagster.DynamicPartitionsDefinition).
+
+ Parameters: partitions_def_name (str) – The name of the DynamicPartitionsDefinition.
+
+
+
+
+
+
get_latest_materialization_code_versions
+
+
+ Returns the code version used for the latest materialization of each of the provided
+ assets.
+
+ Parameters: asset_keys (Iterable[[*AssetKey*](assets.mdx#dagster.AssetKey)]) – 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](assets.mdx#dagster.AssetKey), Optional[str]]
+
+
+
+
+
+
get_latest_materialization_event
+
+
+ Fetch the latest materialization event for the given asset key.
+
+ Parameters: asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – 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](#dagster.EventLogEntry)]
+
+
+
+
+
+
get_run_by_id
+
+
+ Get a [`DagsterRun`](#dagster.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](#dagster.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](#dagster._core.storage.dagster_run.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[[*RunsFilter*](#dagster.RunsFilter)]) – 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](#dagster._core.storage.dagster_run.RunRecord)]
+
+
+
+
+
+
get_status_by_partition
+
+
+ Get the current status of provided partition_keys for the provided asset.
+
+ Parameters:
+ - asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – The asset to get per-partition status for.
+ - partition_keys (Sequence[str]) – The partitions to get status for.
+ - partitions_def ([*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)) – 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 ([*AssetKey*](assets.mdx#dagster.AssetKey)) – Asset key to check.
+
+
+
+
+
+
has_dynamic_partition
+
+
+ Check if a partition key exists for the [`DynamicPartitionsDefinition`](partitions.mdx#dagster.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
+
+
+ :::warning[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[[*AssetKey*](assets.mdx#dagster.AssetKey)]) – Asset keys to wipe.
+
+
+
+
+
+
+
+
+
+
class dagster._core.instance.InstanceRef
+
+
+ Serializable representation of a [`DagsterInstance`](#dagster.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:
+
+ ```yaml
+ 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 of `ConfigurableClassData`.
+
+ Pieces of the Dagster system which we wish to make pluggable in this way should consume a config
+ type such as:
+
+ ```python
+ {'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.
+
+
+
+
+
+
+
class dagster._core.storage.root.LocalArtifactStorage
+
+
+
+
+
+
+
+
+
+
+
+## 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` and `dagster-daemon` load, based on the values in the
+ `dagster.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`](#dagster._core.storage.runs.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[[*DagsterRunStatus*](#dagster.DagsterRunStatus)]]) – 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`](#dagster._core.storage.runs.SqlRunStorage).
+
+ Users should not directly instantiate concrete subclasses of this class; they are instantiated
+ by internal machinery when `dagster-webserver` and `dagster-graphql` load, based on the values in the
+ `dagster.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` and `dagster-graphql` load, based on the values in the `dagster.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`:
+
+ ```YAML
+ run_storage:
+ module: dagster._core.storage.runs
+ class: SqliteRunStorage
+ config:
+ base_dir: /path/to/dir
+ ```
+ The `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`](#dagster._core.storage.runs.RunStorage).
+
+ Users should not invoke this class directly.
+
+
+
+
+
+See also: [`dagster_postgres.PostgresRunStorage`](libraries/dagster-postgres.mdx#dagster_postgres.PostgresRunStorage) and [`dagster_mysql.MySQLRunStorage`](libraries/dagster-mysql.mdx#dagster_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[[*DagsterEvent*](execution.mdx#dagster.DagsterEvent)]) – 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](execution.mdx#dagster.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`](#dagster._core.storage.event_log.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 ([*DagsterEventType*](execution.mdx#dagster.DagsterEventType)) – Filter argument for dagster event type
+ - asset_key (Optional[[*AssetKey*](assets.mdx#dagster.AssetKey)]) – 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`](#dagster._core.storage.event_log.SqlEventLogStorage).
+
+ Users should not directly instantiate concrete subclasses of this class; they are instantiated
+ by internal machinery when `dagster-webserver` and `dagster-graphql` load, based on the values in the
+ `dagster.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` and `dagster-graphql` load, based on the values in the `dagster.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`:
+
+ ```YAML
+ event_log_storage:
+ module: dagster._core.storage.event_log
+ class: SqliteEventLogStorage
+ config:
+ base_dir: /path/to/dir
+ ```
+ The `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` and `dagster-graphql` load, based on the values in the `dagster.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`:
+
+ ```YAML
+ run_storage:
+ module: dagster._core.storage.event_log
+ class: ConsolidatedSqliteEventLogStorage
+ config:
+ base_dir: /path/to/dir
+ ```
+ The `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`](#dagster._core.storage.event_log.EventLogStorage).
+
+ Users should not invoke this class directly.
+
+
+
+
+
+See also: [`dagster_postgres.PostgresEventLogStorage`](libraries/dagster-postgres.mdx#dagster_postgres.PostgresEventLogStorage) and [`dagster_mysql.MySQLEventLogStorage`](libraries/dagster-mysql.mdx#dagster_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.RunLauncher
+
+
+
+
+
+
+
class dagster._core.launcher.DefaultRunLauncher
+
+ Launches runs against running GRPC servers.
+
+
+
+
+
+
+
+
+
+## Run coordinator
+
+
+
class dagster._core.run_coordinator.DefaultRunCoordinator
+
+ 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.PostgresScheduleStorage`](libraries/dagster-postgres.mdx#dagster_postgres.PostgresScheduleStorage) and [`dagster_mysql.MySQLScheduleStorage`](libraries/dagster-mysql.mdx#dagster_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](https://docs.dagster.io/guides/migrations/from-step-launchers-to-pipes).
+
+
+
class dagster.StepLauncher
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ While there is no plan to remove this functionality, for new projects, we recommend using Dagster Pipes. For more information, see https://docs.dagster.io/guides/build/external-pipelines/.
+
+ :::
+
+ 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.
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/io-managers.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/io-managers.mdx
new file mode 100644
index 0000000000000..4343300c84b47
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/io-managers.mdx
@@ -0,0 +1,1194 @@
+---
+title: 'io managers'
+title_meta: 'io managers API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'io managers Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# IO Managers
+
+IO managers are user-provided objects that store op outputs and load them as inputs to downstream
+ops.
+
+
+
class dagster.ConfigurableIOManager
+
+
+ Base class for Dagster IO managers that utilize structured config.
+
+ This class is a subclass of both [`IOManagerDefinition`](#dagster.IOManagerDefinition), [`Config`](config.mdx#dagster.Config),
+ and [`IOManager`](#dagster.IOManager). Implementers must provide an implementation of the
+ `handle_output()` and `load_input()` methods.
+
+ Example definition:
+
+ ```python
+ class MyIOManager(ConfigurableIOManager):
+ path_prefix: List[str]
+
+ def _get_path(self, context) -> str:
+ return "/".join(context.asset_key.path)
+
+ def handle_output(self, context, obj):
+ write_csv(self._get_path(context), obj)
+
+ def load_input(self, context):
+ return read_csv(self._get_path(context))
+
+ defs = Definitions(
+ ...,
+ resources={
+ "io_manager": MyIOManager(path_prefix=["my", "prefix"])
+ }
+ )
+ ```
+
+
+
+
+
+
class dagster.ConfigurableIOManagerFactory
+
+
+ Base class for Dagster IO managers that utilize structured config. This base class
+ is useful for cases in which the returned IO manager is not the same as the class itself
+ (e.g. when it is a wrapper around the actual IO manager implementation).
+
+ This class is a subclass of both [`IOManagerDefinition`](#dagster.IOManagerDefinition) and [`Config`](config.mdx#dagster.Config).
+ Implementers should provide an implementation of the `resource_function()` method,
+ which should return an instance of [`IOManager`](#dagster.IOManager).
+
+ Example definition:
+
+ ```python
+ class ExternalIOManager(IOManager):
+
+ def __init__(self, connection):
+ self._connection = connection
+
+ def handle_output(self, context, obj):
+ ...
+
+ def load_input(self, context):
+ ...
+
+ class ConfigurableExternalIOManager(ConfigurableIOManagerFactory):
+ username: str
+ password: str
+
+ def create_io_manager(self, context) -> IOManager:
+ with database.connect(username, password) as connection:
+ return MyExternalIOManager(connection)
+
+ defs = Definitions(
+ ...,
+ resources={
+ "io_manager": ConfigurableExternalIOManager(
+ username="dagster",
+ password=EnvVar("DB_PASSWORD")
+ )
+ }
+ )
+ ```
+
+
+
+
+
+
class dagster.IOManager
+
+
+ Base class for user-provided IO managers.
+
+ IOManagers are used to store op outputs and load them as inputs to downstream ops.
+
+ Extend this class to handle how objects are loaded and stored. Users should implement
+ `handle_output` to store an object and `load_input` to retrieve an object.
+
+
+
abstract handle_output
+
+
+ User-defined method that stores an output of an op.
+
+ Parameters:
+ - context ([*OutputContext*](#dagster.OutputContext)) – The context of the step output that produces this object.
+ - obj (Any) – The object, returned by the op, to be stored.
+
+
+
+
+
+
+
+
abstract load_input
+
+
+ User-defined method that loads an input to an op.
+
+ Parameters: context ([*InputContext*](#dagster.InputContext)) – The input context, which describes the input that’s being loaded
+ and the upstream output that’s being loaded from.Returns: The data object.Return type: Any
+
+
+
+
+
+
+
+
+
+
class dagster.IOManagerDefinition
+
+
+ Definition of an IO manager resource.
+
+ IOManagers are used to store op outputs and load them as inputs to downstream ops.
+
+ An IOManagerDefinition is a [`ResourceDefinition`](resources.mdx#dagster.ResourceDefinition) whose resource_fn returns an
+ [`IOManager`](#dagster.IOManager).
+
+ The easiest way to create an IOManagerDefnition is with the [`@io_manager`](#dagster.io_manager)
+ decorator.
+
+
+
static hardcoded_io_manager
+
+
+ A helper function that creates an `IOManagerDefinition` with a hardcoded IOManager.
+
+ Parameters:
+ - value ([*IOManager*](#dagster.IOManager)) – A hardcoded IO Manager which helps mock the definition.
+ - description ([Optional[str]]) – The description of the IO Manager. Defaults to None.
+
+
+ Returns: A hardcoded resource.Return type: [[IOManagerDefinition](#dagster.IOManagerDefinition)]
+
+
+
+
+
+
+
+
+
+
@dagster.io_manager
+
+
+ Define an IO manager.
+
+ IOManagers are used to store op outputs and load them as inputs to downstream ops.
+
+ The decorated function should accept an [`InitResourceContext`](resources.mdx#dagster.InitResourceContext) and return an
+ [`IOManager`](#dagster.IOManager).
+
+ Parameters:
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the resource config. Configuration
+ - description (Optional[str]) – A human-readable description of the resource.
+ - output_config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for per-output config. If not set,
+ - input_config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for per-input config. If not set,
+ - required_resource_keys (Optional[Set[str]]) – Keys for the resources required by the object
+ - version (Optional[str]) – (Experimental) The version of a resource function. Two wrapped
+
+
+ Examples:
+
+ ```python
+ class MyIOManager(IOManager):
+ def handle_output(self, context, obj):
+ write_csv("some/path")
+
+ def load_input(self, context):
+ return read_csv("some/path")
+
+ @io_manager
+ def my_io_manager(init_context):
+ return MyIOManager()
+
+ @op(out=Out(io_manager_key="my_io_manager_key"))
+ def my_op(_):
+ return do_stuff()
+
+ @job(resource_defs={"my_io_manager_key": my_io_manager})
+ def my_job():
+ my_op()
+ ```
+
+
+
+
+
+
+
+## Input and Output Contexts
+
+
+
class dagster.InputContext
+
+
+ The `context` object available to the load_input method of [`InputManager`](#dagster.InputManager).
+
+ Users should not instantiate this object directly. In order to construct
+ an InputContext for testing an IO Manager’s load_input method, use
+ [`dagster.build_input_context()`](#dagster.build_input_context).
+
+ Example:
+
+ ```python
+ from dagster import IOManager, InputContext
+
+ class MyIOManager(IOManager):
+ def load_input(self, context: InputContext):
+ ...
+ ```
+
+
get_asset_identifier
+
+ The sequence of strings making up the AssetKey for the asset being loaded as an input.
+ If the asset is partitioned, the identifier contains the partition key as the final element in the
+ sequence. For example, for the asset key `AssetKey(["foo", "bar", "baz"])`, materialized with
+ partition key “2023-06-01”, `get_asset_identifier` will return `["foo", "bar", "baz", "2023-06-01"]`.
+
+
+
+
+
get_identifier
+
+
+ Utility method to get a collection of identifiers that as a whole represent a unique
+ step input.
+
+ If not using memoization, the unique identifier collection consists of
+
+ - `run_id`: the id of the run which generates the input.
+ - `step_key`: the key for a compute step.
+ - `name`: the name of the output. (default: ‘result’).
+
+
+ If using memoization, the `version` corresponding to the step output is used in place of
+ the `run_id`.
+
+ Returns: A list of identifiers, i.e. (run_id or version), step_key, and output_nameReturn type: List[str, …]
+
+
+
+
+
+
property asset_key
+
+ The `AssetKey` of the asset that is being loaded as an input.
+
+
+
+
+
property asset_partition_key
+
+
+ The partition key for input asset.
+
+ Raises an error if the input asset has no partitioning, or if the run covers a partition
+ range for the input asset.
+
+
+
+
+
+
+
property asset_partition_key_range
+
+
+ The partition key range for input asset.
+
+ Raises an error if the input asset has no partitioning.
+
+
+
+
+
+
+
property asset_partition_keys
+
+
+ The partition keys for input asset.
+
+ Raises an error if the input asset has no partitioning.
+
+
+
+
+
+
+
property asset_partitions_def
+
+ The PartitionsDefinition on the upstream asset corresponding to this input.
+
+
+
+
+
property asset_partitions_time_window
+
+
+ The time window for the partitions of the input asset.
+
+ Raises an error if either of the following are true:
+ - The input asset has no partitioning.
+ - The input asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+
+
+
+
+
+
property config
+
+ The config attached to the input that we’re loading.
+
+
+
+
+
property dagster_type
+
+ The type of this input.
+ Dagster types do not propagate from an upstream output to downstream inputs,
+ and this property only captures type information for the input that is either
+ passed in explicitly with [`AssetIn`](assets.mdx#dagster.AssetIn) or [`In`](ops.mdx#dagster.In), or can be
+ infered from type hints. For an asset input, the Dagster type from the upstream
+ asset definition is ignored.
+
+
+
+
+
property definition_metadata
+
+ A dict of metadata that is assigned to the InputDefinition that we’re loading.
+ This property only contains metadata passed in explicitly with [`AssetIn`](assets.mdx#dagster.AssetIn)
+ or [`In`](ops.mdx#dagster.In). To access metadata of an upstream asset or op definition,
+ use the definition_metadata in [`InputContext.upstream_output`](#dagster.InputContext.upstream_output).
+
+
+
+
+
property has_asset_key
+
+ Returns True if an asset is being loaded as input, otherwise returns False. A return value of False
+ indicates that an output from an op is being loaded as the input.
+
+
+
+
+
property has_asset_partitions
+
+ Returns True if the asset being loaded as input is partitioned.
+
+
+
+
+
property has_input_name
+
+ If we’re the InputContext is being used to load the result of a run from outside the run,
+ then it won’t have an input name.
+
+
+
+
+
property has_partition_key
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property log
+
+ The log manager to use for this input.
+
+
+
+
+
property metadata
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.0.
+ Use definition_metadata instead.
+
+ :::
+
+ Use definitiion_metadata instead.
+
+ Type: Deprecated
+
+
+
+
+
+
property name
+
+ The name of the input that we’re loading.
+
+
+
+
+
property op_def
+
+ The definition of the op that’s loading the input.
+
+
+
+
+
property partition_key
+
+
+ The partition key for the current run.
+
+ Raises an error if the current run is not a partitioned run.
+
+
+
+
+
+
+
property resource_config
+
+ The config associated with the resource that initializes the InputManager.
+
+
+
+
+
property resources
+
+ The resources required by the resource that initializes the
+ input manager. If using the `@input_manager()` decorator, these resources
+ correspond to those requested with the required_resource_keys parameter.
+
+
+
+
+
property upstream_output
+
+ Info about the output that produced the object we’re loading.
+
+
+
+
+
+
+
+
+
class dagster.OutputContext
+
+
+ The context object that is available to the handle_output method of an [`IOManager`](#dagster.IOManager).
+
+ Users should not instantiate this object directly. To construct an
+ OutputContext for testing an IO Manager’s handle_output method, use
+ [`dagster.build_output_context()`](#dagster.build_output_context).
+
+ Example:
+
+ ```python
+ from dagster import IOManager, OutputContext
+
+ class MyIOManager(IOManager):
+ def handle_output(self, context: OutputContext, obj):
+ ...
+ ```
+
+
add_output_metadata
+
+
+ Add a dictionary of metadata to the handled output.
+
+ Metadata entries added will show up in the HANDLED_OUTPUT and ASSET_MATERIALIZATION events for the run.
+
+ Parameters: metadata (Mapping[str, RawMetadataValue]) – A metadata dictionary to log
+ Examples:
+
+ ```python
+ from dagster import IOManager
+
+ class MyIOManager(IOManager):
+ def handle_output(self, context, obj):
+ context.add_output_metadata({"foo": "bar"})
+ ```
+
+
+
+
+
+
get_asset_identifier
+
+ The sequence of strings making up the AssetKey for the asset being stored as an output.
+ If the asset is partitioned, the identifier contains the partition key as the final element in the
+ sequence. For example, for the asset key `AssetKey(["foo", "bar", "baz"])` materialized with
+ partition key “2023-06-01”, `get_asset_identifier` will return `["foo", "bar", "baz", "2023-06-01"]`.
+
+
+
+
+
get_identifier
+
+
+ Utility method to get a collection of identifiers that as a whole represent a unique
+ step output.
+
+ If not using memoization, the unique identifier collection consists of
+
+ - `run_id`: the id of the run which generates the output.
+ - `step_key`: the key for a compute step.
+ - `name`: the name of the output. (default: ‘result’).
+
+
+ If using memoization, the `version` corresponding to the step output is used in place of
+ the `run_id`.
+
+ Returns: A list of identifiers, i.e. (run_id or version), step_key, and output_nameReturn type: Sequence[str, …]
+
+
+
+
+
+
log_event
+
+
+ Log an AssetMaterialization or AssetObservation from within the body of an io manager’s handle_output method.
+
+ Events logged with this method will appear in the event log.
+
+ Parameters: event (Union[[*AssetMaterialization*](ops.mdx#dagster.AssetMaterialization), [*AssetObservation*](assets.mdx#dagster.AssetObservation)]) – The event to log.
+ Examples:
+
+ ```python
+ from dagster import IOManager, AssetMaterialization
+
+ class MyIOManager(IOManager):
+ def handle_output(self, context, obj):
+ context.log_event(AssetMaterialization("foo"))
+ ```
+
+
+
+
+
+
property asset_key
+
+ The `AssetKey` of the asset that is being stored as an output.
+
+
+
+
+
property asset_partition_key
+
+
+ The partition key for output asset.
+
+ Raises an error if the output asset has no partitioning, or if the run covers a partition
+ range for the output asset.
+
+
+
+
+
+
+
property asset_partition_key_range
+
+
+ The partition key range for output asset.
+
+ Raises an error if the output asset has no partitioning.
+
+
+
+
+
+
+
property asset_partition_keys
+
+
+ The partition keys for the output asset.
+
+ Raises an error if the output asset has no partitioning.
+
+
+
+
+
+
+
property asset_partitions_def
+
+ The PartitionsDefinition on the asset corresponding to this output.
+
+
+
+
+
property asset_partitions_time_window
+
+
+ The time window for the partitions of the output asset.
+
+ Raises an error if either of the following are true:
+ - The output asset has no partitioning.
+ - The output asset is not partitioned with a TimeWindowPartitionsDefinition or a
+ MultiPartitionsDefinition with one time-partitioned dimension.
+
+
+
+
+
+
+
property asset_spec
+
+ The `AssetSpec` that is being stored as an output.
+
+
+
+
+
property config
+
+ The configuration for the output.
+
+
+
+
+
property dagster_type
+
+ The type of this output.
+
+
+
+
+
property definition_metadata
+
+ A dict of the metadata that is assigned to the OutputDefinition that produced
+ the output. Metadata is assigned to an OutputDefinition either directly on the OutputDefinition
+ or in the @asset decorator.
+
+
+
+
+
property has_asset_key
+
+ Returns True if an asset is being stored, otherwise returns False. A return value of False
+ indicates that an output from an op is being stored.
+
+
+
+
+
property has_asset_partitions
+
+ Returns True if the asset being stored is partitioned.
+
+
+
+
+
property has_partition_key
+
+ Whether the current run is a partitioned run.
+
+
+
+
+
property log
+
+ The log manager to use for this output.
+
+
+
+
+
property mapping_key
+
+ The key that identifies a unique mapped output. None for regular outputs.
+
+
+
+
+
property metadata
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.0.
+ Use definition_metadata instead.
+
+ :::
+
+ used definition_metadata instead.
+
+ Type: Deprecated
+
+
+
+
+
+
property name
+
+ The name of the output that produced the output.
+
+
+
+
+
property op_def
+
+ The definition of the op that produced the output.
+
+
+
+
+
property output_metadata
+
+ A dict of the metadata that is assigned to the output at execution time.
+
+
+
+
+
property partition_key
+
+
+ The partition key for the current run.
+
+ Raises an error if the current run is not a partitioned run.
+
+
+
+
+
+
+
property resource_config
+
+ The config associated with the resource that initializes the InputManager.
+
+
+
+
+
property resources
+
+ The resources required by the output manager, specified by the required_resource_keys
+ parameter.
+
+
+
+
+
property run_id
+
+ The id of the run that produced the output.
+
+
+
+
+
property step_key
+
+ The step_key for the compute step that produced the output.
+
+
+
+
+
property version
+
+ (Experimental) The version of the output.
+
+
+
+
+
+
+
+
+
dagster.build_input_context
+
+
+ Builds input context from provided parameters.
+
+ `build_input_context` can be used as either a function, or a context manager. If resources
+ that are also context managers are provided, then `build_input_context` must be used as a
+ context manager.
+
+ Parameters:
+ - name (Optional[str]) – The name of the input that we’re loading.
+ - config (Optional[Any]) – The config attached to the input that we’re loading.
+ - definition_metadata (Optional[Dict[str, Any]]) – A dict of metadata that is assigned to the
+ - upstream_output (Optional[[*OutputContext*](#dagster.OutputContext)]) – Info about the output that produced the object
+ - dagster_type (Optional[[*DagsterType*](types.mdx#dagster.DagsterType)]) – The type of this input.
+ - resource_config (Optional[Dict[str, Any]]) – The resource config to make available from the
+ - resources (Optional[Dict[str, Any]]) – The resources to make available from the context.
+ - asset_key (Optional[Union[[*AssetKey*](assets.mdx#dagster.AssetKey), Sequence[str], str]]) – The asset key attached to the InputDefinition.
+ - op_def (Optional[[*OpDefinition*](ops.mdx#dagster.OpDefinition)]) – The definition of the op that’s loading the input.
+ - step_context (Optional[[*StepExecutionContext*](internals.mdx#dagster.StepExecutionContext)]) – For internal use.
+ - partition_key (Optional[str]) – String value representing partition key to execute with.
+ - asset_partition_key_range (Optional[[*PartitionKeyRange*](partitions.mdx#dagster.PartitionKeyRange)]) – The range of asset partition keys
+ - asset_partitions_def – Optional[PartitionsDefinition]: The PartitionsDefinition of the asset
+
+
+ Examples:
+
+ ```python
+ build_input_context()
+
+ with build_input_context(resources={"foo": context_manager_resource}) as context:
+ do_something
+ ```
+
+
+
+
+
+
dagster.build_output_context
+
+
+ Builds output context from provided parameters.
+
+ `build_output_context` can be used as either a function, or a context manager. If resources
+ that are also context managers are provided, then `build_output_context` must be used as a
+ context manager.
+
+ Parameters:
+ - step_key (Optional[str]) – The step_key for the compute step that produced the output.
+ - name (Optional[str]) – The name of the output that produced the output.
+ - definition_metadata (Optional[Mapping[str, Any]]) – A dict of the metadata that is assigned to the
+ - mapping_key (Optional[str]) – The key that identifies a unique mapped output. None for regular outputs.
+ - config (Optional[Any]) – The configuration for the output.
+ - dagster_type (Optional[[*DagsterType*](types.mdx#dagster.DagsterType)]) – The type of this output.
+ - version (Optional[str]) – (Experimental) The version of the output.
+ - resource_config (Optional[Mapping[str, Any]]) – The resource config to make available from the
+ - resources (Optional[Resources]) – The resources to make available from the context.
+ - op_def (Optional[[*OpDefinition*](ops.mdx#dagster.OpDefinition)]) – The definition of the op that produced the output.
+ - asset_key – Optional[Union[AssetKey, Sequence[str], str]]: The asset key corresponding to the
+ - partition_key – Optional[str]: String value representing partition key to execute with.
+ - metadata (Optional[Mapping[str, Any]]) – deprecateddefinition_metadata instead.) Deprecated. Use definition_metadata instead.
+
+
+ Examples:
+
+ ```python
+ build_output_context()
+
+ with build_output_context(resources={"foo": context_manager_resource}) as context:
+ do_something
+ ```
+
+
+
+
+
+
+
+
+
+
+## Built-in IO Managers
+
+
+
dagster.FilesystemIOManager IOManagerDefinition
+
+
+ Built-in filesystem IO manager that stores and retrieves values using pickling.
+
+ The base directory that the pickle files live inside is determined by:
+
+ - The IO manager’s “base_dir” configuration value, if specified. Otherwise…
+ - A “storage/” directory underneath the value for “local_artifact_storage” in your dagster.yaml
+ - A “storage/” directory underneath the directory that the DAGSTER_HOME environment variable
+ - A temporary directory.
+
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ So, with a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ 1. Attach an IO manager to a set of assets using the reserved resource key `"io_manager"`.
+ ```python
+ from dagster import Definitions, asset, FilesystemIOManager
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": FilesystemIOManager(base_dir="/my/base/path")
+ },
+ )
+ ```
+ 2. Specify a job-level IO manager using the reserved resource key `"io_manager"`,
+ which will set the given IO manager on all ops in a job.
+
+ ```python
+ from dagster import FilesystemIOManager, job, op
+
+ @op
+ def op_a():
+ # create df ...
+ return df
+
+ @op
+ def op_b(df):
+ return df[:5]
+
+ @job(
+ resource_defs={
+ "io_manager": FilesystemIOManager(base_dir="/my/base/path")
+ }
+ )
+ def job():
+ op_b(op_a())
+ ```
+ 3. Specify IO manager on [`Out`](ops.mdx#dagster.Out), which allows you to set different IO managers on
+ different step outputs.
+
+ ```python
+ from dagster import FilesystemIOManager, job, op, Out
+
+ @op(out=Out(io_manager_key="my_io_manager"))
+ def op_a():
+ # create df ...
+ return df
+
+ @op
+ def op_b(df):
+ return df[:5]
+
+ @job(resource_defs={"my_io_manager": FilesystemIOManager()})
+ def job():
+ op_b(op_a())
+ ```
+
+
+
+
+
+
dagster.InMemoryIOManager IOManagerDefinition
+
+ I/O manager that stores and retrieves values in memory. After execution is complete, the values will
+ be garbage-collected. Note that this means that each run will not have access to values from previous runs.
+
+
+
+The `UPathIOManager` can be used to easily define filesystem-based IO Managers.
+
+
+
class dagster.UPathIOManager
+
+
+ Abstract IOManager base class compatible with local and cloud storage via universal-pathlib and fsspec.
+
+ Features:
+ - handles partitioned assets
+ - handles loading a single upstream partition
+ - handles loading multiple upstream partitions (with respect to [`PartitionMapping`](partitions.mdx#dagster.PartitionMapping))
+ - supports loading multiple partitions concurrently with async load_from_path method
+ - the get_metadata method can be customized to add additional metadata to the output
+ - the allow_missing_partitions metadata value can be set to True to skip missing partitions
+
+
+
+
+
+
+
+
+
+
+
+
+## Input Managers (Experimental)
+
+Input managers load inputs from either upstream outputs or from provided default values.
+
+
+
@dagster.input_manager
+
+
+ Define an input manager.
+
+ Input managers load op inputs, either from upstream outputs or by providing default values.
+
+ The decorated function should accept a [`InputContext`](#dagster.InputContext) and resource config, and return
+ a loaded object that will be passed into one of the inputs of an op.
+
+ The decorator produces an [`InputManagerDefinition`](#dagster.InputManagerDefinition).
+
+ Parameters:
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the resource-level config. If not
+ - description (Optional[str]) – A human-readable description of the resource.
+ - input_config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – A schema for the input-level config. Each
+ - required_resource_keys (Optional[Set[str]]) – Keys for the resources required by the input
+ - version (Optional[str]) – (Experimental) the version of the input manager definition.
+
+
+ Examples:
+
+ ```python
+ from dagster import input_manager, op, job, In
+
+ @input_manager
+ def csv_loader(_):
+ return read_csv("some/path")
+
+ @op(ins={"input1": In(input_manager_key="csv_loader_key")})
+ def my_op(_, input1):
+ do_stuff(input1)
+
+ @job(resource_defs={"csv_loader_key": csv_loader})
+ def my_job():
+ my_op()
+
+ @input_manager(config_schema={"base_dir": str})
+ def csv_loader(context):
+ return read_csv(context.resource_config["base_dir"] + "/some/path")
+
+ @input_manager(input_config_schema={"path": str})
+ def csv_loader(context):
+ return read_csv(context.config["path"])
+ ```
+
+
+
+
+
+
class dagster.InputManager
+
+ Base interface for classes that are responsible for loading solid inputs.
+
+
+
+
+
class dagster.InputManagerDefinition
+
+
+ Definition of an input manager resource.
+
+ Input managers load op inputs.
+
+ An InputManagerDefinition is a [`ResourceDefinition`](resources.mdx#dagster.ResourceDefinition) whose resource_fn returns an
+ [`InputManager`](#dagster.InputManager).
+
+ The easiest way to create an InputManagerDefinition is with the
+ [`@input_manager`](#dagster.input_manager) decorator.
+
+
+
+
+
+
+
+
+
+
+## Legacy
+
+
+
dagster.fs_io_manager IOManagerDefinition
+
+
+ Built-in filesystem IO manager that stores and retrieves values using pickling.
+
+ The base directory that the pickle files live inside is determined by:
+
+ - The IO manager’s “base_dir” configuration value, if specified. Otherwise…
+ - A “storage/” directory underneath the value for “local_artifact_storage” in your dagster.yaml
+ - A “storage/” directory underneath the directory that the DAGSTER_HOME environment variable
+ - A temporary directory.
+
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ So, with a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ 1. Attach an IO manager to a set of assets using the reserved resource key `"io_manager"`.
+ ```python
+ from dagster import Definitions, asset, fs_io_manager
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": fs_io_manager.configured({"base_dir": "/my/base/path"})
+ },
+ )
+ ```
+ 2. Specify a job-level IO manager using the reserved resource key `"io_manager"`,
+ which will set the given IO manager on all ops in a job.
+
+ ```python
+ from dagster import fs_io_manager, job, op
+
+ @op
+ def op_a():
+ # create df ...
+ return df
+
+ @op
+ def op_b(df):
+ return df[:5]
+
+ @job(
+ resource_defs={
+ "io_manager": fs_io_manager.configured({"base_dir": "/my/base/path"})
+ }
+ )
+ def job():
+ op_b(op_a())
+ ```
+ 3. Specify IO manager on [`Out`](ops.mdx#dagster.Out), which allows you to set different IO managers on
+ different step outputs.
+
+ ```python
+ from dagster import fs_io_manager, job, op, Out
+
+ @op(out=Out(io_manager_key="my_io_manager"))
+ def op_a():
+ # create df ...
+ return df
+
+ @op
+ def op_b(df):
+ return df[:5]
+
+ @job(resource_defs={"my_io_manager": fs_io_manager})
+ def job():
+ op_b(op_a())
+ ```
+
+
+
+
+
+
dagster.mem_io_manager IOManagerDefinition
+
+ Built-in IO manager that stores and retrieves values in memory.
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/jobs.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/jobs.mdx
new file mode 100644
index 0000000000000..3deb6ef9bfa62
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/jobs.mdx
@@ -0,0 +1,387 @@
+---
+title: 'jobs'
+title_meta: 'jobs API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'jobs Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Jobs
+
+A `Job` binds a `Graph` and the resources it needs to be executable.
+
+Jobs are created by calling `GraphDefinition.to_job()` on a graph instance, or using the `job` decorator.
+
+
+
@dagster.job
+
+
+ Creates a job with the specified parameters from the decorated graph/op invocation function.
+
+ Using this decorator allows you to build an executable job by writing a function that invokes
+ ops (or graphs).
+
+ Parameters:
+ - (Callable[... (compose_fn) – The decorated function. The body should contain op or graph invocations. Unlike op
+ - Any] – The decorated function. The body should contain op or graph invocations. Unlike op
+ - name (Optional[str]) – The name for the Job. Defaults to the name of the this graph.
+ - resource_defs (Optional[Mapping[str, object]]) – Resources that are required by this graph for execution.
+ - config –
+
+ Describes how the job is parameterized at runtime.
+
+ If no value is provided, then the schema for the job’s run config is a standard
+ format based on its ops and resources.
+
+ If a dictionary is provided, then it must conform to the standard config schema, and
+ it will be used as the job’s run config for the job whenever the job is executed.
+ The values provided will be viewable and editable in the Dagster UI, so be
+ careful with secrets.
+
+ If a [`RunConfig`](config.mdx#dagster.RunConfig) object is provided, then it will be used directly as the run config
+ for the job whenever the job is executed, similar to providing a dictionary.
+
+ If a [`ConfigMapping`](config.mdx#dagster.ConfigMapping) object is provided, then the schema for the job’s run config is
+ determined by the config mapping, and the ConfigMapping, which should return
+ configuration in the standard format to configure the job.
+
+ - tags (Optional[Mapping[str, object]]) – A set of key-value tags that annotate the job and can
+ - run_tags (Optional[Mapping[str, object]]) – A set of key-value tags that will be automatically attached to runs launched by this
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary information that will be attached to the JobDefinition and be viewable in the Dagster UI.
+ - logger_defs (Optional[Dict[str, [*LoggerDefinition*](loggers.mdx#dagster.LoggerDefinition)]]) – A dictionary of string logger identifiers to their implementations.
+ - executor_def (Optional[[*ExecutorDefinition*](internals.mdx#dagster.ExecutorDefinition)]) – How this Job will be executed. Defaults to [`multiprocess_executor`](execution.mdx#dagster.multiprocess_executor) .
+ - op_retry_policy (Optional[[*RetryPolicy*](ops.mdx#dagster.RetryPolicy)]) – The default retry policy for all ops in this job.
+ - partitions_def (Optional[[*PartitionsDefinition*](partitions.mdx#dagster.PartitionsDefinition)]) – Defines a discrete set of partition keys
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of a job.
+
+
+ Examples:
+
+ ```python
+ @op
+ def return_one():
+ return 1
+
+ @op
+ def add_one(in1):
+ return in1 + 1
+
+ @job
+ def job1():
+ add_one(return_one())
+ ```
+
+
+
+
+
+
class dagster.JobDefinition
+
+
+ Defines a Dagster job.
+
+
+
execute_in_process
+
+
+ Execute the Job in-process, gathering results in-memory.
+
+ The executor_def on the Job will be ignored, and replaced with the in-process executor.
+ If using the default io_manager, it will switch from filesystem to in-memory.
+
+ Parameters:
+ - (Optional[Mapping[str (run_config) – The configuration for the run
+ - Any]] – The configuration for the run
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The instance to execute against, an ephemeral one will be used if none provided.
+ - partition_key – (Optional[str])
+ - raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur.
+ - op_selection (Optional[Sequence[str]]) – A list of op selection queries (including single op
+ - input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of the job. Input
+ - resources (Optional[Mapping[str, Any]]) – The resources needed if any are required. Can provide resource instances directly,
+
+
+ Returns: [`ExecuteInProcessResult`](execution.mdx#dagster.ExecuteInProcessResult)
+
+
+
+
+
+
run_request_for_partition
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.0.
+ Directly instantiate `RunRequest(partition_key=...)` instead..
+
+ :::
+
+ Creates a RunRequest object for a run that processes the given partition.
+
+ Parameters:
+ - partition_key – The key of the partition to request a run for.
+ - run_key (Optional[str]) – A string key to identify this launched run. For sensors, ensures that
+ - tags (Optional[Dict[str, str]]) – A dictionary of tags (string key-value pairs) to attach
+ - (Optional[Mapping[str (run_config) – Configuration for the run. If the job has
+ - Any]] – Configuration for the run. If the job has
+ - current_time (Optional[datetime]) – Used to determine which time-partitions exist.
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: an object that requests a run to process the given partition.Return type: [RunRequest](schedules-sensors.mdx#dagster.RunRequest)
+
+
+
+
+
+
with_hooks
+
+ Apply a set of hooks to all op instances within the job.
+
+
+
+
+
with_top_level_resources
+
+ Apply a set of resources to all op instances within the job.
+
+
+
+
+
property config_mapping
+
+
+ The config mapping for the job, if it has one.
+
+ A config mapping defines a way to map a top-level config schema to run config for the job.
+
+
+
+
+
+
+
property executor_def
+
+
+ Returns the default [`ExecutorDefinition`](internals.mdx#dagster.ExecutorDefinition) for the job.
+
+ If the user has not specified an executor definition, then this will default to the
+ [`multi_or_in_process_executor()`](execution.mdx#dagster.multi_or_in_process_executor). If a default is specified on the
+ [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to, then that will be used instead.
+
+
+
+
+
+
+
property has_specified_executor
+
+ Returns True if this job has explicitly specified an executor, and False if the executor
+ was inherited through defaults or the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to.
+
+
+
+
+
property has_specified_loggers
+
+ Returns true if the job explicitly set loggers, and False if loggers were inherited
+ through defaults or the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to.
+
+
+
+
+
property loggers
+
+
+ Returns the set of LoggerDefinition objects specified on the job.
+
+ If the user has not specified a mapping of [`LoggerDefinition`](loggers.mdx#dagster.LoggerDefinition) objects, then this
+ will default to the `colored_console_logger()` under the key console. If a default
+ is specified on the [`Definitions`](definitions.mdx#dagster.Definitions) object the job was provided to, then that will
+ be used instead.
+
+
+
+
+
+
+
property partitioned_config
+
+
+ The partitioned config for the job, if it has one.
+
+ A partitioned config defines a way to map partition keys to run config for the job.
+
+
+
+
+
+
+
property partitions_def
+
+
+ Returns the [`PartitionsDefinition`](partitions.mdx#dagster.PartitionsDefinition) for the job, if it has one.
+
+ A partitions definition defines the set of partition keys the job operates on.
+
+
+
+
+
+
+
property resource_defs
+
+
+ Returns the set of ResourceDefinition objects specified on the job.
+
+ This may not be the complete set of resources required by the job, since those can also be
+ provided on the [`Definitions`](definitions.mdx#dagster.Definitions) object the job may be provided to.
+
+
+
+
+
+
+
+
+
+
+
+## Reconstructable jobs
+
+
+
class dagster.reconstructable
+
+
+ Create a `ReconstructableJob` from a
+ function that returns a [`JobDefinition`](#dagster.JobDefinition)/[`JobDefinition`](#dagster.JobDefinition),
+ or a function decorated with [`@job`](#dagster.job).
+
+ When your job must cross process boundaries, e.g., for execution on multiple nodes or
+ in different systems (like `dagstermill`), Dagster must know how to reconstruct the job
+ on the other side of the process boundary.
+
+ Passing a job created with `~dagster.GraphDefinition.to_job` to `reconstructable()`,
+ requires you to wrap that job’s definition in a module-scoped function, and pass that function
+ instead:
+
+ ```python
+ from dagster import graph, reconstructable
+
+ @graph
+ def my_graph():
+ ...
+
+ def define_my_job():
+ return my_graph.to_job()
+
+ reconstructable(define_my_job)
+ ```
+ This function implements a very conservative strategy for reconstruction, so that its behavior
+ is easy to predict, but as a consequence it is not able to reconstruct certain kinds of jobs
+ or jobs, such as those defined by lambdas, in nested scopes (e.g., dynamically within a method
+ call), or in interactive environments such as the Python REPL or Jupyter notebooks.
+
+ If you need to reconstruct objects constructed in these ways, you should use
+ `build_reconstructable_job()` instead, which allows you to
+ specify your own reconstruction strategy.
+
+ Examples:
+
+ ```python
+ from dagster import job, reconstructable
+
+ @job
+ def foo_job():
+ ...
+
+ reconstructable_foo_job = reconstructable(foo_job)
+
+
+ @graph
+ def foo():
+ ...
+
+ def make_bar_job():
+ return foo.to_job()
+
+ reconstructable_bar_job = reconstructable(make_bar_job)
+ ```
+
+
+
+
+
+
dagster.build_reconstructable_job
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Create a `dagster._core.definitions.reconstructable.ReconstructableJob`.
+
+ When your job must cross process boundaries, e.g., for execution on multiple nodes or in
+ different systems (like `dagstermill`), Dagster must know how to reconstruct the job
+ on the other side of the process boundary.
+
+ This function allows you to use the strategy of your choice for reconstructing jobs, so
+ that you can reconstruct certain kinds of jobs that are not supported by
+ [`reconstructable()`](execution.mdx#dagster.reconstructable), such as those defined by lambdas, in nested scopes (e.g.,
+ dynamically within a method call), or in interactive environments such as the Python REPL or
+ Jupyter notebooks.
+
+ If you need to reconstruct jobs constructed in these ways, use this function instead of
+ [`reconstructable()`](execution.mdx#dagster.reconstructable).
+
+ Parameters:
+ - reconstructor_module_name (str) – The name of the module containing the function to use to
+ - reconstructor_function_name (str) – The name of the function to use to reconstruct the
+ - reconstructable_args (Tuple) – Args to the function to use to reconstruct the job.
+ - reconstructable_kwargs (Dict[str, Any]) – Kwargs to the function to use to reconstruct the
+
+
+ Examples:
+
+ ```python
+ # module: mymodule
+
+ from dagster import JobDefinition, job, build_reconstructable_job
+
+ class JobFactory:
+ def make_job(*args, **kwargs):
+
+ @job
+ def _job(...):
+ ...
+
+ return _job
+
+ def reconstruct_job(*args):
+ factory = JobFactory()
+ return factory.make_job(*args)
+
+ factory = JobFactory()
+
+ foo_job_args = (...,...)
+
+ foo_job_kwargs = {...:...}
+
+ foo_job = factory.make_job(*foo_job_args, **foo_job_kwargs)
+
+ reconstructable_foo_job = build_reconstructable_job(
+ 'mymodule',
+ 'reconstruct_job',
+ foo_job_args,
+ foo_job_kwargs,
+ )
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-airbyte.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-airbyte.mdx
new file mode 100644
index 0000000000000..f03dcce678eb4
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-airbyte.mdx
@@ -0,0 +1,462 @@
+---
+title: 'airbyte (dagster-airbyte)'
+title_meta: 'airbyte (dagster-airbyte) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'airbyte (dagster-airbyte) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Airbyte (dagster-airbyte)
+
+This library provides a Dagster integration with [Airbyte](https://www.airbyte.com/).
+
+For more information on getting started, see the [Airbyte integration guide](https://docs.dagster.io/integrations/libraries/airbyte/).
+
+
+
+
+## Assets (Airbyte API)
+
+
+
class dagster_airbyte.AirbyteCloudWorkspace
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This class represents a Airbyte Cloud workspace and provides utilities
+ to interact with Airbyte APIs.
+
+
+
sync_and_poll
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Executes a sync and poll process to materialize Airbyte Cloud assets.
+ This method can only be used in the context of an asset execution.
+
+ Parameters: context ([*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)) – The execution context
+ from within @airbyte_assets.Returns:
+ An iterator of MaterializeResult
+ or AssetMaterialization.
+
+ Return type: Iterator[Union[[AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [MaterializeResult](../assets.mdx#dagster.MaterializeResult)]]
+
+
+
+
+
+
+
+
+
+
class dagster_airbyte.DagsterAirbyteTranslator
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Translator class which converts a AirbyteConnectionTableProps object into AssetSpecs.
+ Subclass this class to implement custom logic how to translate Airbyte content into asset spec.
+
+
+
+
+
+
+
dagster_airbyte.load_airbyte_cloud_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Airbyte content in the workspace.
+
+ Parameters:
+ - workspace ([*AirbyteCloudWorkspace*](#dagster_airbyte.AirbyteCloudWorkspace)) – The Airbyte Cloud workspace to fetch assets from.
+ - dagster_airbyte_translator (Optional[[*DagsterAirbyteTranslator*](#dagster_airbyte.DagsterAirbyteTranslator)], optional) – The translator to use
+
+
+ Returns: The set of assets representing the Airbyte content in the workspace.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+ Examples:
+
+ Loading the asset specs for a given Airbyte Cloud workspace:
+
+ ```python
+ from dagster_airbyte import AirbyteCloudWorkspace, load_airbyte_cloud_asset_specs
+
+ import dagster as dg
+
+ airbyte_cloud_workspace = AirbyteCloudWorkspace(
+ workspace_id=dg.EnvVar("AIRBYTE_CLOUD_WORKSPACE_ID"),
+ client_id=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_ID"),
+ client_secret=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_SECRET"),
+ )
+
+
+ airbyte_cloud_specs = load_airbyte_cloud_asset_specs(airbyte_cloud_workspace)
+ defs = dg.Definitions(assets=airbyte_cloud_specs)
+ ```
+
+
+
+
+
+
@dagster_airbyte.airbyte_assets
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Create a definition for how to sync the tables of a given Airbyte connection.
+
+ Parameters:
+ - connection_id (str) – The Airbyte Connection ID.
+ - workspace ([*AirbyteCloudWorkspace*](#dagster_airbyte.AirbyteCloudWorkspace)) – The Airbyte workspace to fetch assets from.
+ - name (Optional[str], optional) – The name of the op.
+ - group_name (Optional[str], optional) – The name of the asset group.
+ - dagster_airbyte_translator (Optional[[*DagsterAirbyteTranslator*](#dagster_airbyte.DagsterAirbyteTranslator)], optional) – The translator to use
+
+
+ Examples:
+
+ Sync the tables of an Airbyte connection:
+
+ ```python
+ from dagster_airbyte import AirbyteCloudWorkspace, airbyte_assets
+
+ import dagster as dg
+
+ airbyte_workspace = AirbyteCloudWorkspace(
+ workspace_id=dg.EnvVar("AIRBYTE_CLOUD_WORKSPACE_ID"),
+ client_id=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_ID"),
+ client_secret=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_SECRET"),
+ )
+
+
+ @airbyte_assets(
+ connection_id="airbyte_connection_id",
+ workspace=airbyte_workspace,
+ )
+ def airbyte_connection_assets(context: dg.AssetExecutionContext, airbyte: AirbyteCloudWorkspace):
+ yield from airbyte.sync_and_poll(context=context)
+
+
+ defs = dg.Definitions(
+ assets=[airbyte_connection_assets],
+ resources={"airbyte": airbyte_workspace},
+ )
+ ```
+ Sync the tables of an Airbyte connection with a custom translator:
+
+ ```python
+ from dagster_airbyte import (
+ DagsterAirbyteTranslator,
+ AirbyteConnectionTableProps,
+ AirbyteCloudWorkspace,
+ airbyte_assets
+ )
+
+ import dagster as dg
+
+ class CustomDagsterAirbyteTranslator(DagsterAirbyteTranslator):
+ def get_asset_spec(self, props: AirbyteConnectionTableProps) -> dg.AssetSpec:
+ default_spec = super().get_asset_spec(props)
+ return default_spec.merge_attributes(
+ metadata={"custom": "metadata"},
+ )
+
+ airbyte_workspace = AirbyteCloudWorkspace(
+ workspace_id=dg.EnvVar("AIRBYTE_CLOUD_WORKSPACE_ID"),
+ client_id=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_ID"),
+ client_secret=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_SECRET"),
+ )
+
+
+ @airbyte_assets(
+ connection_id="airbyte_connection_id",
+ workspace=airbyte_workspace,
+ dagster_airbyte_translator=CustomDagsterAirbyteTranslator()
+ )
+ def airbyte_connection_assets(context: dg.AssetExecutionContext, airbyte: AirbyteCloudWorkspace):
+ yield from airbyte.sync_and_poll(context=context)
+
+
+ defs = dg.Definitions(
+ assets=[airbyte_connection_assets],
+ resources={"airbyte": airbyte_workspace},
+ )
+ ```
+
+
+
+
+
+
dagster_airbyte.build_airbyte_assets_definitions
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ The list of AssetsDefinition for all connections in the Airbyte workspace.
+
+ Parameters:
+ - workspace ([*AirbyteCloudWorkspace*](#dagster_airbyte.AirbyteCloudWorkspace)) – The Airbyte workspace to fetch assets from.
+ - dagster_airbyte_translator (Optional[[*DagsterAirbyteTranslator*](#dagster_airbyte.DagsterAirbyteTranslator)], optional) – The translator to use
+
+
+ Returns: The list of AssetsDefinition for all connections in the Airbyte workspace.Return type: List[[AssetsDefinition](../assets.mdx#dagster.AssetsDefinition)]
+ Examples:
+
+ Sync the tables of a Airbyte connection:
+
+ ```python
+ from dagster_airbyte import AirbyteCloudWorkspace, build_airbyte_assets_definitions
+
+ import dagster as dg
+
+ airbyte_workspace = AirbyteCloudWorkspace(
+ workspace_id=dg.EnvVar("AIRBYTE_CLOUD_WORKSPACE_ID"),
+ client_id=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_ID"),
+ client_secret=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_SECRET"),
+ )
+
+
+ airbyte_assets = build_airbyte_assets_definitions(workspace=workspace)
+
+ defs = dg.Definitions(
+ assets=airbyte_assets,
+ resources={"airbyte": airbyte_workspace},
+ )
+ ```
+ Sync the tables of a Airbyte connection with a custom translator:
+
+ ```python
+ from dagster_airbyte import (
+ DagsterAirbyteTranslator,
+ AirbyteConnectionTableProps,
+ AirbyteCloudWorkspace,
+ build_airbyte_assets_definitions
+ )
+
+ import dagster as dg
+
+ class CustomDagsterAirbyteTranslator(DagsterAirbyteTranslator):
+ def get_asset_spec(self, props: AirbyteConnectionTableProps) -> dg.AssetSpec:
+ default_spec = super().get_asset_spec(props)
+ return default_spec.merge_attributes(
+ metadata={"custom": "metadata"},
+ )
+
+ airbyte_workspace = AirbyteCloudWorkspace(
+ workspace_id=dg.EnvVar("AIRBYTE_CLOUD_WORKSPACE_ID"),
+ client_id=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_ID"),
+ client_secret=dg.EnvVar("AIRBYTE_CLOUD_CLIENT_SECRET"),
+ )
+
+
+ airbyte_assets = build_airbyte_assets_definitions(
+ workspace=workspace,
+ dagster_airbyte_translator=CustomDagsterAirbyteTranslator()
+ )
+
+ defs = dg.Definitions(
+ assets=airbyte_assets,
+ resources={"airbyte": airbyte_workspace},
+ )
+ ```
+
+
+
+ This resource allows users to programatically interface with the Airbyte REST API to launch
+ syncs and monitor their progress.
+
+ Examples:
+
+ ```python
+ from dagster import job, EnvVar
+ from dagster_airbyte import AirbyteResource
+
+ my_airbyte_resource = AirbyteResource(
+ host=EnvVar("AIRBYTE_HOST"),
+ port=EnvVar("AIRBYTE_PORT"),
+ # If using basic auth
+ username=EnvVar("AIRBYTE_USERNAME"),
+ password=EnvVar("AIRBYTE_PASSWORD"),
+ )
+
+ airbyte_assets = build_airbyte_assets(
+ connection_id="87b7fe85-a22c-420e-8d74-b30e7ede77df",
+ destination_tables=["releases", "tags", "teams"],
+ )
+
+ defs = Definitions(
+ assets=[airbyte_assets],
+ resources={"airbyte": my_airbyte_resource},
+ )
+ ```
+
+
+
+
+
+
dagster_airbyte.load_assets_from_airbyte_instance
+
+
+ Loads Airbyte connection assets from a configured AirbyteResource instance. This fetches information
+ about defined connections at initialization time, and will error on workspace load if the Airbyte
+ instance is not reachable.
+
+ Parameters:
+ - airbyte ([*ResourceDefinition*](../resources.mdx#dagster.ResourceDefinition)) – An AirbyteResource configured with the appropriate connection
+ - workspace_id (Optional[str]) – The ID of the Airbyte workspace to load connections from. Only
+ - key_prefix (Optional[CoercibleToAssetKeyPrefix]) – A prefix for the asset keys created.
+ - create_assets_for_normalization_tables (bool) – If True, assets will be created for tables
+ - connection_to_group_fn (Optional[Callable[[str], Optional[str]]]) – Function which returns an asset
+ - connection_meta_to_group_fn (Optional[Callable[[AirbyteConnectionMetadata], Optional[str]]]) – Function which
+ - io_manager_key (Optional[str]) – The I/O manager key to use for all assets. Defaults to “io_manager”.
+ - connection_to_io_manager_key_fn (Optional[Callable[[str], Optional[str]]]) – Function which returns an
+ - connection_filter (Optional[Callable[[AirbyteConnectionMetadata], bool]]) – Optional function which takes
+ - connection_to_asset_key_fn (Optional[Callable[[AirbyteConnectionMetadata, str], [*AssetKey*](../assets.mdx#dagster.AssetKey)]]) – Optional function which
+ - connection_to_freshness_policy_fn (Optional[Callable[[AirbyteConnectionMetadata], Optional[FreshnessPolicy]]]) – Optional function
+ - connection_to_auto_materialize_policy_fn (Optional[Callable[[AirbyteConnectionMetadata], Optional[AutoMaterializePolicy]]]) – Optional
+
+
+ Examples:
+
+ Loading all Airbyte connections as assets:
+
+ ```python
+ from dagster_airbyte import airbyte_resource, load_assets_from_airbyte_instance
+
+ airbyte_instance = airbyte_resource.configured(
+ {
+ "host": "localhost",
+ "port": "8000",
+ }
+ )
+ airbyte_assets = load_assets_from_airbyte_instance(airbyte_instance)
+ ```
+ Filtering the set of loaded connections:
+
+ ```python
+ from dagster_airbyte import airbyte_resource, load_assets_from_airbyte_instance
+
+ airbyte_instance = airbyte_resource.configured(
+ {
+ "host": "localhost",
+ "port": "8000",
+ }
+ )
+ airbyte_assets = load_assets_from_airbyte_instance(
+ airbyte_instance,
+ connection_filter=lambda meta: "snowflake" in meta.name,
+ )
+ ```
+
+
+
+
+
+
dagster_airbyte.build_airbyte_assets
+
+
+ Builds a set of assets representing the tables created by an Airbyte sync operation.
+
+ Parameters:
+ - connection_id (str) – The Airbyte Connection ID that this op will sync. You can retrieve this
+ - destination_tables (List[str]) – The names of the tables that you want to be represented
+ - destination_database (Optional[str]) – The name of the destination database.
+ - destination_schema (Optional[str]) – The name of the destination schema.
+ - normalization_tables (Optional[Mapping[str, List[str]]]) – If you are using Airbyte’s
+ - asset_key_prefix (Optional[List[str]]) – A prefix for the asset keys inside this asset.
+ - deps (Optional[Sequence[Union[[*AssetsDefinition*](../assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](../assets.mdx#dagster.SourceAsset), str, [*AssetKey*](../assets.mdx#dagster.AssetKey)]]]) – A list of assets to add as sources.
+ - upstream_assets (Optional[Set[[*AssetKey*](../assets.mdx#dagster.AssetKey)]]) – Deprecated, use deps instead. A list of assets to add as sources.
+ - freshness_policy (Optional[FreshnessPolicy]) – A freshness policy to apply to the assets
+ - stream_to_asset_map (Optional[Mapping[str, str]]) – A mapping of an Airbyte stream name to a Dagster asset.
+ - auto_materialize_policy (Optional[AutoMaterializePolicy]) – An auto materialization policy to apply to the assets.
+
+
+
+
+
+
+
+
dagster_airbyte.airbyte_sync_op `=` \
+
+
+
+
+ Executes a Airbyte job sync for a given `connection_id`, and polls until that sync
+ completes, raising an error if it is unsuccessful. It outputs a AirbyteOutput which contains
+ the job details for a given `connection_id`.
+
+ It requires the use of the `airbyte_resource`, which allows it to
+ communicate with the Airbyte API.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_airbyte import airbyte_resource, airbyte_sync_op
+
+ my_airbyte_resource = airbyte_resource.configured(
+ {
+ "host": {"env": "AIRBYTE_HOST"},
+ "port": {"env": "AIRBYTE_PORT"},
+ }
+ )
+
+ sync_foobar = airbyte_sync_op.configured({"connection_id": "foobar"}, name="sync_foobar")
+
+ @job(resource_defs={"airbyte": my_airbyte_resource})
+ def my_simple_airbyte_job():
+ sync_foobar()
+
+ @job(resource_defs={"airbyte": my_airbyte_resource})
+ def my_composed_airbyte_job():
+ final_foobar_state = sync_foobar(start_after=some_op())
+ other_op(final_foobar_state)
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-airlift.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-airlift.mdx
new file mode 100644
index 0000000000000..2f6c9de25ea00
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-airlift.mdx
@@ -0,0 +1,920 @@
+---
+title: 'airlift (dagster-airlift)'
+title_meta: 'airlift (dagster-airlift) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'airlift (dagster-airlift) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Airlift (dagster-airlift)
+
+
+
+
+## Core (dagster_airlift.core)
+
+
+
+
+### AirflowInstance
+
+
+
class dagster_airlift.core.AirflowInstance
+
+
+ A class that represents a running Airflow Instance and provides methods for interacting with its REST API.
+
+ Parameters:
+ - auth_backend ([*AirflowAuthBackend*](#dagster_airlift.core.AirflowAuthBackend)) – The authentication backend to use when making requests to the Airflow instance.
+ - name (str) – The name of the Airflow instance. This will be prefixed to any assets automatically created using this instance.
+ - batch_task_instance_limit (int) – The number of task instances to query at a time when fetching task instances. Defaults to 100.
+ - batch_dag_runs_limit (int) – The number of dag runs to query at a time when fetching dag runs. Defaults to 100.
+
+
+
+
get_run_state
+
+
+ Given a run ID of an airflow dag, return the state of that run.
+
+ Parameters:
+ - dag_id (str) – The dag id.
+ - run_id (str) – The run id.
+
+
+ Returns: The state of the run. Will be one of the states defined by Airflow.Return type: str
+
+
+
+
+
+
trigger_dag
+
+
+ Trigger a dag run for the given dag_id.
+
+ Does not wait for the run to finish. To wait for the completed run to finish, use [`wait_for_run_completion()`](#dagster_airlift.core.AirflowInstance.wait_for_run_completion).
+
+ Parameters:
+ - dag_id (str) – The dag id to trigger.
+ - logical_date (Optional[datetime.datetime]) – The Airflow logical_date to use for the dag run. If not provided, the current time will be used. Previously known as execution_date in Airflow; find more information in the Airflow docs: [https://airflow.apache.org/docs/apache-airflow/stable/faq.html#what-does-execution-date-mean](https://airflow.apache.org/docs/apache-airflow/stable/faq.html#what-does-execution-date-mean)
+
+
+ Returns: The dag run id.Return type: str
+
+
+
+
+
+
wait_for_run_completion
+
+
+ Given a run ID of an airflow dag, wait for that run to reach a completed state.
+
+ Parameters:
+ - dag_id (str) – The dag id.
+ - run_id (str) – The run id.
+ - timeout (int) – The number of seconds to wait before timing out.
+
+
+ Returns: None
+
+
+
+
+
+
+
+
+
+
class dagster_airlift.core.AirflowAuthBackend
+
+
+ An abstract class that represents an authentication backend for an Airflow instance.
+
+ Requires two methods to be implemented by subclasses:
+ - get_session: Returns a requests.Session object that can be used to make requests to the Airflow instance, and handles authentication.
+ - get_webserver_url: Returns the base URL of the Airflow webserver.
+
+ The dagster-airlift package provides the following default implementations:
+ - `dagster-airlift.core.AirflowBasicAuthBackend`: An authentication backend that uses Airflow’s basic auth to authenticate with the Airflow instance.
+ - `dagster-airlift.mwaa.MwaaSessionAuthBackend`: An authentication backend that uses AWS MWAA’s web login token to authenticate with the Airflow instance (requires dagster-airlift[mwaa]).
+
+
+
+
+
+
+
class dagster_airlift.core.AirflowBasicAuthBackend
+
+
+ A [`dagster_airlift.core.AirflowAuthBackend`](#dagster_airlift.core.AirflowAuthBackend) that authenticates using basic auth.
+
+ Parameters:
+ - webserver_url (str) – The URL of the webserver.
+ - username (str) – The username to authenticate with.
+ - password (str) – The password to authenticate with.
+
+
+ Examples:
+
+ Creating a [`AirflowInstance`](#dagster_airlift.core.AirflowInstance) using this backend.
+
+ ```python
+ from dagster_airlift.core import AirflowInstance, AirflowBasicAuthBackend
+
+ af_instance = AirflowInstance(
+ name="my-instance",
+ auth_backend=AirflowBasicAuthBackend(
+ webserver_url="https://my-webserver-hostname",
+ username="my-username",
+ password="my-password"
+ )
+ )
+ ```
+
+
+
+ Builds a [`dagster.Definitions`](../definitions.mdx#dagster.Definitions) object from an Airflow instance.
+
+ For every DAG in the Airflow instance, this function will create a Dagster asset for the DAG
+ with an asset key instance_name/dag/dag_id. It will also create a sensor that polls the Airflow
+ instance for DAG runs and emits Dagster events for each successful run.
+
+ An optional defs argument can be provided, where the user can pass in a [`dagster.Definitions`](../definitions.mdx#dagster.Definitions)
+ object containing assets which are mapped to Airflow DAGs and tasks. These assets will be enriched with
+ metadata from the Airflow instance, and placed upstream of the automatically generated DAG assets.
+
+ An optional event_transformer_fn can be provided, which allows the user to modify the Dagster events
+ produced by the sensor. The function takes the Dagster events produced by the sensor and returns a sequence
+ of Dagster events.
+
+ An optional dag_selector_fn can be provided, which allows the user to filter which DAGs assets are created for.
+ The function takes a [`dagster_airlift.core.serialization.serialized_data.DagInfo`](#dagster_airlift.core.DagInfo) object and returns a
+ boolean indicating whether the DAG should be included.
+
+ Parameters:
+ - airflow_instance ([*AirflowInstance*](#dagster_airlift.core.AirflowInstance)) – The Airflow instance to build assets and the sensor from.
+ - defs – Optional[Definitions]: A [`dagster.Definitions`](../definitions.mdx#dagster.Definitions) object containing assets that are
+ - sensor_minimum_interval_seconds (int) – The minimum interval in seconds between sensor runs.
+ - event_transformer_fn (DagsterEventTransformerFn) – A function that allows for modifying the Dagster events
+ - dag_selector_fn (Optional[DagSelectorFn]) – A function that allows for filtering which DAGs assets are created for.
+
+
+ Returns: A [`dagster.Definitions`](../definitions.mdx#dagster.Definitions) object containing the assets and sensor.Return type: [Definitions](../definitions.mdx#dagster.Definitions)
+ Examples:
+
+ Building a [`dagster.Definitions`](../definitions.mdx#dagster.Definitions) object from an Airflow instance.
+
+ ```python
+ from dagster_airlift.core import (
+ AirflowInstance,
+ AirflowBasicAuthBackend,
+ build_defs_from_airflow_instance,
+ )
+
+ from .constants import AIRFLOW_BASE_URL, AIRFLOW_INSTANCE_NAME, PASSWORD, USERNAME
+
+ airflow_instance = AirflowInstance(
+ auth_backend=AirflowBasicAuthBackend(
+ webserver_url=AIRFLOW_BASE_URL, username=USERNAME, password=PASSWORD
+ ),
+ name=AIRFLOW_INSTANCE_NAME,
+ )
+
+
+ defs = build_defs_from_airflow_instance(airflow_instance=airflow_instance)
+ ```
+ Providing task-mapped assets to the function.
+
+ ```python
+ from dagster import Definitions
+ from dagster_airlift.core import (
+ AirflowInstance,
+ AirflowBasicAuthBackend,
+ assets_with_task_mappings,
+ build_defs_from_airflow_instance,
+ )
+ ...
+
+
+ defs = build_defs_from_airflow_instance(
+ airflow_instance=airflow_instance, # same as above
+ defs=Definitions(
+ assets=assets_with_task_mappings(
+ dag_id="rebuild_iris_models",
+ task_mappings={
+ "my_task": [AssetSpec("my_first_asset"), AssetSpec("my_second_asset")],
+ },
+ ),
+ ),
+ )
+ ```
+ Providing a custom event transformer function.
+
+ ```python
+ from typing import Sequence
+ from dagster import Definitions, SensorEvaluationContext
+ from dagster_airlift.core import (
+ AirflowInstance,
+ AirflowBasicAuthBackend,
+ AssetEvent,
+ assets_with_task_mappings,
+ build_defs_from_airflow_instance,
+ AirflowDefinitionsData,
+ )
+ ...
+
+ def add_tags_to_events(
+ context: SensorEvaluationContext,
+ defs_data: AirflowDefinitionsData,
+ events: Sequence[AssetEvent]
+ ) -> Sequence[AssetEvent]:
+ altered_events = []
+ for event in events:
+ altered_events.append(event._replace(tags={"my_tag": "my_value"}))
+ return altered_events
+
+ defs = build_defs_from_airflow_instance(
+ airflow_instance=airflow_instance, # same as above
+ event_transformer_fn=add_tags_to_events,
+ )
+ ```
+ Filtering which DAGs assets are created for.
+
+ ```python
+ from dagster import Definitions
+ from dagster_airlift.core import (
+ AirflowInstance,
+ AirflowBasicAuthBackend,
+ AssetEvent,
+ assets_with_task_mappings,
+ build_defs_from_airflow_instance,
+ DagInfo,
+ )
+ ...
+
+ def only_include_dag(dag_info: DagInfo) -> bool:
+ return dag_info.dag_id == "my_dag_id"
+
+ defs = build_defs_from_airflow_instance(
+ airflow_instance=airflow_instance, # same as above
+ dag_selector_fn=only_include_dag,
+ )
+ ```
+
+
+
+ Modify assets to be associated with a particular task in Airlift tooling.
+
+ Used in concert with build_defs_from_airflow_instance to observe an airflow
+ instance to monitor the tasks that are associated with the assets and
+ keep their materialization histories up to date.
+
+ Concretely this adds metadata to all asset specs in the provided definitions
+ with the provided dag_id and task_id. The dag_id comes from the dag_id argument;
+ the task_id comes from the key of the provided task_mappings dictionary.
+ There is a single metadata key “airlift/task-mapping” that is used to store
+ this information. It is a list of dictionaries with keys “dag_id” and “task_id”.
+
+ Example:
+
+ ```python
+ from dagster import AssetSpec, Definitions, asset
+ from dagster_airlift.core import assets_with_task_mappings
+
+ @asset
+ def asset_one() -> None: ...
+
+ defs = Definitions(
+ assets=assets_with_task_mappings(
+ dag_id="dag_one",
+ task_mappings={
+ "task_one": [asset_one],
+ "task_two": [AssetSpec(key="asset_two"), AssetSpec(key="asset_three")],
+ },
+ )
+ )
+ ```
+
+
+
+
+
+
dagster_airlift.core.assets_with_dag_mappings
+
+
+ Modify assets to be associated with a particular dag in Airlift tooling.
+
+ Used in concert with build_defs_from_airflow_instance to observe an airflow
+ instance to monitor the dags that are associated with the assets and
+ keep their materialization histories up to date.
+
+ In contrast with assets_with_task_mappings, which maps assets on a per-task basis, this is used in concert with
+ proxying_to_dagster dag-level mappings where an entire dag is migrated at once.
+
+ Concretely this adds metadata to all asset specs in the provided definitions
+ with the provided dag_id. The dag_id comes from the key of the provided dag_mappings dictionary.
+ There is a single metadata key “airlift/dag-mapping” that is used to store
+ this information. It is a list of strings, where each string is a dag_id which the asset is associated with.
+
+ Example:
+
+ ```python
+ from dagster import AssetSpec, Definitions, asset
+ from dagster_airlift.core import assets_with_dag_mappings
+
+ @asset
+ def asset_one() -> None: ...
+
+ defs = Definitions(
+ assets=assets_with_dag_mappings(
+ dag_mappings={
+ "dag_one": [asset_one],
+ "dag_two": [AssetSpec(key="asset_two"), AssetSpec(key="asset_three")],
+ },
+ )
+ )
+ ```
+
+
+
+ Given an asset or assets definition, return a new asset or assets definition with metadata
+ that indicates that it is targeted by multiple airflow tasks. An example of this would
+ be a separate weekly and daily dag that contains a task that targets a single asset.
+
+ ```python
+ from dagster import Definitions, AssetSpec, asset
+ from dagster_airlift import (
+ build_defs_from_airflow_instance,
+ targeted_by_multiple_tasks,
+ assets_with_task_mappings,
+ )
+
+ # Asset maps to a single task.
+ @asset
+ def other_asset(): ...
+
+ # Asset maps to a physical entity which is produced by two different airflow tasks.
+ @asset
+ def scheduled_twice(): ...
+
+ defs = build_defs_from_airflow_instance(
+ airflow_instance=airflow_instance,
+ defs=Definitions(
+ assets=[
+ *assets_with_task_mappings(
+ dag_id="other_dag",
+ task_mappings={
+ "task1": [other_asset]
+ },
+ ),
+ *assets_with_multiple_task_mappings(
+ assets=[scheduled_twice],
+ task_handles=[
+ {"dag_id": "weekly_dag", "task_id": "task1"},
+ {"dag_id": "daily_dag", "task_id": "task1"},
+ ],
+ ),
+ ]
+ ),
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+#### Annotations for customizable components:
+
+
+
dagster_airlift.core.DagSelectorFn
+
+ alias of `Callable`[[[`DagInfo`](#dagster_airlift.core.DagInfo)], `bool`]
+
+
+
+
+
dagster_airlift.core.DagsterEventTransformerFn
+
+ alias of `Callable`[[`SensorEvaluationContext`, [`AirflowDefinitionsData`](#dagster_airlift.core.AirflowDefinitionsData), `Sequence`[[`AssetMaterialization`](../ops.mdx#dagster.AssetMaterialization)]], `Iterable`[[`AssetMaterialization`](../ops.mdx#dagster.AssetMaterialization) | [`AssetObservation`](../assets.mdx#dagster.AssetObservation) | `AssetCheckEvaluation`]]
+
+
+
+
+
class dagster_airlift.core.TaskHandleDict
+
+
+
+
+
+
+
+
+
+
+
+#### Objects for retrieving information about the Airflow/Dagster mapping:
+
+
+
class dagster_airlift.core.DagInfo
+
+
+ A record containing information about a given airflow dag.
+
+ Users should not instantiate this class directly. It is provided when customizing which DAGs are included
+ in the generated definitions using the dag_selector_fn argument of [`build_defs_from_airflow_instance()`](#dagster_airlift.core.build_defs_from_airflow_instance).
+
+
+
metadata
+
+
+ The metadata associated with the dag, retrieved by the Airflow REST API:
+ [https://airflow.apache.org/docs/apache-airflow/stable/stable-rest-api-ref.html#operation/get_dags](https://airflow.apache.org/docs/apache-airflow/stable/stable-rest-api-ref.html#operation/get_dags)
+
+ Type: Dict[str, Any]
+
+
+
+
+
+
+
+
+
+
class dagster_airlift.core.AirflowDefinitionsData
+
+
+ A class that holds data about the assets that are mapped to Airflow dags and tasks, and
+ provides methods for retrieving information about the mappings.
+ The user should not instantiate this class directly. It is provided when customizing the events
+ that are generated by the Airflow sensor using the event_transformer_fn argument of
+ [`build_defs_from_airflow_instance()`](#dagster_airlift.core.build_defs_from_airflow_instance).
+
+
+
asset_keys_in_task
+
+
+ Returns the asset keys that are mapped to the given task.
+
+ Parameters:
+ - dag_id (str) – The dag id.
+ - task_id (str) – The task id.
+
+
+
+
+
+
+
+
task_ids_in_dag
+
+
+ Returns the task ids within the given dag_id.
+
+ Parameters: dag_id (str) – The dag id.
+
+
+
+
+
+
property instance_name
+
+ The name of the Airflow instance.
+
+
+
+
+
+
+
+
+
+
+
+
+
+## MWAA (dagster_airlift.mwaa)
+
+
+
class dagster_airlift.mwaa.MwaaSessionAuthBackend
+
+
+ A [`dagster_airlift.core.AirflowAuthBackend`](#dagster_airlift.core.AirflowAuthBackend) that authenticates to AWS MWAA.
+
+ Under the hood, this class uses the MWAA boto3 session to request a web login token and then
+ uses the token to authenticate to the MWAA web server.
+
+ Parameters:
+ - mwaa_session (boto3.Session) – The boto3 MWAA session
+ - env_name (str) – The name of the MWAA environment
+
+
+ Examples:
+
+ Creating an AirflowInstance pointed at a MWAA environment.
+
+ ```python
+ import boto3
+ from dagster_airlift.mwaa import MwaaSessionAuthBackend
+ from dagster_airlift.core import AirflowInstance
+
+ boto_client = boto3.client("mwaa")
+ af_instance = AirflowInstance(
+ name="my-mwaa-instance",
+ auth_backend=MwaaSessionAuthBackend(
+ mwaa_client=boto_client,
+ env_name="my-mwaa-env"
+ )
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+## In Airflow (dagster_airlift.in_airflow)
+
+
+
+### Proxying
+
+
+
dagster_airlift.in_airflow.proxying_to_dagster
+
+
+ Proxies tasks and dags to Dagster based on provided proxied state.
+ Expects a dictionary of in-scope global variables to be provided (typically retrieved with globals()), and a proxied state dictionary
+ (typically retrieved with [`load_proxied_state_from_yaml()`](#dagster_airlift.in_airflow.load_proxied_state_from_yaml)) for dags in that global state. This function will modify in-place the
+ dictionary of global variables to replace proxied tasks with appropriate Dagster operators.
+
+ In the case of task-level proxying, the proxied tasks will be replaced with new operators that are constructed by the provided build_from_task_fn.
+ A default implementation of this function is provided in DefaultProxyTaskToDagsterOperator.
+ In the case of dag-level proxying, the entire dag structure will be replaced with a single task that is constructed by the provided build_from_dag_fn.
+ A default implementation of this function is provided in DefaultProxyDAGToDagsterOperator.
+
+ Parameters:
+ - global_vars (Dict[str, Any]) – The global variables in the current context. In most cases, retrieved with globals() (no import required).
+ - proxied_state (AirflowMigrationState) – The proxied state for the dags.
+ - logger (Optional[logging.Logger]) – The logger to use. Defaults to logging.getLogger(“dagster_airlift”).
+
+
+ Examples:
+
+ Typical usage of this function is to be called at the end of a dag file, retrieving proxied_state from an accompanying proxied_state path.
+
+ ```python
+ from pathlib import Path
+
+ from airflow import DAG
+ from airflow.operators.python import PythonOperator
+ from dagster._time import get_current_datetime_midnight
+ from dagster_airlift.in_airflow import proxying_to_dagster
+ from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml
+
+
+ with DAG(
+ dag_id="daily_interval_dag",
+ ...,
+ ) as minute_dag:
+ PythonOperator(task_id="my_task", python_callable=...)
+
+ # At the end of the dag file, so we can ensure dags are loaded into globals.
+ proxying_to_dagster(
+ proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"),
+ global_vars=globals(),
+ )
+ ```
+ You can also provide custom implementations of the build_from_task_fn function to customize the behavior of task-level proxying.
+
+ ```python
+ from dagster_airlift.in_airflow import proxying_to_dagster, BaseProxyTaskToDagsterOperator
+ from airflow.models.operator import BaseOperator
+
+ ... # Dag code here
+
+ class CustomAuthTaskProxyOperator(BaseProxyTaskToDagsterOperator):
+ def get_dagster_session(self, context: Context) -> requests.Session:
+ # Add custom headers to the session
+ return requests.Session(headers={"Authorization": "Bearer my_token"})
+
+ def get_dagster_url(self, context: Context) -> str:
+ # Use a custom environment variable for the dagster url
+ return os.environ["CUSTOM_DAGSTER_URL"]
+
+ @classmethod
+ def build_from_task(cls, task: BaseOperator) -> "CustomAuthTaskProxyOperator":
+ # Custom logic to build the operator from the task (task_id should remain the same)
+ if task.task_id == "my_task_needs_more_retries":
+ return CustomAuthTaskProxyOperator(task_id=task_id, retries=3)
+ else:
+ return CustomAuthTaskProxyOperator(task_id=task_id)
+
+ proxying_to_dagster(
+ proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"),
+ global_vars=globals(),
+ build_from_task_fn=CustomAuthTaskProxyOperator.build_from_task,
+ )
+ ```
+ You can do the same for dag-level proxying by providing a custom implementation of the build_from_dag_fn function.
+
+ ```python
+ from dagster_airlift.in_airflow import proxying_to_dagster, BaseProxyDAGToDagsterOperator
+ from airflow.models.dag import DAG
+
+ ... # Dag code here
+
+ class CustomAuthDAGProxyOperator(BaseProxyDAGToDagsterOperator):
+ def get_dagster_session(self, context: Context) -> requests.Session:
+ # Add custom headers to the session
+ return requests.Session(headers={"Authorization": "Bearer my_token"})
+
+ def get_dagster_url(self, context: Context) -> str:
+ # Use a custom environment variable for the dagster url
+ return os.environ["CUSTOM_DAGSTER_URL"]
+
+ @classmethod
+ def build_from_dag(cls, dag: DAG) -> "CustomAuthDAGProxyOperator":
+ # Custom logic to build the operator from the dag (DAG id should remain the same)
+ if dag.dag_id == "my_dag_needs_more_retries":
+ return CustomAuthDAGProxyOperator(task_id="custom override", retries=3, dag=dag)
+ else:
+ return CustomAuthDAGProxyOperator(task_id="basic_override", dag=dag)
+
+ proxying_to_dagster(
+ proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"),
+ global_vars=globals(),
+ build_from_dag_fn=CustomAuthDAGProxyOperator.build_from_dag,
+ )
+ ```
+
+
+
+
+
+
+
class dagster_airlift.in_airflow.BaseDagsterAssetsOperator
+
+
+ Interface for an operator which materializes dagster assets.
+
+ This operator needs to implement the following methods:
+
+ >
+
+ - get_dagster_session: Returns a requests session that can be used to make requests to the Dagster API.
+ - get_dagster_url: Returns the URL for the Dagster instance.
+ - filter_asset_nodes: Filters asset nodes (which are returned from Dagster’s graphql API) to only include those
+
+
+
+
+Optionally, these methods can be overridden as well:
+
+>
+
+ - get_partition_key: Determines the partition key to use to trigger the dagster run. This method will only be
+
+
+
+
+
+
+
+ Loads the proxied state from a directory of yaml files.
+
+ Expects the directory to contain yaml files, where each file corresponds to the id of a dag (ie: dag_id.yaml).
+ This directory is typically constructed using the dagster-airlift CLI:
+
+ >
+
+ ```bash
+ AIRFLOW_HOME=... dagster-airlift proxy scaffold
+ ```
+
+
+The file should have either of the following structure.
+In the case of task-level proxying:
+
+>
+
+ ```yaml
+ tasks:
+ - id: task_id
+ proxied: true
+ - id: task_id
+ proxied: false
+ ```
+
+
+In the case of dag-level proxying:
+
+>
+
+ ```yaml
+ proxied: true
+ ```
+
+
+Parameters: proxied_yaml_path (Path) – The path to the directory containing the yaml files.Returns: The proxied state of the dags and tasks in Airflow.Return type: [AirflowProxiedState](#dagster_airlift.in_airflow.AirflowProxiedState)
+
+
+
+
+
+
+
+#### Proxying State
+
+
+
class dagster_airlift.in_airflow.AirflowProxiedState
+
+
+ A class to store the proxied state of dags and tasks in Airflow.
+ Typically, this is constructed by [`load_proxied_state_from_yaml()`](#dagster_airlift.in_airflow.load_proxied_state_from_yaml).
+
+ Parameters: dags (Dict[str, [*DagProxiedState*](#dagster_airlift.in_airflow.DagProxiedState)]) – A dictionary of dag_id to DagProxiedState.
+
+
+
+
+
+
class dagster_airlift.in_airflow.DagProxiedState
+
+
+ A class to store the proxied state of tasks in a dag.
+
+ Parameters:
+ - tasks (Dict[str, [*TaskProxiedState*](#dagster_airlift.in_airflow.TaskProxiedState)]) – A dictionary of task_id to TaskProxiedState. If the entire dag is proxied, or proxied state
+ - proxied (Optional[bool]) – A boolean indicating whether the entire dag is proxied. If this is None, then the dag proxies at the task level (or
+ - all). (proxying state has not been set at)
+
+
+
+
+
+
+
+
class dagster_airlift.in_airflow.TaskProxiedState
+
+
+ A class to store the proxied state of a task.
+
+ Parameters:
+ - task_id (str) – The id of the task.
+ - proxied (bool) – A boolean indicating whether the task is proxied.
+
+
+
+
+
+
+
+
+
+
+#### Task-level Proxying
+
+
+
+
class dagster_airlift.in_airflow.BaseProxyTaskToDagsterOperator
+
+
+ An operator that proxies task execution to Dagster assets with metadata that map to this task’s dag ID and task ID.
+
+ For the DAG ID and task ID that this operator proxies, it expects there to be corresponding assets
+ in the linked Dagster deployment that have metadata entries with the key dagster-airlift/task-mapping that
+ map to this DAG ID and task ID. This metadata is typically set using the
+ [`dagster_airlift.core.assets_with_task_mappings()`](#dagster_airlift.core.assets_with_task_mappings) function.
+
+ The following methods must be implemented by subclasses:
+
+ >
+
+ - `get_dagster_session()` (inherited from [`BaseDagsterAssetsOperator`](#dagster_airlift.in_airflow.BaseDagsterAssetsOperator))
+ - `get_dagster_url()` (inherited from [`BaseDagsterAssetsOperator`](#dagster_airlift.in_airflow.BaseDagsterAssetsOperator))
+ - `build_from_task()` A class method which takes the task to be proxied, and constructs
+
+
+
+
+There is a default implementation of this operator, [`DefaultProxyTaskToDagsterOperator`](#dagster_airlift.in_airflow.DefaultProxyTaskToDagsterOperator),
+which is used by [`proxying_to_dagster()`](#dagster_airlift.in_airflow.proxying_to_dagster) if no override operator is provided.
+
+
+
+
+
+
+
class dagster_airlift.in_airflow.DefaultProxyTaskToDagsterOperator
+
+
+ The default task proxying operator - which opens a blank session and expects the dagster URL to be set in the environment.
+ The dagster url is expected to be set in the environment as DAGSTER_URL.
+
+ This operator should not be instantiated directly - it is instantiated by [`proxying_to_dagster()`](#dagster_airlift.in_airflow.proxying_to_dagster) if no
+ override operator is provided.
+
+
+
+
+
+
+
+
+
+#### Dag-level Proxying
+
+
+
+
class dagster_airlift.in_airflow.BaseProxyDAGToDagsterOperator
+
+
+ An operator base class that proxies the entire DAG’s execution to Dagster assets with
+ metadata that map to the DAG id used by this task.
+
+ For the Dag ID that this operator proxies, it expects there to be corresponding assets
+ in the linked Dagster deployment that have metadata entries with the key dagster-airlift/dag-mapping that
+ map to this Dag ID. This metadata is typically set using the
+ [`dagster_airlift.core.assets_with_dag_mappings()`](#dagster_airlift.core.assets_with_dag_mappings) function.
+
+ The following methods must be implemented by subclasses:
+
+ >
+
+ - `get_dagster_session()` (inherited from [`BaseDagsterAssetsOperator`](#dagster_airlift.in_airflow.BaseDagsterAssetsOperator))
+ - `get_dagster_url()` (inherited from [`BaseDagsterAssetsOperator`](#dagster_airlift.in_airflow.BaseDagsterAssetsOperator))
+ - `build_from_dag()` A class method which takes the DAG to be proxied, and constructs
+
+
+
+
+There is a default implementation of this operator, [`DefaultProxyDAGToDagsterOperator`](#dagster_airlift.in_airflow.DefaultProxyDAGToDagsterOperator),
+which is used by [`proxying_to_dagster()`](#dagster_airlift.in_airflow.proxying_to_dagster) if no override operator is provided.
+
+
+
+
+
+
+
class dagster_airlift.in_airflow.DefaultProxyDAGToDagsterOperator
+
+
+ The default task proxying operator - which opens a blank session and expects the dagster URL to be set in the environment.
+ The dagster url is expected to be set in the environment as DAGSTER_URL.
+
+ This operator should not be instantiated directly - it is instantiated by [`proxying_to_dagster()`](#dagster_airlift.in_airflow.proxying_to_dagster) if no
+ override operator is provided.
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-aws.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-aws.mdx
new file mode 100644
index 0000000000000..16d39be4ff5a5
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-aws.mdx
@@ -0,0 +1,1316 @@
+---
+title: 'aws (dagster-aws)'
+title_meta: 'aws (dagster-aws) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'aws (dagster-aws) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# AWS (dagster-aws)
+
+Utilities for interfacing with AWS with Dagster.
+
+
+
+
+## S3
+
+
+
dagster_aws.s3.S3Resource ResourceDefinition
+
+
+
+
+ Resource that gives access to S3.
+
+ The underlying S3 session is created by calling
+ `boto3.session.Session(profile_name)`.
+ The returned resource object is an S3 client, an instance of botocore.client.S3.
+
+ Example:
+
+ ```python
+ from dagster import job, op, Definitions
+ from dagster_aws.s3 import S3Resource
+
+ @op
+ def example_s3_op(s3: S3Resource):
+ return s3.get_client().list_objects_v2(
+ Bucket='my-bucket',
+ Prefix='some-key'
+ )
+
+ @job
+ def example_job():
+ example_s3_op()
+
+ defs = Definitions(
+ jobs=[example_job],
+ resources={'s3': S3Resource(region_name='us-west-1')}
+ )
+ ```
+
+
+
+ Persistent IO manager using S3 for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for S3 and the backing bucket.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ ```python
+ from dagster import asset, Definitions
+ from dagster_aws.s3 import S3PickleIOManager, S3Resource
+
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": S3PickleIOManager(
+ s3_resource=S3Resource(),
+ s3_bucket="my-cool-bucket",
+ s3_prefix="my-cool-prefix",
+ )
+ }
+ )
+ ```
+
+
+
+
+
+
class dagster_aws.s3.S3ComputeLogManager
+
+
+ Logs compute function stdout and stderr to S3.
+
+ Users should not instantiate this class directly. Instead, use a YAML block in `dagster.yaml`
+ such as the following:
+
+ ```YAML
+ compute_logs:
+ module: dagster_aws.s3.compute_log_manager
+ class: S3ComputeLogManager
+ config:
+ bucket: "mycorp-dagster-compute-logs"
+ local_dir: "/tmp/cool"
+ prefix: "dagster-test-"
+ use_ssl: true
+ verify: true
+ verify_cert_path: "/path/to/cert/bundle.pem"
+ endpoint_url: "http://alternate-s3-host.io"
+ skip_empty_files: true
+ upload_interval: 30
+ upload_extra_args:
+ ServerSideEncryption: "AES256"
+ show_url_only: false
+ region: "us-west-1"
+ ```
+ Parameters:
+ - bucket (str) – The name of the s3 bucket to which to log.
+ - local_dir (Optional[str]) – Path to the local directory in which to stage logs. Default:
+ - prefix (Optional[str]) – Prefix for the log file keys.
+ - use_ssl (Optional[bool]) – Whether or not to use SSL. Default True.
+ - verify (Optional[bool]) – Whether or not to verify SSL certificates. Default True.
+ - verify_cert_path (Optional[str]) – A filename of the CA cert bundle to use. Only used if
+ - endpoint_url (Optional[str]) – Override for the S3 endpoint url.
+ - skip_empty_files – (Optional[bool]): Skip upload of empty log files.
+ - upload_interval – (Optional[int]): Interval in seconds to upload partial log files to S3. By default, will only upload when the capture is complete.
+ - upload_extra_args – (Optional[dict]): Extra args for S3 file upload
+ - show_url_only – (Optional[bool]): Only show the URL of the log file in the UI, instead of fetching and displaying the full content. Default False.
+ - region – (Optional[str]): The region of the S3 bucket. If not specified, will use the default region of the AWS session.
+ - inst_data (Optional[[*ConfigurableClassData*](../internals.mdx#dagster._serdes.ConfigurableClassData)]) – Serializable representation of the compute
+
+
+
+
+
+
+
+
dagster_aws.s3.S3Coordinate DagsterType
+
+
+ A [`dagster.DagsterType`](../types.mdx#dagster.DagsterType) intended to make it easier to pass information about files on S3
+ from op to op. Objects of this type should be dicts with `'bucket'` and `'key'` keys,
+ and may be hydrated from config in the intuitive way, e.g., for an input with the name
+ `s3_file`:
+
+ ```YAML
+ inputs:
+ s3_file:
+ value:
+ bucket: my-bucket
+ key: my-key
+ ```
+
+
+
+ Base class for Dagster resources that utilize structured config.
+
+ This class is a subclass of both `ResourceDefinition` and `Config`.
+
+ Example definition:
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ prefix: str
+
+ def output(self, text: str) -> None:
+ print(f"{self.prefix}{text}")
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def asset_that_uses_writer(writer: WriterResource):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[asset_that_uses_writer],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+ You can optionally use this class to model configuration only and vend an object
+ of a different type for use at runtime. This is useful for those who wish to
+ have a separate object that manages configuration and a separate object at runtime. Or
+ where you want to directly use a third-party class that you do not control.
+
+ To do this you override the create_resource methods to return a different object.
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ str: prefix
+
+ def create_resource(self, context: InitResourceContext) -> Writer:
+ # Writer is pre-existing class defined else
+ return Writer(self.prefix)
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def use_preexisting_writer_as_resource(writer: ResourceParam[Writer]):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[use_preexisting_writer_as_resource],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+## ECS
+
+
+
dagster_aws.ecs.EcsRunLauncher RunLauncher
+
+
+
+
+ RunLauncher that starts a task in ECS for each Dagster job run.
+
+
+
+
+
+
+
dagster_aws.ecs.ecs_executor ExecutorDefinition
+
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Executor which launches steps as ECS tasks.
+
+ To use the ecs_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster_aws.ecs import ecs_executor
+
+ from dagster import job, op
+
+
+ @op(
+ tags={"ecs/cpu": "256", "ecs/memory": "512"},
+ )
+ def ecs_op():
+ pass
+
+
+ @job(executor_def=ecs_executor)
+ def ecs_job():
+ ecs_op()
+
+
+ ```
+ Then you can configure the executor with run config as follows:
+
+ ```YAML
+ execution:
+ config:
+ cpu: 1024
+ memory: 2048
+ ephemeral_storage: 10
+ task_overrides:
+ containerOverrides:
+ - name: run
+ environment:
+ - name: MY_ENV_VAR
+ value: "my_value"
+ ```
+ max_concurrent limits the number of ECS tasks that will execute concurrently for one run. By default
+ there is no limit- it will maximally parallel as allowed by the DAG. Note that this is not a
+ global limit.
+
+ Configuration set on the ECS tasks created by the ECSRunLauncher will also be
+ set on the tasks created by the ecs_executor.
+
+ Configuration set using tags on a @job will only apply to the run level. For configuration
+ to apply at each step it must be set using tags for each @op.
+
+
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ While there is no plan to remove this functionality, for new projects, we recommend using Dagster Pipes. For more information, see https://docs.dagster.io/guides/build/external-pipelines/.
+
+ :::
+
+ - spark_config:
+ - cluster_id: Name of the job flow (cluster) on which to execute.
+ - region_name: The AWS region that the cluster is in.
+ - action_on_failure: The EMR action to take when the cluster step fails: [https://docs.aws.amazon.com/emr/latest/APIReference/API_StepConfig.html](https://docs.aws.amazon.com/emr/latest/APIReference/API_StepConfig.html)
+ - staging_bucket: S3 bucket to use for passing files between the plan process and EMR process.
+ - staging_prefix: S3 key prefix inside the staging_bucket to use for files passed the plan process and EMR process
+ - wait_for_logs: If set, the system will wait for EMR logs to appear on S3. Note that logs are copied every 5 minutes, so enabling this will add several minutes to the job runtime.
+ - local_job_package_path: Absolute path to the package that contains the job definition(s) whose steps will execute remotely on EMR. This is a path on the local fileystem of the process executing the job. The expectation is that this package will also be available on the python path of the launched process running the Spark step on EMR, either deployed on step launch via the deploy_local_job_package option, referenced on s3 via the s3_job_package_path option, or installed on the cluster via bootstrap actions.
+ - local_pipeline_package_path: (legacy) Absolute path to the package that contains the pipeline definition(s) whose steps will execute remotely on EMR. This is a path on the local fileystem of the process executing the pipeline. The expectation is that this package will also be available on the python path of the launched process running the Spark step on EMR, either deployed on step launch via the deploy_local_pipeline_package option, referenced on s3 via the s3_pipeline_package_path option, or installed on the cluster via bootstrap actions.
+ - deploy_local_job_package: If set, before every step run, the launcher will zip up all the code in local_job_package_path, upload it to s3, and pass it to spark-submit’s –py-files option. This gives the remote process access to up-to-date user code. If not set, the assumption is that some other mechanism is used for distributing code to the EMR cluster. If this option is set to True, s3_job_package_path should not also be set.
+ - deploy_local_pipeline_package: (legacy) If set, before every step run, the launcher will zip up all the code in local_job_package_path, upload it to s3, and pass it to spark-submit’s –py-files option. This gives the remote process access to up-to-date user code. If not set, the assumption is that some other mechanism is used for distributing code to the EMR cluster. If this option is set to True, s3_job_package_path should not also be set.
+ - s3_job_package_path: If set, this path will be passed to the –py-files option of spark-submit. This should usually be a path to a zip file. If this option is set, deploy_local_job_package should not be set to True.
+ - s3_pipeline_package_path: If set, this path will be passed to the –py-files option of spark-submit. This should usually be a path to a zip file. If this option is set, deploy_local_pipeline_package should not be set to True.
+
+
+
+
+
+ Core class for defining loggers.
+
+ Loggers are job-scoped logging handlers, which will be automatically invoked whenever
+ dagster messages are logged from within a job.
+
+ Parameters:
+ - logger_fn (Callable[[[*InitLoggerContext*](../loggers.mdx#dagster.InitLoggerContext)], logging.Logger]) – User-provided function to
+ - config_schema (Optional[[*ConfigSchema*](../config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of this logger.
+
+
+
+
+
+
+
+
+
+
+
+
+## SecretsManager
+
+Resources which surface SecretsManager secrets for use in Dagster resources and jobs.
+
+
+
+ Resource that provides a dict which maps selected SecretsManager secrets to
+ their string values. Also optionally sets chosen secrets as environment variables.
+
+ Example:
+
+ ```python
+ import os
+ from dagster import build_op_context, job, op, ResourceParam
+ from dagster_aws.secretsmanager import SecretsManagerSecretsResource
+
+ @op
+ def example_secretsmanager_secrets_op(secrets: SecretsManagerSecretsResource):
+ return secrets.fetch_secrets().get("my-secret-name")
+
+ @op
+ def example_secretsmanager_secrets_op_2(secrets: SecretsManagerSecretsResource):
+ with secrets.secrets_in_environment():
+ return os.getenv("my-other-secret-name")
+
+ @job
+ def example_job():
+ example_secretsmanager_secrets_op()
+ example_secretsmanager_secrets_op_2()
+
+ defs = Definitions(
+ jobs=[example_job],
+ resources={
+ 'secrets': SecretsManagerSecretsResource(
+ region_name='us-west-1',
+ secrets_tag="dagster",
+ add_to_environment=True,
+ )
+ }
+ )
+ ```
+ Note that your ops must also declare that they require this resource with or it will not be initialized
+ for the execution of their compute functions.
+
+
+
+
+
+
+
+
+
+
+
+## Pipes
+
+
+
+
+### Context Injectors
+
+
+
class dagster_aws.pipes.PipesS3ContextInjector
+
+
+ A context injector that injects context by writing to a temporary S3 location.
+
+ Parameters:
+ - bucket (str) – The S3 bucket to write to.
+ - client (boto3.client) – A boto3 client to use to write to S3.
+ - key_prefix (Optional[str]) – An optional prefix to use for the S3 key. Defaults to a random
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesLambdaEventContextInjector
+
+ Injects context via AWS Lambda event input.
+ Should be paired with :py:class`~dagster_pipes.PipesMappingParamsLoader` on the Lambda side.
+
+
+
+
+
+
+
+
+
+### Message Readers
+
+
+
class dagster_aws.pipes.PipesS3MessageReader
+
+
+ Message reader that reads messages by periodically reading message chunks from a specified S3
+ bucket.
+
+ If log_readers is passed, this reader will also start the passed readers
+ when the first message is received from the external process.
+
+ Parameters:
+ - interval (float) – interval in seconds between attempts to download a chunk
+ - bucket (str) – The S3 bucket to read from.
+ - client (WorkspaceClient) – A boto3 client.
+ - log_readers (Optional[Sequence[PipesLogReader]]) – A set of log readers for logs on S3.
+ - include_stdio_in_messages (bool) – Whether to send stdout/stderr to Dagster via Pipes messages. Defaults to False.
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesCloudWatchMessageReader
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Message reader that consumes AWS CloudWatch logs to read pipes messages.
+
+
+
+
+
+
+
+
+
+
+
+### Clients
+
+
+
class dagster_aws.pipes.PipesLambdaClient
+
+
+ A pipes client for invoking AWS lambda.
+
+ By default context is injected via the lambda input event and messages are parsed out of the
+ 4k tail of logs.
+
+ Parameters:
+ - client (boto3.client) – The boto lambda client used to call invoke.
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+
+
+
+
run
+
+
+ Synchronously invoke a lambda function, enriched with the pipes protocol.
+
+ Parameters:
+ - function_name (str) – The name of the function to use.
+ - event (Mapping[str, Any]) – A JSON serializable object to pass as input to the lambda.
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesGlueClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client for invoking AWS Glue jobs.
+
+ Parameters:
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - client (Optional[boto3.client]) – The boto Glue client used to launch the Glue job
+ - forward_termination (bool) – Whether to cancel the Glue job run when the Dagster process receives a termination signal.
+
+
+
+
run
+
+
+ Start a Glue job, enriched with the pipes protocol.
+
+ See also: [AWS API Documentation](https://docs.aws.amazon.com/goto/WebAPI/glue-2017-03-31/StartJobRun)
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+ - start_job_run_params (Dict) – Parameters for the `start_job_run` boto3 Glue client call.
+ - extras (Optional[Dict[str, Any]]) – Additional Dagster metadata to pass to the Glue job.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesECSClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client for running AWS ECS tasks.
+
+ Parameters:
+ - client (Any) – The boto ECS client used to launch the ECS task
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - forward_termination (bool) – Whether to cancel the ECS task when the Dagster process receives a termination signal.
+
+
+
+
run
+
+
+ Run ECS tasks, enriched with the pipes protocol.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+ - run_task_params (dict) – Parameters for the `run_task` boto3 ECS client call.
+ - extras (Optional[Dict[str, Any]]) – Additional information to pass to the Pipes session in the external process.
+ - pipes_container_name (Optional[str]) – If running more than one container in the task,
+ - waiter_config (Optional[WaiterConfig]) – Optional waiter configuration to use. Defaults to 70 days (Delay: 6, MaxAttempts: 1000000).
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesEMRClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client for running jobs on AWS EMR.
+
+ Parameters:
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - client (Optional[boto3.client]) – The boto3 EMR client used to interact with AWS EMR.
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - forward_termination (bool) – Whether to cancel the EMR job if the Dagster process receives a termination signal.
+ - wait_for_s3_logs_seconds (int) – The number of seconds to wait for S3 logs to be written after execution completes.
+ - s3_application_logs_prefix (str) – The prefix to use when looking for application logs in S3.
+
+
+
+
run
+
+
+ Run a job on AWS EMR, enriched with the pipes protocol.
+
+ Starts a new EMR cluster for each invocation.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+ - run_job_flow_params (Optional[dict]) – Parameters for the `run_job_flow` boto3 EMR client call.
+ - extras (Optional[Dict[str, Any]]) – Additional information to pass to the Pipes session in the external process.
+
+
+ Returns: Wrapper containing results reported by the external process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesEMRContainersClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client for running workloads on AWS EMR Containers.
+
+ Parameters:
+ - client (Optional[boto3.client]) – The boto3 AWS EMR containers client used to interact with AWS EMR Containers.
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - forward_termination (bool) – Whether to cancel the AWS EMR Containers workload if the Dagster process receives a termination signal.
+ - pipes_params_bootstrap_method (Literal["args", "env"]) – The method to use to inject parameters into the AWS EMR Containers workload. Defaults to “args”.
+ - waiter_config (Optional[WaiterConfig]) – Optional waiter configuration to use. Defaults to 70 days (Delay: 6, MaxAttempts: 1000000).
+
+
+
+
run
+
+
+ Run a workload on AWS EMR Containers, enriched with the pipes protocol.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+ - params (dict) – Parameters for the `start_job_run` boto3 AWS EMR Containers client call.
+ - extras (Optional[Dict[str, Any]]) – Additional information to pass to the Pipes session in the external process.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_aws.pipes.PipesEMRServerlessClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client for running workloads on AWS EMR Serverless.
+
+ Parameters:
+ - client (Optional[boto3.client]) – The boto3 AWS EMR Serverless client used to interact with AWS EMR Serverless.
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - forward_termination (bool) – Whether to cancel the AWS EMR Serverless workload if the Dagster process receives a termination signal.
+ - poll_interval (float) – The interval in seconds to poll the AWS EMR Serverless workload for status updates. Defaults to 5 seconds.
+
+
+
+
run
+
+
+ Run a workload on AWS EMR Serverless, enriched with the pipes protocol.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The context of the currently executing Dagster op or asset.
+ - params (dict) – Parameters for the `start_job_run` boto3 AWS EMR Serverless client call.
+ - extras (Optional[Dict[str, Any]]) – Additional information to pass to the Pipes session in the external process.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Please use S3PickleIOManager instead..
+
+ :::
+
+ Renamed to S3PickleIOManager. See S3PickleIOManager for documentation.
+
+
+
+
+
+
+
dagster_aws.s3.s3_resource ResourceDefinition
+
+
+
+
+ Resource that gives access to S3.
+
+ The underlying S3 session is created by calling
+ `boto3.session.Session(profile_name)`.
+ The returned resource object is an S3 client, an instance of botocore.client.S3.
+
+ Example:
+
+ ```python
+ from dagster import build_op_context, job, op
+ from dagster_aws.s3 import s3_resource
+
+ @op(required_resource_keys={'s3'})
+ def example_s3_op(context):
+ return context.resources.s3.list_objects_v2(
+ Bucket='my-bucket',
+ Prefix='some-key'
+ )
+
+ @job(resource_defs={'s3': s3_resource})
+ def example_job():
+ example_s3_op()
+
+ example_job.execute_in_process(
+ run_config={
+ 'resources': {
+ 's3': {
+ 'config': {
+ 'region_name': 'us-west-1',
+ }
+ }
+ }
+ }
+ )
+ ```
+ Note that your ops must also declare that they require this resource with
+ required_resource_keys, or it will not be initialized for the execution of their compute
+ functions.
+
+ You may configure this resource as follows:
+
+ ```YAML
+ resources:
+ s3:
+ config:
+ region_name: "us-west-1"
+ # Optional[str]: Specifies a custom region for the S3 session. Default is chosen
+ # through the ordinary boto credential chain.
+ use_unsigned_session: false
+ # Optional[bool]: Specifies whether to use an unsigned S3 session. Default: True
+ endpoint_url: "http://localhost"
+ # Optional[str]: Specifies a custom endpoint for the S3 session. Default is None.
+ profile_name: "dev"
+ # Optional[str]: Specifies a custom profile for S3 session. Default is default
+ # profile as specified in ~/.aws/credentials file
+ use_ssl: true
+ # Optional[bool]: Whether or not to use SSL. By default, SSL is used.
+ verify: None
+ # Optional[str]: Whether or not to verify SSL certificates. By default SSL certificates are verified.
+ # You can also specify this argument if you want to use a different CA cert bundle than the one used by botocore."
+ aws_access_key_id: None
+ # Optional[str]: The access key to use when creating the client.
+ aws_secret_access_key: None
+ # Optional[str]: The secret key to use when creating the client.
+ aws_session_token: None
+ # Optional[str]: The session token to use when creating the client.
+ ```
+
+
+
+ Persistent IO manager using S3 for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for S3 and the backing bucket.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ 1. Attach this IO manager to a set of assets.
+ ```python
+ from dagster import Definitions, asset
+ from dagster_aws.s3 import s3_pickle_io_manager, s3_resource
+
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": s3_pickle_io_manager.configured(
+ {"s3_bucket": "my-cool-bucket", "s3_prefix": "my-cool-prefix"}
+ ),
+ "s3": s3_resource,
+ },
+ )
+ ```
+ 2. Attach this IO manager to your job to make it available to your ops.
+ ```python
+ from dagster import job
+ from dagster_aws.s3 import s3_pickle_io_manager, s3_resource
+
+ @job(
+ resource_defs={
+ "io_manager": s3_pickle_io_manager.configured(
+ {"s3_bucket": "my-cool-bucket", "s3_prefix": "my-cool-prefix"}
+ ),
+ "s3": s3_resource,
+ },
+ )
+ def my_job():
+ ...
+ ```
+
+
+
+
+
+
dagster_aws.s3.s3_file_manager ResourceDefinition
+
+
+
+
+ FileManager that provides abstract access to S3.
+
+ Implements the [`FileManager`](../internals.mdx#dagster._core.storage.file_manager.FileManager) API.
+
+
+
+
+ Resource that provides a dict which maps selected SecretsManager secrets to
+ their string values. Also optionally sets chosen secrets as environment variables.
+
+ Example:
+
+ ```python
+ import os
+ from dagster import build_op_context, job, op
+ from dagster_aws.secretsmanager import secretsmanager_secrets_resource
+
+ @op(required_resource_keys={'secrets'})
+ def example_secretsmanager_secrets_op(context):
+ return context.resources.secrets.get("my-secret-name")
+
+ @op(required_resource_keys={'secrets'})
+ def example_secretsmanager_secrets_op_2(context):
+ return os.getenv("my-other-secret-name")
+
+ @job(resource_defs={'secrets': secretsmanager_secrets_resource})
+ def example_job():
+ example_secretsmanager_secrets_op()
+ example_secretsmanager_secrets_op_2()
+
+ example_job.execute_in_process(
+ run_config={
+ 'resources': {
+ 'secrets': {
+ 'config': {
+ 'region_name': 'us-west-1',
+ 'secrets_tag': 'dagster',
+ 'add_to_environment': True,
+ }
+ }
+ }
+ }
+ )
+ ```
+ Note that your ops must also declare that they require this resource with
+ required_resource_keys, or it will not be initialized for the execution of their compute
+ functions.
+
+ You may configure this resource as follows:
+
+ ```YAML
+ resources:
+ secretsmanager:
+ config:
+ region_name: "us-west-1"
+ # Optional[str]: Specifies a custom region for the SecretsManager session. Default is chosen
+ # through the ordinary boto credential chain.
+ profile_name: "dev"
+ # Optional[str]: Specifies a custom profile for SecretsManager session. Default is default
+ # profile as specified in ~/.aws/credentials file
+ secrets: ["arn:aws:secretsmanager:region:aws_account_id:secret:appauthexample-AbCdEf"]
+ # Optional[List[str]]: Specifies a list of secret ARNs to pull from SecretsManager.
+ secrets_tag: "dagster"
+ # Optional[str]: Specifies a tag, all secrets which have the tag set will be pulled
+ # from SecretsManager.
+ add_to_environment: true
+ # Optional[bool]: Whether to set the selected secrets as environment variables. Defaults
+ # to false.
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-azure.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-azure.mdx
new file mode 100644
index 0000000000000..5bce54f557cf4
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-azure.mdx
@@ -0,0 +1,448 @@
+---
+title: 'azure (dagster-azure)'
+title_meta: 'azure (dagster-azure) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'azure (dagster-azure) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Azure (dagster-azure)
+
+Utilities for using Azure Storage Accounts with Dagster. This is mostly aimed at Azure Data Lake
+Storage Gen 2 (ADLS2) but also contains some utilities for Azure Blob Storage.
+
+
+
+ Resource containing clients to access Azure Data Lake Storage Gen2.
+
+ Contains a client for both the Data Lake and Blob APIs, to work around the limitations
+ of each.
+
+ Example usage:
+
+ Attach this resource to your Definitions to be used by assets and jobs.
+
+ ```python
+ from dagster import Definitions, asset, job, op
+ from dagster_azure.adls2 import ADLS2Resource, ADLS2SASToken
+
+ @asset
+ def asset1(adls2: ADLS2Resource):
+ adls2.adls2_client.list_file_systems()
+ ...
+
+ @op
+ def my_op(adls2: ADLS2Resource):
+ adls2.adls2_client.list_file_systems()
+ ...
+
+ @job
+ def my_job():
+ my_op()
+
+ defs = Definitions(
+ assets=[asset1],
+ jobs=[my_job],
+ resources={
+ "adls2": ADLS2Resource(
+ storage_account="my-storage-account",
+ credential=ADLS2SASToken(token="my-sas-token"),
+ )
+ },
+ )
+ ```
+ Attach this resource to your job to make it available to your ops.
+
+ ```python
+ from dagster import job, op
+ from dagster_azure.adls2 import ADLS2Resource, ADLS2SASToken
+
+ @op
+ def my_op(adls2: ADLS2Resource):
+ adls2.adls2_client.list_file_systems()
+ ...
+
+ @job(
+ resource_defs={
+ "adls2": ADLS2Resource(
+ storage_account="my-storage-account",
+ credential=ADLS2SASToken(token="my-sas-token"),
+ )
+ },
+ )
+ def my_job():
+ my_op()
+ ```
+
+
+
+ Stateful mock of an ADLS2Resource for testing.
+
+ Wraps a `mock.MagicMock`. Containers are implemented using an in-memory dict.
+
+
+
+
+
+
+
class dagster_azure.blob.AzureBlobComputeLogManager
+
+
+ Logs op compute function stdout and stderr to Azure Blob Storage.
+
+ This is also compatible with Azure Data Lake Storage.
+
+ Users should not instantiate this class directly. Instead, use a YAML block in `dagster.yaml`. Examples provided below
+ will show how to configure with various credentialing schemes.
+
+ Parameters:
+ - storage_account (str) – The storage account name to which to log.
+ - container (str) – The container (or ADLS2 filesystem) to which to log.
+ - secret_credential (Optional[dict]) – Secret credential for the storage account. This should be
+ - access_key_or_sas_token (Optional[str]) – Access key or SAS token for the storage account.
+ - default_azure_credential (Optional[dict]) – Use and configure DefaultAzureCredential.
+ - local_dir (Optional[str]) – Path to the local directory in which to stage logs. Default:
+ - prefix (Optional[str]) – Prefix for the log file keys.
+ - upload_interval (Optional[int]) – Interval in seconds to upload partial log files blob storage. By default, will only upload when the capture is complete.
+ - show_url_only (bool) – Only show the URL of the log file in the UI, instead of fetching and displaying the full content. Default False.
+ - inst_data (Optional[[*ConfigurableClassData*](../internals.mdx#dagster._serdes.ConfigurableClassData)]) – Serializable representation of the compute
+
+
+ Examples:
+ Using an Azure Blob Storage account with an [AzureSecretCredential](https://learn.microsoft.com/en-us/python/api/azure-identity/azure.identity.clientsecretcredential?view=azure-python):
+
+ ```YAML
+ compute_logs:
+ module: dagster_azure.blob.compute_log_manager
+ class: AzureBlobComputeLogManager
+ config:
+ storage_account: my-storage-account
+ container: my-container
+ secret_credential:
+ client_id: my-client-id
+ client_secret: my-client-secret
+ tenant_id: my-tenant-id
+ prefix: "dagster-test-"
+ local_dir: "/tmp/cool"
+ upload_interval: 30
+ show_url_only: false
+ ```
+ Using an Azure Blob Storage account with a [DefaultAzureCredential](https://learn.microsoft.com/en-us/python/api/azure-identity/azure.identity.defaultazurecredential?view=azure-python):
+
+ ```YAML
+ compute_logs:
+ module: dagster_azure.blob.compute_log_manager
+ class: AzureBlobComputeLogManager
+ config:
+ storage_account: my-storage-account
+ container: my-container
+ default_azure_credential:
+ exclude_environment_credential: false
+ prefix: "dagster-test-"
+ local_dir: "/tmp/cool"
+ upload_interval: 30
+ show_url_only: false
+ ```
+ Using an Azure Blob Storage account with an access key:
+
+ ```YAML
+ compute_logs:
+ module: dagster_azure.blob.compute_log_manager
+ class: AzureBlobComputeLogManager
+ config:
+ storage_account: my-storage-account
+ container: my-container
+ access_key_or_sas_token: my-access-key
+ prefix: "dagster-test-"
+ local_dir: "/tmp/cool"
+ upload_interval: 30
+ show_url_only: false
+ ```
+
+
+
+ Persistent IO manager using Azure Data Lake Storage Gen2 for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for ADLS and the backing
+ container.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ 1. Attach this IO manager to a set of assets.
+ ```python
+ from dagster import Definitions, asset
+ from dagster_azure.adls2 import ADLS2PickleIOManager, ADLS2Resource, ADLS2SASToken
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return df[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": ADLS2PickleIOManager(
+ adls2_file_system="my-cool-fs",
+ adls2_prefix="my-cool-prefix",
+ adls2=ADLS2Resource(
+ storage_account="my-storage-account",
+ credential=ADLS2SASToken(token="my-sas-token"),
+ ),
+ ),
+ },
+ )
+ ```
+ 2. Attach this IO manager to your job to make it available to your ops.
+ ```python
+ from dagster import job
+ from dagster_azure.adls2 import ADLS2PickleIOManager, ADLS2Resource, ADLS2SASToken
+
+ @job(
+ resource_defs={
+ "io_manager": ADLS2PickleIOManager(
+ adls2_file_system="my-cool-fs",
+ adls2_prefix="my-cool-prefix",
+ adls2=ADLS2Resource(
+ storage_account="my-storage-account",
+ credential=ADLS2SASToken(token="my-sas-token"),
+ ),
+ ),
+ },
+ )
+ def my_job():
+ ...
+ ```
+
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Please use ADLS2PickleIOManager instead..
+
+ :::
+
+ Renamed to ADLS2PickleIOManager. See ADLS2PickleIOManager for documentation.
+
+
+
+
+ Resource that gives ops access to Azure Data Lake Storage Gen2.
+
+ The underlying client is a `DataLakeServiceClient`.
+
+ Attach this resource definition to a [`JobDefinition`](../jobs.mdx#dagster.JobDefinition) in order to make it
+ available to your ops.
+
+ Example:
+
+ ```python
+ from dagster import job, op
+ from dagster_azure.adls2 import adls2_resource
+
+ @op(required_resource_keys={'adls2'})
+ def example_adls2_op(context):
+ return list(context.resources.adls2.adls2_client.list_file_systems())
+
+ @job(resource_defs={"adls2": adls2_resource})
+ def my_job():
+ example_adls2_op()
+ ```
+ Note that your ops must also declare that they require this resource with
+ required_resource_keys, or it will not be initialized for the execution of their compute
+ functions.
+
+ You may pass credentials to this resource using either a SAS token, a key or by passing the
+ DefaultAzureCredential object.
+
+ ```YAML
+ resources:
+ adls2:
+ config:
+ storage_account: my_storage_account
+ # str: The storage account name.
+ credential:
+ sas: my_sas_token
+ # str: the SAS token for the account.
+ key:
+ env: AZURE_DATA_LAKE_STORAGE_KEY
+ # str: The shared access key for the account.
+ DefaultAzureCredential: {}
+ # dict: The keyword arguments used for DefaultAzureCredential
+ # or leave the object empty for no arguments
+ DefaultAzureCredential:
+ exclude_environment_credential: true
+ ```
+
+
+
+ Persistent IO manager using Azure Data Lake Storage Gen2 for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for ADLS and the backing
+ container.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at “\/\”. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of “/my/base/path”, an asset with key
+ AssetKey([“one”, “two”, “three”]) would be stored in a file called “three” in a directory
+ with path “/my/base/path/one/two/”.
+
+ Example usage:
+
+ Attach this IO manager to a set of assets.
+
+ ```python
+ from dagster import Definitions, asset
+ from dagster_azure.adls2 import adls2_pickle_io_manager, adls2_resource
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return df[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": adls2_pickle_io_manager.configured(
+ {"adls2_file_system": "my-cool-fs", "adls2_prefix": "my-cool-prefix"}
+ ),
+ "adls2": adls2_resource,
+ },
+ )
+ ```
+ Attach this IO manager to your job to make it available to your ops.
+
+ ```python
+ from dagster import job
+ from dagster_azure.adls2 import adls2_pickle_io_manager, adls2_resource
+
+ @job(
+ resource_defs={
+ "io_manager": adls2_pickle_io_manager.configured(
+ {"adls2_file_system": "my-cool-fs", "adls2_prefix": "my-cool-prefix"}
+ ),
+ "adls2": adls2_resource,
+ },
+ )
+ def my_job():
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-celery-docker.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-celery-docker.mdx
new file mode 100644
index 0000000000000..2c29c99f768f5
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-celery-docker.mdx
@@ -0,0 +1,88 @@
+---
+title: 'orchestration on celery + docker'
+title_meta: 'orchestration on celery + docker API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'orchestration on celery + docker Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+ Celery-based executor which launches tasks in docker containers.
+
+ The Celery executor exposes config settings for the underlying Celery app under
+ the `config_source` key. This config corresponds to the “new lowercase settings” introduced
+ in Celery version 4.0 and the object constructed from config will be passed to the
+ `celery.Celery` constructor as its `config_source` argument.
+ (See [https://docs.celeryq.dev/en/stable/userguide/configuration.html](https://docs.celeryq.dev/en/stable/userguide/configuration.html) for details.)
+
+ The executor also exposes the `broker`, backend, and `include` arguments to the
+ `celery.Celery` constructor.
+
+ In the most common case, you may want to modify the `broker` and `backend` (e.g., to use
+ Redis instead of RabbitMQ). We expect that `config_source` will be less frequently
+ modified, but that when op executions are especially fast or slow, or when there are
+ different requirements around idempotence or retry, it may make sense to execute jobs
+ with variations on these settings.
+
+ To use the celery_docker_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster import job
+ from dagster_celery_docker.executor import celery_docker_executor
+
+ @job(executor_def=celery_docker_executor)
+ def celery_enabled_job():
+ pass
+ ```
+ Then you can configure the executor as follows:
+
+ ```YAML
+ execution:
+ config:
+ docker:
+ image: 'my_repo.com/image_name:latest'
+ registry:
+ url: 'my_repo.com'
+ username: 'my_user'
+ password: {env: 'DOCKER_PASSWORD'}
+ env_vars: ["DAGSTER_HOME"] # environment vars to pass from celery worker to docker
+ container_kwargs: # keyword args to be passed to the container. example:
+ volumes: ['/home/user1/:/mnt/vol2','/var/www:/mnt/vol1']
+
+ broker: 'pyamqp://guest@localhost//' # Optional[str]: The URL of the Celery broker
+ backend: 'rpc://' # Optional[str]: The URL of the Celery results backend
+ include: ['my_module'] # Optional[List[str]]: Modules every worker should import
+ config_source: # Dict[str, Any]: Any additional parameters to pass to the
+ #... # Celery workers. This dict will be passed as the `config_source`
+ #... # argument of celery.Celery().
+ ```
+ Note that the YAML you provide here must align with the configuration with which the Celery
+ workers on which you hope to run were started. If, for example, you point the executor at a
+ different broker than the one your workers are listening to, the workers will never be able to
+ pick up tasks for execution.
+
+ In deployments where the celery_docker_job_executor is used all appropriate celery and dagster_celery
+ commands must be invoked with the -A dagster_celery_docker.app argument.
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-celery-k8s.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-celery-k8s.mdx
new file mode 100644
index 0000000000000..ef1f37ca2462e
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-celery-k8s.mdx
@@ -0,0 +1,120 @@
+---
+title: 'orchestration on celery + kubernetes'
+title_meta: 'orchestration on celery + kubernetes API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'orchestration on celery + kubernetes Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+ In contrast to the `K8sRunLauncher`, which launches dagster runs as single K8s
+ Jobs, this run launcher is intended for use in concert with
+ [`dagster_celery_k8s.celery_k8s_job_executor()`](#dagster_celery_k8s.celery_k8s_job_executor).
+
+ With this run launcher, execution is delegated to:
+
+ >
+
+ 1. A run worker Kubernetes Job, which traverses the dagster run execution plan and
+ submits steps to Celery queues for execution;
+ 2. The step executions which are submitted to Celery queues are picked up by Celery workers,
+ and each step execution spawns a step execution Kubernetes Job. See the implementation
+ defined in `dagster_celery_k8.executor.create_k8s_job_task()`.
+
+
+You can configure a Dagster instance to use this RunLauncher by adding a section to your
+`dagster.yaml` like the following:
+
+ ```yaml
+ run_launcher:
+ module: dagster_k8s.launcher
+ class: CeleryK8sRunLauncher
+ config:
+ instance_config_map: "dagster-k8s-instance-config-map"
+ dagster_home: "/some/path"
+ postgres_password_secret: "dagster-k8s-pg-password"
+ broker: "some_celery_broker_url"
+ backend: "some_celery_backend_url"
+ ```
+
+
+
+ Celery-based executor which launches tasks as Kubernetes Jobs.
+
+ The Celery executor exposes config settings for the underlying Celery app under
+ the `config_source` key. This config corresponds to the “new lowercase settings” introduced
+ in Celery version 4.0 and the object constructed from config will be passed to the
+ `celery.Celery` constructor as its `config_source` argument.
+ (See [https://docs.celeryq.dev/en/stable/userguide/configuration.html](https://docs.celeryq.dev/en/stable/userguide/configuration.html) for details.)
+
+ The executor also exposes the `broker`, backend, and `include` arguments to the
+ `celery.Celery` constructor.
+
+ In the most common case, you may want to modify the `broker` and `backend` (e.g., to use
+ Redis instead of RabbitMQ). We expect that `config_source` will be less frequently
+ modified, but that when op executions are especially fast or slow, or when there are
+ different requirements around idempotence or retry, it may make sense to execute dagster jobs
+ with variations on these settings.
+
+ To use the celery_k8s_job_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster import job
+ from dagster_celery_k8s.executor import celery_k8s_job_executor
+
+
+ @job(executor_def=celery_k8s_job_executor)
+ def celery_enabled_job():
+ pass
+ ```
+ Then you can configure the executor as follows:
+
+ ```YAML
+ execution:
+ config:
+ job_image: 'my_repo.com/image_name:latest'
+ job_namespace: 'some-namespace'
+ broker: 'pyamqp://guest@localhost//' # Optional[str]: The URL of the Celery broker
+ backend: 'rpc://' # Optional[str]: The URL of the Celery results backend
+ include: ['my_module'] # Optional[List[str]]: Modules every worker should import
+ config_source: # Dict[str, Any]: Any additional parameters to pass to the
+ #... # Celery workers. This dict will be passed as the `config_source`
+ #... # argument of celery.Celery().
+ ```
+ Note that the YAML you provide here must align with the configuration with which the Celery
+ workers on which you hope to run were started. If, for example, you point the executor at a
+ different broker than the one your workers are listening to, the workers will never be able to
+ pick up tasks for execution.
+
+ In deployments where the celery_k8s_job_executor is used all appropriate celery and dagster_celery
+ commands must be invoked with the -A dagster_celery_k8s.app argument.
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-celery.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-celery.mdx
new file mode 100644
index 0000000000000..2f02160b1a903
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-celery.mdx
@@ -0,0 +1,290 @@
+---
+title: 'celery (dagster-celery)'
+title_meta: 'celery (dagster-celery) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'celery (dagster-celery) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Celery (dagster-celery)
+
+
+
+
+## Quickstart
+
+To get a local rabbitmq broker started and available via the default
+`pyamqp://guest@localhost:5672`, in the `dagster/python_modules/libraries/dagster-celery/`
+directory run:
+
+ ```bash
+ docker-compose up
+ ```
+To run a celery worker:
+
+ ```bash
+ celery -A dagster_celery.app worker -l info
+ ```
+To start multiple workers in the background, run:
+
+ ```bash
+ celery multi start w2 -A dagster_celery.app -l info
+ ```
+To execute a job using the celery-backed executor, you’ll need to set the job’s `executor_def` to
+the celery_executor.
+
+ ```python
+ from dagster import job
+ from dagster_celery import celery_executor
+
+ @job(executor_def=celery_executor)
+ def my_job():
+ pass
+ ```
+
+
+
+### Monitoring your Celery tasks
+
+We advise using [Flower](https://celery.readthedocs.io/en/latest/userguide/monitoring.html#flower-real-time-celery-web-monitor):
+
+ ```bash
+ celery -A dagster_celery.app flower
+ ```
+
+
+
+
+
+
+### Customizing the Celery broker, backend, and other app configuration
+
+By default this will use `amqp://guest:**@localhost:5672//` as the Celery broker URL and
+`rpc://` as the results backend. In production, you will want to change these values. Pending the
+introduction of a dagster_celery CLI, that would entail writing a Python module `my_module` as
+follows:
+
+ ```python
+ from celery import Celery
+
+ from dagster_celery.tasks import create_task
+
+ app = Celery('dagster', broker_url='some://custom@value', ...)
+
+ execute_plan = create_task(app)
+
+ if __name__ == '__main__':
+ app.worker_main()
+ ```
+You can then run the celery worker using:
+
+ ```bash
+ celery -A my_module worker --loglevel=info
+ ```
+This customization mechanism is used to implement dagster_celery_k8s and dagster_celery_k8s which delegate the execution of steps to ephemeral kubernetes pods and docker containers, respectively.
+
+
+
+
+
+
+
+## API
+
+
+
dagster_celery.celery_executor ExecutorDefinition
+
+
+
+
+ Celery-based executor.
+
+ The Celery executor exposes config settings for the underlying Celery app under
+ the `config_source` key. This config corresponds to the “new lowercase settings” introduced
+ in Celery version 4.0 and the object constructed from config will be passed to the
+ `celery.Celery` constructor as its `config_source` argument.
+ (See [https://docs.celeryq.dev/en/stable/userguide/configuration.html](https://docs.celeryq.dev/en/stable/userguide/configuration.html) for details.)
+
+ The executor also exposes the `broker`, backend, and `include` arguments to the
+ `celery.Celery` constructor.
+
+ In the most common case, you may want to modify the `broker` and `backend` (e.g., to use
+ Redis instead of RabbitMQ). We expect that `config_source` will be less frequently
+ modified, but that when solid executions are especially fast or slow, or when there are
+ different requirements around idempotence or retry, it may make sense to execute jobs
+ with variations on these settings.
+
+ To use the celery_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster import job
+ from dagster_celery import celery_executor
+
+ @job(executor_def=celery_executor)
+ def celery_enabled_job():
+ pass
+ ```
+ Then you can configure the executor as follows:
+
+ ```YAML
+ execution:
+ config:
+ broker: 'pyamqp://guest@localhost//' # Optional[str]: The URL of the Celery broker
+ backend: 'rpc://' # Optional[str]: The URL of the Celery results backend
+ include: ['my_module'] # Optional[List[str]]: Modules every worker should import
+ config_source: # Dict[str, Any]: Any additional parameters to pass to the
+ #... # Celery workers. This dict will be passed as the `config_source`
+ #... # argument of celery.Celery().
+ ```
+ Note that the YAML you provide here must align with the configuration with which the Celery
+ workers on which you hope to run were started. If, for example, you point the executor at a
+ different broker than the one your workers are listening to, the workers will never be able to
+ pick up tasks for execution.
+
+
+
+
+
+
+
+
+
+
+
+## CLI
+
+The `dagster-celery` CLI lets you start, monitor, and terminate workers.
+
+
+ The name of the worker. Defaults to a unique name prefixed with “dagster-” and ending with the hostname.
+
+
+
+
+
-y, --config-yaml \
+
+ Specify the path to a config YAML file with options for the worker. This is the same config block that you provide to dagster_celery.celery_executor when configuring a job for execution with Celery, with, e.g., the URL of the broker to use.
+
+
+
+
+
-q, --queue \
+
+ Names of the queues on which this worker should listen for tasks. Provide multiple -q arguments to specify multiple queues. Note that each celery worker may listen on no more than four queues.
+
+
+
+
+
-d, --background
+
+ Set this flag to run the worker in the background.
+
+
+
+
+
-i, --includes \
+
+ Python modules the worker should import. Provide multiple -i arguments to specify multiple modules.
+
+
+
+
+
-l, --loglevel \
+
+ Log level for the worker.
+
+
+
+
+
-A, --app \
+
+
+
+
+
+Arguments:
+
+
+
ADDITIONAL_ARGS
+
+ Optional argument(s)
+
+
+
+
+
+
+
+
+
+### dagster-celery worker list
+
+List running dagster-celery workers. Note that we use the broker to contact the workers.
+
+ ```shell
+ dagster-celery worker list [OPTIONS]
+ ```
+Options:
+
+
+
-y, --config-yaml \
+
+ Specify the path to a config YAML file with options for the workers you are trying to manage. This is the same config block that you provide to dagster_celery.celery_executor when configuring a job for execution with Celery, with, e.g., the URL of the broker to use. Without this config file, you will not be able to find your workers (since the CLI won’t know how to reach the broker).
+
+
+
+
+
+
+
+
+### dagster-celery worker terminate
+
+Shut down dagster-celery workers. Note that we use the broker to send signals to the workers to terminate – if the broker is not running, this command is a no-op. Provide the argument NAME to terminate a specific worker by name.
+
+ ```shell
+ dagster-celery worker terminate [OPTIONS] [NAME]
+ ```
+Options:
+
+
+
-a, --all
+
+ Set this flag to terminate all running workers.
+
+
+
+
+
-y, --config-yaml \
+
+ Specify the path to a config YAML file with options for the workers you are trying to manage. This is the same config block that you provide to dagster_celery.celery_executor when configuring a job for execution with Celery, with, e.g., the URL of the broker to use. Without this config file, you will not be able to terminate your workers (since the CLI won’t know how to reach the broker).
+
+
+
+Arguments:
+
+
+
NAME
+
+ Optional argument
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-census.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-census.mdx
new file mode 100644
index 0000000000000..9bd967bd14f25
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-census.mdx
@@ -0,0 +1,132 @@
+---
+title: 'census (dagster-census)'
+title_meta: 'census (dagster-census) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'census (dagster-census) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Census (dagster-census)
+
+This library provides an integration with Census.
+
+
+
+ Executes a Census sync for a given `sync_id` and polls until that sync completes, raising
+ an error if it is unsuccessful.
+
+ It outputs a [`CensusOutput`](#dagster_census.CensusOutput) which contains the details of the Census
+ sync after it successfully completes.
+
+ It requires the use of the [`census_resource`](#dagster_census.census_resource), which allows it to
+ communicate with the Census API.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_census import census_resource, census_sync_op
+
+ my_census_resource = census_resource.configured(
+ {
+ "api_key": {"env": "CENSUS_API_KEY"},
+ }
+ )
+
+ sync_foobar = census_sync_op.configured({"sync_id": "foobar"}, name="sync_foobar")
+
+ @job(resource_defs={"census": my_census_resource})
+ def my_simple_census_job():
+ sync_foobar()
+ ```
+
+
+
+
+
+
dagster_census.census_resource ResourceDefinition
+
+
+
+
+ This resource allows users to programatically interface with the Census REST API to launch
+ syncs and monitor their progress. This currently implements only a subset of the functionality
+ exposed by the API.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_census import census_resource
+
+ my_census_resource = census_resource.configured(
+ {
+ "api_key": {"env": "CENSUS_API_KEY"},
+ }
+ )
+
+ @job(resource_defs={"census":my_census_resource})
+ def my_census_job():
+ ...
+ ```
+
+
+
+
+
+
class dagster_census.CensusResource
+
+ This class exposes methods on top of the Census REST API.
+
+
+
+
+
class dagster_census.CensusOutput
+
+
+ Contains recorded information about the state of a Census sync after a sync completes.
+
+
+
sync_run
+
+
+ The details of the specific sync run.
+
+ Type: Dict[str, Any]
+
+
+
+
+
+
source
+
+
+ Information about the source for the Census sync.
+
+ Type: Dict[str, Any]
+
+
+
+
+
+
destination
+
+
+ Information about the destination for the Census sync.
+
+ Type: Dict[str, Any]
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-dask.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-dask.mdx
new file mode 100644
index 0000000000000..a30381cfe6b6c
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-dask.mdx
@@ -0,0 +1,57 @@
+---
+title: 'dask (dagster-dask)'
+title_meta: 'dask (dagster-dask) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dask (dagster-dask) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Dask (dagster-dask)
+
+See also the [Dask deployment guide](https://docs.dagster.io/deployment/guides/dask/).
+
+
+
dagster_dask.dask_executor ExecutorDefinition
+
+
+
+
+ Dask-based executor.
+
+ The ‘cluster’ can be one of the following:
+ (‘existing’, ‘local’, ‘yarn’, ‘ssh’, ‘pbs’, ‘moab’, ‘sge’, ‘lsf’, ‘slurm’, ‘oar’, ‘kube’).
+
+ If the Dask executor is used without providing executor-specific config, a local Dask cluster
+ will be created (as when calling `dask.distributed.Client()`
+ with `dask.distributed.LocalCluster()`).
+
+ The Dask executor optionally takes the following config:
+
+ ```none
+ cluster:
+ {
+ local?: # takes distributed.LocalCluster parameters
+ {
+ timeout?: 5, # Timeout duration for initial connection to the scheduler
+ n_workers?: 4 # Number of workers to start
+ threads_per_worker?: 1 # Number of threads per each worker
+ }
+ }
+ ```
+ To use the dask_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster import job
+ from dagster_dask import dask_executor
+
+ @job(executor_def=dask_executor)
+ def dask_enabled_job():
+ pass
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-databricks.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-databricks.mdx
new file mode 100644
index 0000000000000..73e55601e0b95
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-databricks.mdx
@@ -0,0 +1,359 @@
+---
+title: 'databricks (dagster-databricks)'
+title_meta: 'databricks (dagster-databricks) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'databricks (dagster-databricks) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Databricks (dagster-databricks)
+
+The `dagster_databricks` package provides these main pieces of functionality:
+
+ - A resource, `databricks_pyspark_step_launcher`, which will execute a op within a Databricks
+ - An op factory, `create_databricks_run_now_op`, which creates an op that launches an existing
+ - A op factory, `create_databricks_submit_run_op`, which creates an op that submits a one-time run
+
+
+Note that, for the `databricks_pyspark_step_launcher`, either S3 or Azure Data Lake Storage config
+must be specified for ops to succeed, and the credentials for this storage must also be
+stored as a Databricks Secret and stored in the resource config so that the Databricks cluster can
+access storage.
+
+
+
+ Resource which provides a Python client for interacting with Databricks within an
+ op or asset.
+
+
+
+
+
+
+
class dagster_databricks.DatabricksClient
+
+
+ A thin wrapper over the Databricks REST API.
+
+
+
property workspace_client
+
+
+ Retrieve a reference to the underlying Databricks Workspace client. For more information,
+ see the [Databricks SDK for Python](https://docs.databricks.com/dev-tools/sdk-python.html).
+
+ Examples:
+
+ ```python
+ from dagster import op
+ from databricks.sdk import WorkspaceClient
+
+ @op(required_resource_keys={"databricks_client"})
+ def op1(context):
+ # Initialize the Databricks Jobs API
+ client = context.resources.databricks_client.api_client
+
+ # Example 1: Run a Databricks job with some parameters.
+ client.jobs.run_now(...)
+
+ # Example 2: Trigger a one-time run of a Databricks workload.
+ client.jobs.submit(...)
+
+ # Example 3: Get an existing run.
+ client.jobs.get_run(...)
+
+ # Example 4: Cancel a run.
+ client.jobs.cancel_run(...)
+ ```
+ Returns: The authenticated Databricks SDK Workspace Client.Return type: WorkspaceClient
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Ops
+
+
+
dagster_databricks.create_databricks_run_now_op
+
+
+ Creates an op that launches an existing databricks job.
+
+ As config, the op accepts a blob of the form described in Databricks’ Job API:
+ [https://docs.databricks.com/api/workspace/jobs/runnow](https://docs.databricks.com/api/workspace/jobs/runnow). The only required field is
+ `job_id`, which is the ID of the job to be executed. Additional fields can be used to specify
+ override parameters for the Databricks Job.
+
+ Parameters:
+ - databricks_job_id (int) – The ID of the Databricks Job to be executed.
+ - databricks_job_configuration (dict) – Configuration for triggering a new job run of a
+ - poll_interval_seconds (float) – How often to poll the Databricks API to check whether the
+ - max_wait_time_seconds (float) – How long to wait for the Databricks job to finish running
+ - name (Optional[str]) – The name of the op. If not provided, the name will be
+ - databricks_resource_key (str) – The name of the resource key used by this op. If not
+
+
+ Returns: An op definition to run the Databricks Job.Return type: [OpDefinition](../ops.mdx#dagster.OpDefinition)
+ Example:
+
+ ```python
+ from dagster import job
+ from dagster_databricks import create_databricks_run_now_op, DatabricksClientResource
+
+ DATABRICKS_JOB_ID = 1234
+
+
+ run_now_op = create_databricks_run_now_op(
+ databricks_job_id=DATABRICKS_JOB_ID,
+ databricks_job_configuration={
+ "python_params": [
+ "--input",
+ "schema.db.input_table",
+ "--output",
+ "schema.db.output_table",
+ ],
+ },
+ )
+
+ @job(
+ resource_defs={
+ "databricks": DatabricksClientResource(
+ host=EnvVar("DATABRICKS_HOST"),
+ token=EnvVar("DATABRICKS_TOKEN")
+ )
+ }
+ )
+ def do_stuff():
+ run_now_op()
+ ```
+
+
+
+ Creates an op that submits a one-time run of a set of tasks on Databricks.
+
+ As config, the op accepts a blob of the form described in Databricks’ Job API:
+ [https://docs.databricks.com/api/workspace/jobs/submit](https://docs.databricks.com/api/workspace/jobs/submit).
+
+ Parameters:
+ - databricks_job_configuration (dict) – Configuration for submitting a one-time run of a set
+ - poll_interval_seconds (float) – How often to poll the Databricks API to check whether the
+ - max_wait_time_seconds (float) – How long to wait for the Databricks job to finish running
+ - name (Optional[str]) – The name of the op. If not provided, the name will be
+ - databricks_resource_key (str) – The name of the resource key used by this op. If not
+
+
+ Returns: An op definition to submit a one-time run of a set of tasks on Databricks.Return type: [OpDefinition](../ops.mdx#dagster.OpDefinition)
+ Example:
+
+ ```python
+ from dagster import job
+ from dagster_databricks import create_databricks_submit_run_op, DatabricksClientResource
+
+
+ submit_run_op = create_databricks_submit_run_op(
+ databricks_job_configuration={
+ "new_cluster": {
+ "spark_version": '2.1.0-db3-scala2.11',
+ "num_workers": 2
+ },
+ "notebook_task": {
+ "notebook_path": "/Users/dagster@example.com/PrepareData",
+ },
+ }
+ )
+
+ @job(
+ resource_defs={
+ "databricks": DatabricksClientResource(
+ host=EnvVar("DATABRICKS_HOST"),
+ token=EnvVar("DATABRICKS_TOKEN")
+ )
+ }
+ )
+ def do_stuff():
+ submit_run_op()
+ ```
+
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ While there is no plan to remove this functionality, for new projects, we recommend using Dagster Pipes. For more information, see https://docs.dagster.io/guides/build/external-pipelines/.
+
+ :::
+
+ Resource for running ops as a Databricks Job.
+
+ When this resource is used, the op will be executed in Databricks using the ‘Run Submit’
+ API. Pipeline code will be zipped up and copied to a directory in DBFS along with the op’s
+ execution context.
+
+ Use the ‘run_config’ configuration to specify the details of the Databricks cluster used, and
+ the ‘storage’ key to configure persistent storage on that cluster. Storage is accessed by
+ setting the credentials in the Spark context, as documented [here for S3](https://docs.databricks.com/data/data-sources/aws/amazon-s3.html#alternative-1-set-aws-keys-in-the-spark-context) and [here for ADLS](https://docs.microsoft.com/en-gb/azure/databricks/data/data-sources/azure/azure-datalake-gen2#--access-directly-using-the-storage-account-access-key).
+
+
+
+
+
+
+
+
+
+
+
+## Pipes
+
+
+
class dagster_databricks.PipesDatabricksClient
+
+
+ Pipes client for databricks.
+
+ Parameters:
+ - client (WorkspaceClient) – A databricks WorkspaceClient object.
+ - (Optional[Mapping[str (env) – An optional dict of environment
+ - str]] – An optional dict of environment
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - poll_interval_seconds (float) – How long to sleep between checking the status of the job run.
+ - forward_termination (bool) – Whether to cancel the Databricks job if the orchestration process
+
+
+
+
+
+
+
+
class dagster_databricks.PipesDbfsContextInjector
+
+
+ A context injector that injects context into a Databricks job by writing a JSON file to DBFS.
+
+ Parameters: client (WorkspaceClient) – A databricks WorkspaceClient object.
+
+
+
+
+
+
class dagster_databricks.PipesDbfsMessageReader
+
+
+ Message reader that reads messages by periodically reading message chunks from an
+ automatically-generated temporary directory on DBFS.
+
+ If log_readers is passed, this reader will also start the passed readers
+ when the first message is received from the external process.
+
+ Parameters:
+ - interval (float) – interval in seconds between attempts to download a chunk
+ - client (WorkspaceClient) – A databricks WorkspaceClient object.
+ - cluster_log_root (Optional[str]) – The root path on DBFS where the cluster logs are written.
+ - include_stdio_in_messages (bool) – Whether to send stdout/stderr to Dagster via Pipes messages. Defaults to False.
+ - log_readers (Optional[Sequence[PipesLogReader]]) – A set of log readers for logs on DBFS.
+
+
+
+
+
+
+
+
class dagster_databricks.PipesDbfsLogReader
+
+
+ Reader that reads a log file from DBFS.
+
+ Parameters:
+ - interval (float) – interval in seconds between attempts to download a log chunk
+ - remote_log_name (Literal["stdout", "stderr"]) – The name of the log file to read.
+ - target_stream (TextIO) – The stream to which to forward log chunks that have been read.
+ - client (WorkspaceClient) – A databricks WorkspaceClient object.
+ - debug_info (Optional[str]) – An optional message containing debug information about the log reader.
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-datadog.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-datadog.mdx
new file mode 100644
index 0000000000000..0f71e87fd3fcf
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-datadog.mdx
@@ -0,0 +1,126 @@
+---
+title: 'datadog (dagster-datadog)'
+title_meta: 'datadog (dagster-datadog) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'datadog (dagster-datadog) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Datadog (dagster-datadog)
+
+This library provides an integration with Datadog, to support publishing metrics to Datadog from
+within Dagster ops.
+
+We use the Python [datadogpy](https://github.com/DataDog/datadogpy) library. To use it, you’ll
+first need to create a DataDog account and get both [API and Application keys](https://docs.datadoghq.com/account_management/api-app-keys/).
+
+The integration uses [DogStatsD](https://docs.datadoghq.com/developers/dogstatsd/), so you’ll need
+to ensure the datadog agent is running on the host you’re sending metrics from.
+
+
+
+ This legacy resource is a thin wrapper over the
+ [dogstatsd library](https://datadogpy.readthedocs.io/en/latest/).
+
+ Prefer using [`DatadogResource`](#dagster_datadog.DatadogResource).
+
+ As such, we directly mirror the public API methods of DogStatsd here; you can refer to the
+ [DataDog documentation](https://docs.datadoghq.com/developers/dogstatsd/) for how to use this
+ resource.
+
+ Examples:
+
+ ```python
+ @op(required_resource_keys={'datadog'})
+ def datadog_op(context):
+ dd = context.resources.datadog
+
+ dd.event('Man down!', 'This server needs assistance.')
+ dd.gauge('users.online', 1001, tags=["protocol:http"])
+ dd.increment('page.views')
+ dd.decrement('page.views')
+ dd.histogram('album.photo.count', 26, tags=["gender:female"])
+ dd.distribution('album.photo.count', 26, tags=["color:blue"])
+ dd.set('visitors.uniques', 999, tags=["browser:ie"])
+ dd.service_check('svc.check_name', dd.WARNING)
+ dd.timing("query.response.time", 1234)
+
+ # Use timed decorator
+ @dd.timed('run_fn')
+ def run_fn():
+ pass
+
+ run_fn()
+
+ @job(resource_defs={'datadog': datadog_resource})
+ def dd_job():
+ datadog_op()
+
+ result = dd_job.execute_in_process(
+ run_config={'resources': {'datadog': {'config': {'api_key': 'YOUR_KEY', 'app_key': 'YOUR_KEY'}}}}
+ )
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-datahub.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-datahub.mdx
new file mode 100644
index 0000000000000..cd9ca3a41ea4e
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-datahub.mdx
@@ -0,0 +1,172 @@
+---
+title: 'datahub (dagster-datahub)'
+title_meta: 'datahub (dagster-datahub) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'datahub (dagster-datahub) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Datahub (dagster-datahub)
+
+This library provides an integration with Datahub, to support pushing metadata to Datahub from
+within Dagster ops.
+
+
+
+We use the [Datahub Python Library](https://github.com/datahub-project/datahub). To use it, you’ll
+first need to start up a Datahub Instance. [Datahub Quickstart Guide](https://datahubproject.io/docs/quickstart).
+
+
+
+ Base class for Dagster resources that utilize structured config.
+
+ This class is a subclass of both `ResourceDefinition` and `Config`.
+
+ Example definition:
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ prefix: str
+
+ def output(self, text: str) -> None:
+ print(f"{self.prefix}{text}")
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def asset_that_uses_writer(writer: WriterResource):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[asset_that_uses_writer],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+ You can optionally use this class to model configuration only and vend an object
+ of a different type for use at runtime. This is useful for those who wish to
+ have a separate object that manages configuration and a separate object at runtime. Or
+ where you want to directly use a third-party class that you do not control.
+
+ To do this you override the create_resource methods to return a different object.
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ str: prefix
+
+ def create_resource(self, context: InitResourceContext) -> Writer:
+ # Writer is pre-existing class defined else
+ return Writer(self.prefix)
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def use_preexisting_writer_as_resource(writer: ResourceParam[Writer]):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[use_preexisting_writer_as_resource],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+
+
+
+ Base class for Dagster resources that utilize structured config.
+
+ This class is a subclass of both `ResourceDefinition` and `Config`.
+
+ Example definition:
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ prefix: str
+
+ def output(self, text: str) -> None:
+ print(f"{self.prefix}{text}")
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def asset_that_uses_writer(writer: WriterResource):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[asset_that_uses_writer],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+ You can optionally use this class to model configuration only and vend an object
+ of a different type for use at runtime. This is useful for those who wish to
+ have a separate object that manages configuration and a separate object at runtime. Or
+ where you want to directly use a third-party class that you do not control.
+
+ To do this you override the create_resource methods to return a different object.
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ str: prefix
+
+ def create_resource(self, context: InitResourceContext) -> Writer:
+ # Writer is pre-existing class defined else
+ return Writer(self.prefix)
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def use_preexisting_writer_as_resource(writer: ResourceParam[Writer]):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[use_preexisting_writer_as_resource],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-dbt.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-dbt.mdx
new file mode 100644
index 0000000000000..d56591a47b60e
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-dbt.mdx
@@ -0,0 +1,2058 @@
+---
+title: 'dbt (dagster-dbt)'
+title_meta: 'dbt (dagster-dbt) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dbt (dagster-dbt) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# dbt (dagster-dbt)
+
+Dagster orchestrates dbt alongside other technologies, so you can combine dbt with Spark, Python,
+etc. in a single workflow. Dagster’s software-defined asset abstractions make it simple to define
+data assets that depend on specific dbt models, or to define the computation required to compute
+the sources that your dbt models depend on.
+
+Related documentation pages: [dbt](https://docs.dagster.io/integrations/libraries/dbt/) and
+[dbt Cloud](https://docs.dagster.io/integrations/dbt-cloud).
+
+
+
+
+## dagster-dbt
+
+
+
+
+### dagster-dbt project
+
+Commands for using a dbt project in Dagster.
+
+ ```shell
+ dagster-dbt project [OPTIONS] COMMAND [ARGS]...
+ ```
+
+
+
+#### prepare-and-package
+
+This command will invoke `prepare_and_package` on [`DbtProject`](#dagster_dbt.DbtProject) found in the target module or file.
+Note that this command runs dbt deps and dbt parse.
+
+ ```shell
+ dagster-dbt project prepare-and-package [OPTIONS]
+ ```
+Options:
+
+
+
--file \
+
+ Required The file containing DbtProject definitions to prepare.
+
+
+
+
+
+
+
+
+
+#### scaffold
+
+This command will initialize a new Dagster project and create directories and files that
+load assets from an existing dbt project.
+
+ ```shell
+ dagster-dbt project scaffold [OPTIONS]
+ ```
+Options:
+
+
+
--project-name \
+
+ Required The name of the Dagster project to initialize for your dbt project.
+
+
+
+
+
--dbt-project-dir \
+
+ The path of your dbt project directory. This path must contain a dbt_project.yml file. By default, this command will assume that the current working directory contains a dbt project, but you can set a different directory by setting this option.
+
+
+
+
+
+
+
+
+
+## dbt Core
+
+Here, we provide interfaces to manage dbt projects invoked by the local dbt command line interface
+(dbt CLI).
+
+
+
+
+### Assets (dbt Core)
+
+
+
@dagster_dbt.dbt_assets
+
+
+ Create a definition for how to compute a set of dbt resources, described by a manifest.json.
+ When invoking dbt commands using [`DbtCliResource`](#dagster_dbt.DbtCliResource)’s
+ [`cli()`](#dagster_dbt.DbtCliResource.cli) method, Dagster events are emitted by calling
+ `yield from` on the event stream returned by [`stream()`](#dagster_dbt.DbtCliInvocation.stream).
+
+ Parameters:
+ - manifest (Union[Mapping[str, Any], str, Path]) – The contents of a manifest.json file
+ - select (str) – A dbt selection string for the models in a project that you want
+ - exclude (Optional[str]) – A dbt selection string for the models in a project that you want
+ - name (Optional[str]) – The name of the op.
+ - io_manager_key (Optional[str]) – The IO manager key that will be set on each of the returned
+ - partitions_def (Optional[[*PartitionsDefinition*](../partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - dagster_dbt_translator (Optional[[*DagsterDbtTranslator*](#dagster_dbt.DagsterDbtTranslator)]) – Allows customizing how to map
+ - backfill_policy (Optional[[*BackfillPolicy*](../partitions.mdx#dagster.BackfillPolicy)]) – If a partitions_def is defined, this determines
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the assets.
+ - required_resource_keys (Optional[Set[str]]) – Set of required resource handles.
+ - project (Optional[[*DbtProject*](#dagster_dbt.DbtProject)]) – A DbtProject instance which provides a pointer to the dbt
+ - retry_policy (Optional[[*RetryPolicy*](../ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that computes the asset.
+
+
+ Examples:
+
+ Running `dbt build` for a dbt project:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ yield from dbt.cli(["build"], context=context).stream()
+ ```
+ Running dbt commands with flags:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ yield from dbt.cli(["build", "--full-refresh"], context=context).stream()
+ ```
+ Running dbt commands with `--vars`:
+
+ ```python
+ import json
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_vars = {"key": "value"}
+
+ yield from dbt.cli(["build", "--vars", json.dumps(dbt_vars)], context=context).stream()
+ ```
+ Retrieving dbt artifacts after running a dbt command:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_build_invocation = dbt.cli(["build"], context=context)
+
+ yield from dbt_build_invocation.stream()
+
+ run_results_json = dbt_build_invocation.get_artifact("run_results.json")
+ ```
+ Running multiple dbt commands for a dbt project:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ yield from dbt.cli(["run"], context=context).stream()
+ yield from dbt.cli(["test"], context=context).stream()
+ ```
+ Accessing the dbt event stream alongside the Dagster event stream:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_cli_invocation = dbt.cli(["build"], context=context)
+
+ # Each dbt event is structured: https://docs.getdbt.com/reference/events-logging
+ for dbt_event in dbt_invocation.stream_raw_events():
+ for dagster_event in dbt_event.to_default_asset_events(
+ manifest=dbt_invocation.manifest,
+ dagster_dbt_translator=dbt_invocation.dagster_dbt_translator,
+ context=dbt_invocation.context,
+ target_path=dbt_invocation.target_path,
+ ):
+ # Manipulate `dbt_event`
+ ...
+
+ # Then yield the Dagster event
+ yield dagster_event
+ ```
+ Customizing the Dagster asset definition metadata inferred from a dbt project using [`DagsterDbtTranslator`](#dagster_dbt.DagsterDbtTranslator):
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DagsterDbtTranslator, DbtCliResource, dbt_assets
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ ...
+
+
+ @dbt_assets(
+ manifest=Path("target", "manifest.json"),
+ dagster_dbt_translator=CustomDagsterDbtTranslator(),
+ )
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ yield from dbt.cli(["build"], context=context).stream()
+ ```
+ Using a custom resource key for dbt:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, my_custom_dbt_resource_key: DbtCliResource):
+ yield from my_custom_dbt_resource_key.cli(["build"], context=context).stream()
+ ```
+ Using a dynamically generated resource key for dbt using required_resource_keys:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ dbt_resource_key = "my_custom_dbt_resource_key"
+
+ @dbt_assets(manifest=Path("target", "manifest.json"), required_resource_keys={my_custom_dbt_resource_key})
+ def my_dbt_assets(context: AssetExecutionContext):
+ dbt = getattr(context.resources, dbt_resource_key)
+ yield from dbt.cli(["build"], context=context).stream()
+ ```
+ Invoking another Dagster [`ResourceDefinition`](../resources.mdx#dagster.ResourceDefinition) alongside dbt:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+ from dagster_slack import SlackResource
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource, slack: SlackResource):
+ yield from dbt.cli(["build"], context=context).stream()
+
+ slack_client = slack.get_client()
+ slack_client.chat_postMessage(channel="#my-channel", text="dbt build succeeded!")
+ ```
+ Defining and accessing Dagster [`Config`](../config.mdx#dagster.Config) alongside dbt:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext, Config
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ class MyDbtConfig(Config):
+ full_refresh: bool
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource, config: MyDbtConfig):
+ dbt_build_args = ["build"]
+ if config.full_refresh:
+ dbt_build_args += ["--full-refresh"]
+
+ yield from dbt.cli(dbt_build_args, context=context).stream()
+ ```
+ Defining Dagster `PartitionDefinition` alongside dbt:
+
+ ```python
+ import json
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext, DailyPartitionDefinition
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(
+ manifest=Path("target", "manifest.json"),
+ partitions_def=DailyPartitionsDefinition(start_date="2023-01-01")
+ )
+ def partitionshop_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ time_window = context.partition_time_window
+
+ dbt_vars = {
+ "min_date": time_window.start.isoformat(),
+ "max_date": time_window.end.isoformat()
+ }
+ dbt_build_args = ["build", "--vars", json.dumps(dbt_vars)]
+
+ yield from dbt.cli(dbt_build_args, context=context).stream()
+ ```
+
+
+
+
+
+
class dagster_dbt.DagsterDbtTranslator
+
+
+ Holds a set of methods that derive Dagster asset definition metadata given a representation
+ of a dbt resource (models, tests, sources, etc).
+
+ This class is exposed so that methods can be overriden to customize how Dagster asset metadata
+ is derived.
+
+
+
get_asset_key
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster asset key that represents that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom asset key for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: The Dagster asset key for the dbt resource.Return type: [AssetKey](../assets.mdx#dagster.AssetKey)
+ Examples:
+
+ Adding a prefix to the default asset key generated for each dbt resource:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster import AssetKey
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_asset_key(self, dbt_resource_props: Mapping[str, Any]) -> AssetKey:
+ return super().get_asset_key(dbt_resource_props).with_prefix("prefix")
+ ```
+ Adding a prefix to the default asset key generated for each dbt resource, but only for dbt sources:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster import AssetKey
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_asset_key(self, dbt_resource_props: Mapping[str, Any]) -> AssetKey:
+ asset_key = super().get_asset_key(dbt_resource_props)
+
+ if dbt_resource_props["resource_type"] == "source":
+ asset_key = asset_key.with_prefix("my_prefix")
+
+ return asset_key
+ ```
+
+
+
+
+
+
get_auto_materialize_policy
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster `dagster.AutoMaterializePolicy` for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom auto-materialize policy for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A Dagster auto-materialize policy.Return type: Optional[AutoMaterializePolicy]
+ Examples:
+
+ Set a custom auto-materialize policy for all dbt resources:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_auto_materialize_policy(self, dbt_resource_props: Mapping[str, Any]) -> Optional[AutoMaterializePolicy]:
+ return AutoMaterializePolicy.eager()
+ ```
+ Set a custom auto-materialize policy for dbt resources with a specific tag:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_auto_materialize_policy(self, dbt_resource_props: Mapping[str, Any]) -> Optional[AutoMaterializePolicy]:
+ auto_materialize_policy = None
+ if "my_custom_tag" in dbt_resource_props.get("tags", []):
+ auto_materialize_policy = AutoMaterializePolicy.eager()
+
+ return auto_materialize_policy
+ ```
+
+
+
+
+
+
get_automation_condition
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster `dagster.AutoMaterializePolicy` for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom AutomationCondition for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A Dagster auto-materialize policy.Return type: Optional[AutoMaterializePolicy]
+ Examples:
+
+ Set a custom AutomationCondition for all dbt resources:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_automation_condition(self, dbt_resource_props: Mapping[str, Any]) -> Optional[AutomationCondition]:
+ return AutomationCondition.eager()
+ ```
+ Set a custom AutomationCondition for dbt resources with a specific tag:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_automation_condition(self, dbt_resource_props: Mapping[str, Any]) -> Optional[AutomationCondition]:
+ automation_condition = None
+ if "my_custom_tag" in dbt_resource_props.get("tags", []):
+ automation_condition = AutomationCondition.eager()
+
+ return automation_condition
+ ```
+
+
+
+
+
+
get_code_version
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster code version for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom code version for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A Dagster code version.Return type: Optional[str]
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_code_version(self, dbt_resource_props: Mapping[str, Any]) -> Optional[str]:
+ return dbt_resource_props["checksum"]["checksum"]
+ ```
+
+
+
+
+
+
get_description
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster description for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom description for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: The description for the dbt resource.Return type: str
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_description(self, dbt_resource_props: Mapping[str, Any]) -> str:
+ return "custom description"
+ ```
+
+
+
+
+
+
get_freshness_policy
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster `dagster.FreshnessPolicy` for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom freshness policy for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A Dagster freshness policy.Return type: Optional[FreshnessPolicy]
+ Examples:
+
+ Set a custom freshness policy for all dbt resources:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_freshness_policy(self, dbt_resource_props: Mapping[str, Any]) -> Optional[FreshnessPolicy]:
+ return FreshnessPolicy(maximum_lag_minutes=60)
+ ```
+ Set a custom freshness policy for dbt resources with a specific tag:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_freshness_policy(self, dbt_resource_props: Mapping[str, Any]) -> Optional[FreshnessPolicy]:
+ freshness_policy = None
+ if "my_custom_tag" in dbt_resource_props.get("tags", []):
+ freshness_policy = FreshnessPolicy(maximum_lag_minutes=60)
+
+ return freshness_policy
+ ```
+
+
+
+
+
+
get_group_name
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster group name for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom group name for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A Dagster group name.Return type: Optional[str]
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_group_name(self, dbt_resource_props: Mapping[str, Any]) -> Optional[str]:
+ return "custom_group_prefix" + dbt_resource_props.get("config", {}).get("group")
+ ```
+
+
+
+
+
+
get_metadata
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster metadata for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom metadata for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A dictionary representing the Dagster metadata for the dbt resource.Return type: Mapping[str, Any]
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_metadata(self, dbt_resource_props: Mapping[str, Any]) -> Mapping[str, Any]:
+ return {"custom": "metadata"}
+ ```
+
+
+
+
+
+
get_owners
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster owners for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide custom owners for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A set of Dagster owners.Return type: Optional[Sequence[str]]
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_owners(self, dbt_resource_props: Mapping[str, Any]) -> Optional[Sequence[str]]:
+ return ["user@owner.com", "team:team@owner.com"]
+ ```
+
+
+
+
+
+
get_partition_mapping
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A function that takes two dictionaries: the first, representing properties of a dbt
+ resource; and the second, representing the properties of a parent dependency to the first
+ dbt resource. The function returns the Dagster partition mapping for the dbt dependency.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ This method can be overridden to provide a custom partition mapping for a dbt dependency.
+
+ Parameters:
+ - dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt child resource.
+ - dbt_parent_resource_props (Mapping[str, Any]) – A dictionary representing the dbt parent resource, in relationship to the child.
+
+
+ Returns: The Dagster partition mapping for the dbt resource. If None is returned, the
+ default partition mapping will be used.Return type: Optional[[PartitionMapping](../partitions.mdx#dagster.PartitionMapping)]
+
+
+
+
+
+
get_tags
+
+
+ A function that takes a dictionary representing properties of a dbt resource, and
+ returns the Dagster tags for that resource.
+
+ Note that a dbt resource is unrelated to Dagster’s resource concept, and simply represents
+ a model, seed, snapshot or source in a given dbt project. You can learn more about dbt
+ resources and the properties available in this dictionary here:
+ [https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details](https://docs.getdbt.com/reference/artifacts/manifest-json#resource-details)
+
+ dbt tags are strings, but Dagster tags are key-value pairs. To bridge this divide, the dbt
+ tag string is used as the Dagster tag key, and the Dagster tag value is set to the empty
+ string, “”.
+
+ Any dbt tags that don’t match Dagster’s supported tag key format (e.g. they contain
+ unsupported characters) will be ignored.
+
+ This method can be overridden to provide custom tags for a dbt resource.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.Returns: A dictionary representing the Dagster tags for the dbt resource.Return type: Mapping[str, str]
+ Examples:
+
+ ```python
+ from typing import Any, Mapping
+
+ from dagster_dbt import DagsterDbtTranslator
+
+
+ class CustomDagsterDbtTranslator(DagsterDbtTranslator):
+ def get_tags(self, dbt_resource_props: Mapping[str, Any]) -> Mapping[str, str]:
+ return {"custom": "tag"}
+ ```
+
+
+
+
+
+
+
+
+
+
class dagster_dbt.DagsterDbtTranslatorSettings
+
+
+ Settings to enable Dagster features for your dbt project.
+
+ Parameters:
+ - enable_asset_checks (bool) – Whether to load dbt tests as Dagster asset checks.
+ - enable_duplicate_source_asset_keys (bool) – Whether to allow dbt sources with duplicate
+ - enable_code_references (bool) – Whether to enable Dagster code references for dbt resources.
+ - enable_dbt_selection_by_name (bool) – Whether to enable selecting dbt resources by name,
+
+
+
+
+
+
+
+
class dagster_dbt.DbtManifestAssetSelection
+
+
+ Defines a selection of assets from a dbt manifest wrapper and a dbt selection string.
+
+ Parameters:
+ - manifest (Mapping[str, Any]) – The dbt manifest blob.
+ - select (str) – A dbt selection string to specify a set of dbt resources.
+ - exclude (Optional[str]) – A dbt selection string to exclude a set of dbt resources.
+
+
+ Examples:
+
+ ```python
+ import json
+ from pathlib import Path
+
+ from dagster_dbt import DbtManifestAssetSelection
+
+ manifest = json.loads(Path("path/to/manifest.json").read_text())
+
+ # select the dbt assets that have the tag "foo".
+ my_selection = DbtManifestAssetSelection(manifest=manifest, select="tag:foo")
+ ```
+
+
+
+
+
+
dagster_dbt.build_dbt_asset_selection
+
+
+ Build an asset selection for a dbt selection string.
+
+ See [https://docs.getdbt.com/reference/node-selection/syntax#how-does-selection-work](https://docs.getdbt.com/reference/node-selection/syntax#how-does-selection-work) for
+ more information.
+
+ Parameters:
+ - dbt_select (str) – A dbt selection string to specify a set of dbt resources.
+ - dbt_exclude (Optional[str]) – A dbt selection string to exclude a set of dbt resources.
+
+
+ Returns: An asset selection for the selected dbt nodes.Return type: [AssetSelection](../assets.mdx#dagster.AssetSelection)
+ Examples:
+
+ ```python
+ from dagster_dbt import dbt_assets, build_dbt_asset_selection
+
+ @dbt_assets(manifest=...)
+ def all_dbt_assets():
+ ...
+
+ # Select the dbt assets that have the tag "foo".
+ foo_selection = build_dbt_asset_selection([dbt_assets], dbt_select="tag:foo")
+
+ # Select the dbt assets that have the tag "foo" and all Dagster assets downstream
+ # of them (dbt-related or otherwise)
+ foo_and_downstream_selection = foo_selection.downstream()
+ ```
+ Building an asset selection on a dbt assets definition with an existing selection:
+
+ ```python
+ from dagster_dbt import dbt_assets, build_dbt_asset_selection
+
+ @dbt_assets(
+ manifest=...
+ select="bar+",
+ )
+ def bar_plus_dbt_assets():
+ ...
+
+ # Select the dbt assets that are in the intersection of having the tag "foo" and being
+ # in the existing selection "bar+".
+ bar_plus_and_foo_selection = build_dbt_asset_selection(
+ [bar_plus_dbt_assets],
+ dbt_select="tag:foo"
+ )
+
+ # Furthermore, select all assets downstream (dbt-related or otherwise).
+ bar_plus_and_foo_and_downstream_selection = bar_plus_and_foo_selection.downstream()
+ ```
+
+
+
+
+
+
dagster_dbt.build_schedule_from_dbt_selection
+
+
+ Build a schedule to materialize a specified set of dbt resources from a dbt selection string.
+
+ See [https://docs.getdbt.com/reference/node-selection/syntax#how-does-selection-work](https://docs.getdbt.com/reference/node-selection/syntax#how-does-selection-work) for
+ more information.
+
+ Parameters:
+ - job_name (str) – The name of the job to materialize the dbt resources.
+ - cron_schedule (str) – The cron schedule to define the schedule.
+ - dbt_select (str) – A dbt selection string to specify a set of dbt resources.
+ - dbt_exclude (Optional[str]) – A dbt selection string to exclude a set of dbt resources.
+ - schedule_name (Optional[str]) – The name of the dbt schedule to create.
+ - tags (Optional[Mapping[str, str]]) – A dictionary of tags (string key-value pairs) to attach
+ - config (Optional[[*RunConfig*](../config.mdx#dagster.RunConfig)]) – The config that parameterizes the execution of this schedule.
+ - execution_timezone (Optional[str]) – Timezone in which the schedule should run.
+
+
+ Returns: A definition to materialize the selected dbt resources on a cron schedule.Return type: [ScheduleDefinition](../schedules-sensors.mdx#dagster.ScheduleDefinition)
+ Examples:
+
+ ```python
+ from dagster_dbt import dbt_assets, build_schedule_from_dbt_selection
+
+ @dbt_assets(manifest=...)
+ def all_dbt_assets():
+ ...
+
+ daily_dbt_assets_schedule = build_schedule_from_dbt_selection(
+ [all_dbt_assets],
+ job_name="all_dbt_assets",
+ cron_schedule="0 0 * * *",
+ dbt_select="fqn:*",
+ )
+ ```
+
+
+
+
+
+
dagster_dbt.get_asset_key_for_model
+
+
+ Return the corresponding Dagster asset key for a dbt model, seed, or snapshot.
+
+ Parameters:
+ - dbt_assets ([*AssetsDefinition*](../assets.mdx#dagster.AssetsDefinition)) – An AssetsDefinition object produced by @dbt_assets.
+ - model_name (str) – The name of the dbt model, seed, or snapshot.
+
+
+ Returns: The corresponding Dagster asset key.Return type: [AssetKey](../assets.mdx#dagster.AssetKey)
+ Examples:
+
+ ```python
+ from dagster import asset
+ from dagster_dbt import dbt_assets, get_asset_key_for_model
+
+ @dbt_assets(manifest=...)
+ def all_dbt_assets():
+ ...
+
+
+ @asset(deps={get_asset_key_for_model([all_dbt_assets], "customers")})
+ def cleaned_customers():
+ ...
+ ```
+
+
+
+
+
+
dagster_dbt.get_asset_key_for_source
+
+
+ Returns the corresponding Dagster asset key for a dbt source with a singular table.
+
+ Parameters: source_name (str) – The name of the dbt source.Raises: DagsterInvalidInvocationError – If the source has more than one table.Returns: The corresponding Dagster asset key.Return type: [AssetKey](../assets.mdx#dagster.AssetKey)
+ Examples:
+
+ ```python
+ from dagster import asset
+ from dagster_dbt import dbt_assets, get_asset_key_for_source
+
+ @dbt_assets(manifest=...)
+ def all_dbt_assets():
+ ...
+
+ @asset(key=get_asset_key_for_source([all_dbt_assets], "my_source"))
+ def upstream_python_asset():
+ ...
+ ```
+
+
+
+ Returns the corresponding Dagster asset keys for all tables in a dbt source.
+
+ This is a convenience method that makes it easy to define a multi-asset that generates
+ all the tables for a given dbt source.
+
+ Parameters: source_name (str) – The name of the dbt source.Returns:
+ A mapping of the table name to corresponding Dagster asset key
+ for all tables in the given dbt source.
+
+ Return type: Mapping[str, [AssetKey](../assets.mdx#dagster.AssetKey)]
+ Examples:
+
+ ```python
+ from dagster import AssetOut, multi_asset
+ from dagster_dbt import dbt_assets, get_asset_keys_by_output_name_for_source
+
+ @dbt_assets(manifest=...)
+ def all_dbt_assets():
+ ...
+
+ @multi_asset(
+ outs={
+ name: AssetOut(key=asset_key)
+ for name, asset_key in get_asset_keys_by_output_name_for_source(
+ [all_dbt_assets], "raw_data"
+ ).items()
+ },
+ )
+ def upstream_python_asset():
+ ...
+ ```
+
+
+
+
+
+
class dagster_dbt.DbtProject
+
+
+ Representation of a dbt project and related settings that assist with managing the project preparation.
+
+ Using this helps achieve a setup where the dbt manifest file
+ and dbt dependencies are available and up-to-date:
+ * during development, pull the dependencies and reload the manifest at run time to pick up any changes.
+ * when deployed, expect a manifest that was created at build time to reduce start-up time.
+
+ The cli `dagster-dbt project prepare-and-package` can be used as part of the deployment process to
+ handle the project preparation.
+
+ This object can be passed directly to [`DbtCliResource`](#dagster_dbt.DbtCliResource).
+
+ Parameters:
+ - project_dir (Union[str, Path]) – The directory of the dbt project.
+ - target_path (Union[str, Path]) – The path, relative to the project directory, to output artifacts.
+ - profiles_dir (Union[str, Path]) – The path to the directory containing your dbt profiles.yml.
+ - profile (Optional[str]) – The profile from your dbt profiles.yml to use for execution, if it should be explicitly set.
+ - target (Optional[str]) – The target from your dbt profiles.yml to use for execution, if it should be explicitly set.
+ - packaged_project_dir (Optional[Union[str, Path]]) – A directory that will contain a copy of the dbt project and the manifest.json
+ - state_path (Optional[Union[str, Path]]) – The path, relative to the project directory, to reference artifacts from another run.
+
+
+ Examples:
+
+ Creating a DbtProject with by referencing the dbt project directory:
+
+ ```python
+ from pathlib import Path
+
+ from dagster_dbt import DbtProject
+
+ my_project = DbtProject(project_dir=Path("path/to/dbt_project"))
+ ```
+ Creating a DbtProject that changes target based on environment variables and uses manged state artifacts:
+
+ ```python
+ import os
+ from pathlib import Path
+ from dagster_dbt import DbtProject
+
+
+ def get_env():
+ if os.getenv("DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT", "") == "1":
+ return "BRANCH"
+ if os.getenv("DAGSTER_CLOUD_DEPLOYMENT_NAME", "") == "prod":
+ return "PROD"
+ return "LOCAL"
+
+
+ dbt_project = DbtProject(
+ project_dir=Path('path/to/dbt_project'),
+ state_path="target/managed_state",
+ target=get_env(),
+ )
+ ```
+
+
prepare_if_dev
+
+
+ Prepare a dbt project at run time during development, i.e. when dagster dev is used.
+ This method has no effect outside this development context.
+
+ The preparation process ensures that the dbt manifest file and dbt dependencies are available and up-to-date.
+ During development, it pulls the dependencies and reloads the manifest at run time to pick up any changes.
+
+ If this method returns successfully, self.manifest_path will point to a loadable manifest file.
+ This method causes errors if the manifest file has not been correctly created by the preparation process.
+
+ Examples:
+
+ Preparing a DbtProject during development:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import Definitions
+ from dagster_dbt import DbtProject
+
+ my_project = DbtProject(project_dir=Path("path/to/dbt_project"))
+ my_project.prepare_if_dev()
+
+ defs = Definitions(
+ resources={
+ "dbt": DbtCliResource(project_dir=my_project),
+ },
+ ...
+ )
+ ```
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a sequence of freshness checks constructed from the provided dbt assets.
+
+ Freshness checks can be configured on a per-model basis in the model schema configuration.
+
+ For assets which are not partitioned based on time, the freshness check configuration mirrors
+ that of the `build_last_update_freshness_checks()` function. lower_bound_delta is provided in
+ terms of seconds, and deadline_cron is optional.
+
+ For time-partitioned assets, the freshness check configuration mirrors that of the
+ `build_time_partition_freshness_checks()` function.
+
+ Below is example of configuring a non-time-partitioned dbt asset with a freshness check.
+ This code would be placed in the schema.yml file for the dbt model.
+
+ ```YAML
+ models:
+ - name: customers
+ ...
+ meta:
+ dagster:
+ freshness_check:
+ lower_bound_delta_seconds: 86400 # 1 day
+ deadline_cron: "0 0 * * *" # Optional
+ severity: "WARN" # Optional, defaults to "WARN"
+ ```
+ Below is an example of configuring a time-partitioned dbt asset with a freshness check.
+ This code would be placed in the schema.yml file for the dbt model.
+
+ ```yaml
+ models:
+ - name: customers
+ ...
+ meta:
+ dagster:
+ freshness_check:
+ deadline_cron: "0 0 * * *"
+ severity: "WARN" # Optional, defaults to "WARN"
+ ```
+ Parameters: dbt_assets (Sequence[[*AssetsDefinition*](../assets.mdx#dagster.AssetsDefinition)]) – A sequence of dbt assets to construct freshness
+ checks from.Returns:
+ A sequence of asset checks definitions representing the
+ freshness checks for the provided dbt assets.
+
+ Return type: Sequence[[AssetChecksDefinition](../asset-checks.mdx#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
+
+
+
+
+### Resources (dbt Core)
+
+
+
+
+#### CLI Resource
+
+
+
class dagster_dbt.DbtCliResource
+
+
+ A resource used to execute dbt CLI commands.
+
+
+
project_dir
+
+
+ The path to the dbt project directory. This directory should contain a
+ dbt_project.yml. See [https://docs.getdbt.com/reference/dbt_project.yml](https://docs.getdbt.com/reference/dbt_project.yml) for more
+ information.
+
+ Type: str
+
+
+
+
+
+
global_config_flags
+
+
+ A list of global flags configuration to pass to the dbt CLI
+ invocation. Invoke dbt –help to see a full list of global flags.
+
+ Type: List[str]
+
+
+
+
+
+
profiles_dir
+
+
+ The path to the directory containing your dbt profiles.yml.
+ By default, the current working directory is used, which is the dbt project directory.
+ See [https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles](https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles) for more
+ information.
+
+ Type: Optional[str]
+
+
+
+
+
+
profile
+
+
+ The profile from your dbt profiles.yml to use for execution. See
+ [https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles](https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles) for more
+ information.
+
+ Type: Optional[str]
+
+
+
+
+
+
target
+
+
+ The target from your dbt profiles.yml to use for execution. See
+ [https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles](https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles) for more
+ information.
+
+ Type: Optional[str]
+
+
+
+
+
+
dbt_executable
+
+
+ The path to the dbt executable. By default, this is dbt.
+
+ Type: str
+
+
+
+
+
+
state_path
+
+
+ The path, relative to the project directory, to a directory of
+ dbt artifacts to be used with –state / –defer-state.
+
+ Type: Optional[str]
+
+
+
+
+ Examples:
+
+ Creating a dbt resource with only a reference to `project_dir`:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(project_dir="/path/to/dbt/project")
+ ```
+ Creating a dbt resource with a custom `profiles_dir`:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(
+ project_dir="/path/to/dbt/project",
+ profiles_dir="/path/to/dbt/project/profiles",
+ )
+ ```
+ Creating a dbt resource with a custom `profile` and `target`:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(
+ project_dir="/path/to/dbt/project",
+ profiles_dir="/path/to/dbt/project/profiles",
+ profile="jaffle_shop",
+ target="dev",
+ )
+ ```
+ Creating a dbt resource with global configs, e.g. disabling colored logs with `--no-use-color`:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(
+ project_dir="/path/to/dbt/project",
+ global_config_flags=["--no-use-color"],
+ )
+ ```
+ Creating a dbt resource with custom dbt executable path:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(
+ project_dir="/path/to/dbt/project",
+ dbt_executable="/path/to/dbt/executable",
+ )
+ ```
+
+
cli
+
+
+ Create a subprocess to execute a dbt CLI command.
+
+ Parameters:
+ - args (Sequence[str]) – The dbt CLI command to execute.
+ - raise_on_error (bool) – Whether to raise an exception if the dbt CLI command fails.
+ - manifest (Optional[Union[Mapping[str, Any], str, Path]]) – The dbt manifest blob. If an
+ - dagster_dbt_translator (Optional[[*DagsterDbtTranslator*](#dagster_dbt.DagsterDbtTranslator)]) – The translator to link dbt
+ - context (Optional[Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]]) – The execution context from within @dbt_assets.
+ - target_path (Optional[Path]) – An explicit path to a target folder to use to store and
+
+
+ Returns:
+ A invocation instance that can be used to retrieve the output of the
+ dbt CLI command.
+
+ Return type: [DbtCliInvocation](#dagster_dbt.DbtCliInvocation)
+ Examples:
+
+ Streaming Dagster events for dbt asset materializations and observations:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ yield from dbt.cli(["run"], context=context).stream()
+ ```
+ Retrieving a dbt artifact after streaming the Dagster events:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_run_invocation = dbt.cli(["run"], context=context)
+
+ yield from dbt_run_invocation.stream()
+
+ # Retrieve the `run_results.json` dbt artifact as a dictionary:
+ run_results_json = dbt_run_invocation.get_artifact("run_results.json")
+
+ # Retrieve the `run_results.json` dbt artifact as a file path:
+ run_results_path = dbt_run_invocation.target_path.joinpath("run_results.json")
+ ```
+ Customizing the asset materialization metadata when streaming the Dagster events:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_cli_invocation = dbt.cli(["run"], context=context)
+
+ for dagster_event in dbt_cli_invocation.stream():
+ if isinstance(dagster_event, Output):
+ context.add_output_metadata(
+ metadata={
+ "my_custom_metadata": "my_custom_metadata_value",
+ },
+ output_name=dagster_event.output_name,
+ )
+
+ yield dagster_event
+ ```
+ Suppressing exceptions from a dbt CLI command when a non-zero exit code is returned:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import AssetExecutionContext
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
+ dbt_run_invocation = dbt.cli(["run"], context=context, raise_on_error=False)
+
+ if dbt_run_invocation.is_successful():
+ yield from dbt_run_invocation.stream()
+ else:
+ ...
+ ```
+ Invoking a dbt CLI command in a custom asset or op:
+
+ ```python
+ import json
+
+ from dagster import Nothing, Out, asset, op
+ from dagster_dbt import DbtCliResource
+
+
+ @asset
+ def my_dbt_asset(dbt: DbtCliResource):
+ dbt_macro_args = {"key": "value"}
+ dbt.cli(["run-operation", "my-macro", json.dumps(dbt_macro_args)]).wait()
+
+
+ @op(out=Out(Nothing))
+ def my_dbt_op(dbt: DbtCliResource):
+ dbt_macro_args = {"key": "value"}
+ yield from dbt.cli(["run-operation", "my-macro", json.dumps(dbt_macro_args)]).stream()
+ ```
+
+
+
+
+
+
get_defer_args
+
+
+ Build the defer arguments for the dbt CLI command, using the supplied state directory.
+ If no state directory is supplied, or the state directory does not have a manifest for.
+ comparison, an empty list of arguments is returned.
+
+ Returns: The defer arguments for the dbt CLI command.Return type: Sequence[str]
+
+
+
+
+
+
get_state_args
+
+
+ Build the state arguments for the dbt CLI command, using the supplied state directory.
+ If no state directory is supplied, or the state directory does not have a manifest for.
+ comparison, an empty list of arguments is returned.
+
+ Returns: The state arguments for the dbt CLI command.Return type: Sequence[str]
+
+
+
+
+
+
+
+
+
+
class dagster_dbt.DbtCliInvocation
+
+
+ The representation of an invoked dbt command.
+
+ Parameters:
+ - process (subprocess.Popen) – The process running the dbt command.
+ - manifest (Mapping[str, Any]) – The dbt manifest blob.
+ - project_dir (Path) – The path to the dbt project.
+ - target_path (Path) – The path to the dbt target folder.
+ - raise_on_error (bool) – Whether to raise an exception if the dbt command fails.
+
+
+
+
get_artifact
+
+
+ Retrieve a dbt artifact from the target path.
+
+ See [https://docs.getdbt.com/reference/artifacts/dbt-artifacts](https://docs.getdbt.com/reference/artifacts/dbt-artifacts) for more information.
+
+ Parameters: artifact (Union[Literal["manifest.json"], Literal["catalog.json"], Literal["run_results.json"], Literal["sources.json"]]) – The name of the artifact to retrieve.Returns: The artifact as a dictionary.Return type: Dict[str, Any]
+ Examples:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(project_dir="/path/to/dbt/project")
+
+ dbt_cli_invocation = dbt.cli(["run"]).wait()
+
+ # Retrieve the run_results.json artifact.
+ run_results = dbt_cli_invocation.get_artifact("run_results.json")
+ ```
+
+
+
+
+
+
get_error
+
+
+ Return an exception if the dbt CLI process failed.
+
+ Returns: An exception if the dbt CLI process failed, and None otherwise.Return type: Optional[Exception]
+ Examples:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(project_dir="/path/to/dbt/project")
+
+ dbt_cli_invocation = dbt.cli(["run"], raise_on_error=False)
+
+ error = dbt_cli_invocation.get_error()
+ if error:
+ logger.error(error)
+ ```
+
+
+
+
+
+
is_successful
+
+
+ Return whether the dbt CLI process completed successfully.
+
+ Returns: True, if the dbt CLI process returns with a zero exit code, and False otherwise.Return type: bool
+ Examples:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(project_dir="/path/to/dbt/project")
+
+ dbt_cli_invocation = dbt.cli(["run"], raise_on_error=False)
+
+ if dbt_cli_invocation.is_successful():
+ ...
+ ```
+
+
+
+
+
+
stream
+
+
+ Stream the events from the dbt CLI process and convert them to Dagster events.
+
+ Returns:
+ A set of corresponding Dagster events.
+
+ In a Dagster asset definition, the following are yielded:
+ - Output for refables (e.g. models, seeds, snapshots.)
+ - AssetCheckResult for dbt test results that are enabled as asset checks.
+ - AssetObservation for dbt test results that are not enabled as asset checks.
+
+ In a Dagster op definition, the following are yielded:
+ - AssetMaterialization for dbt test results that are not enabled as asset checks.
+ - AssetObservation for dbt test results.
+
+ Return type: Iterator[Union[[Output](../ops.mdx#dagster.Output), [AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [AssetObservation](../assets.mdx#dagster.AssetObservation), [AssetCheckResult](../asset-checks.mdx#dagster.AssetCheckResult)]]
+ Examples:
+
+ ```python
+ from pathlib import Path
+ from dagster_dbt import DbtCliResource, dbt_assets
+
+ @dbt_assets(manifest=Path("target", "manifest.json"))
+ def my_dbt_assets(context, dbt: DbtCliResource):
+ yield from dbt.cli(["run"], context=context).stream()
+ ```
+
+
+
+
+
+
stream_raw_events
+
+
+ Stream the events from the dbt CLI process.
+
+ Returns: An iterator of events from the dbt CLI process.Return type: Iterator[[DbtCliEventMessage](#dagster_dbt.DbtCliEventMessage)]
+
+
+
+
+
+
wait
+
+
+ Wait for the dbt CLI process to complete.
+
+ Returns: The current representation of the dbt CLI invocation.Return type: [DbtCliInvocation](#dagster_dbt.DbtCliInvocation)
+ Examples:
+
+ ```python
+ from dagster_dbt import DbtCliResource
+
+ dbt = DbtCliResource(project_dir="/path/to/dbt/project")
+
+ dbt_cli_invocation = dbt.cli(["run"]).wait()
+ ```
+
+
+
+
+
+
+
+
+
+
class dagster_dbt.core.dbt_cli_invocation.DbtEventIterator
+
+
+ A wrapper around an iterator of dbt events which contains additional methods for
+ post-processing the events, such as fetching row counts for materialized tables.
+
+
+
fetch_column_metadata
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Experimental functionality which will fetch column schema metadata for dbt models in a run
+ once they’re built. It will also fetch schema information for upstream models and generate
+ column lineage metadata using sqlglot, if enabled.
+
+ Parameters: generate_column_lineage (bool) – Whether to generate column lineage metadata using sqlglot.Returns: A set of corresponding Dagster events for dbt models, with column metadata attached,
+ yielded in the order they are emitted by dbt.Return type: Iterator[Union[[Output](../ops.mdx#dagster.Output), [AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [AssetObservation](../assets.mdx#dagster.AssetObservation), [AssetCheckResult](../asset-checks.mdx#dagster.AssetCheckResult)]]
+
+
+
+
+
+
fetch_row_counts
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Experimental functionality which will fetch row counts for materialized dbt
+ models in a dbt run once they are built. Note that row counts will not be fetched
+ for views, since this requires running the view’s SQL query which may be costly.
+
+ Returns: A set of corresponding Dagster events for dbt models, with row counts attached,
+ yielded in the order they are emitted by dbt.Return type: Iterator[Union[[Output](../ops.mdx#dagster.Output), [AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [AssetObservation](../assets.mdx#dagster.AssetObservation), [AssetCheckResult](../asset-checks.mdx#dagster.AssetCheckResult)]]
+
+
+
+
+
+
with_insights
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Associate each warehouse query with the produced asset materializations for use in Dagster
+ Plus Insights. Currently supports Snowflake and BigQuery.
+
+ For more information, see the documentation for
+ dagster_cloud.dagster_insights.dbt_with_snowflake_insights and
+ dagster_cloud.dagster_insights.dbt_with_bigquery_insights.
+
+ Parameters:
+ - skip_config_check (bool) – If true, skips the check that the dbt project config is set up
+ - record_observation_usage (bool) – If True, associates the usage associated with
+
+
+ Example:
+
+ ```python
+ @dbt_assets(manifest=DBT_MANIFEST_PATH)
+ def jaffle_shop_dbt_assets(
+ context: AssetExecutionContext,
+ dbt: DbtCliResource,
+ ):
+ yield from dbt.cli(["build"], context=context).stream().with_insights()
+ ```
+
+
+
+
+
+
+
+
+
+
class dagster_dbt.DbtCliEventMessage
+
+
+ The representation of a dbt CLI event.
+
+ Parameters:
+ - raw_event (Dict[str, Any]) – The raw event dictionary.
+ - event_history_metadata (Dict[str, Any]) – A dictionary of metadata about the
+
+
+
+
to_default_asset_events
+
+
+ Convert a dbt CLI event to a set of corresponding Dagster events.
+
+ Parameters:
+ - manifest (Union[Mapping[str, Any], str, Path]) – The dbt manifest blob.
+ - dagster_dbt_translator ([*DagsterDbtTranslator*](#dagster_dbt.DagsterDbtTranslator)) – Optionally, a custom translator for
+ - context (Optional[Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]]) – The execution context.
+ - target_path (Optional[Path]) – An explicit path to a target folder used to retrieve
+
+
+ Returns:
+ A set of corresponding Dagster events.
+
+ In a Dagster asset definition, the following are yielded:
+ - Output for refables (e.g. models, seeds, snapshots.)
+ - AssetCheckResult for dbt test results that are enabled as asset checks.
+ - AssetObservation for dbt test results that are not enabled as asset checks.
+
+ In a Dagster op definition, the following are yielded:
+ - AssetMaterialization for dbt test results that are not enabled as asset checks.
+ - AssetObservation for dbt test results.
+
+ Return type: Iterator[Union[[Output](../ops.mdx#dagster.Output), [AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [AssetObservation](../assets.mdx#dagster.AssetObservation), [AssetCheckResult](../asset-checks.mdx#dagster.AssetCheckResult)]]
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## dbt Cloud
+
+Here, we provide interfaces to manage dbt projects invoked by the hosted dbt Cloud service.
+
+
+
+
+### Assets (dbt Cloud)
+
+
+
dagster_dbt.load_assets_from_dbt_cloud_job
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Loads a set of dbt models, managed by a dbt Cloud job, into Dagster assets. In order to
+ determine the set of dbt models, the project is compiled to generate the necessary artifacts
+ that define the dbt models and their dependencies.
+
+ One Dagster asset is created for each dbt model.
+
+ Parameters:
+ - dbt_cloud ([*ResourceDefinition*](../resources.mdx#dagster.ResourceDefinition)) – The dbt Cloud resource to use to connect to the dbt Cloud API.
+ - job_id (int) – The ID of the dbt Cloud job to load assets from.
+ - node_info_to_asset_key – (Mapping[str, Any] -> AssetKey): A function that takes a dictionary
+ - node_info_to_group_fn (Dict[str, Any] -> Optional[str]) – A function that takes a
+ - node_info_to_freshness_policy_fn (Dict[str, Any] -> Optional[FreshnessPolicy]) – A function
+ - node_info_to_auto_materialize_policy_fn (Dict[str, Any] -> Optional[AutoMaterializePolicy]) – A function that takes a dictionary of dbt node info and optionally returns a AutoMaterializePolicy
+ - node_info_to_definition_metadata_fn (Dict[str, Any] -> Optional[Dict[str, RawMetadataMapping]]) – A function that takes a dictionary of dbt node info and optionally returns a dictionary
+ - partitions_def (Optional[[*PartitionsDefinition*](../partitions.mdx#dagster.PartitionsDefinition)]) – experimental
+ - partition_key_to_vars_fn (Optional[str -> Dict[str, Any]]) – experimental
+
+
+ Returns: A definition for the loaded assets.Return type: CacheableAssetsDefinition
+ Examples:
+
+ ```python
+ from dagster import repository
+ from dagster_dbt import dbt_cloud_resource, load_assets_from_dbt_cloud_job
+
+ DBT_CLOUD_JOB_ID = 1234
+
+ dbt_cloud = dbt_cloud_resource.configured(
+ {
+ "auth_token": {"env": "DBT_CLOUD_API_TOKEN"},
+ "account_id": {"env": "DBT_CLOUD_ACCOUNT_ID"},
+ }
+ )
+
+ dbt_cloud_assets = load_assets_from_dbt_cloud_job(
+ dbt_cloud=dbt_cloud, job_id=DBT_CLOUD_JOB_ID
+ )
+
+
+ @repository
+ def dbt_cloud_sandbox():
+ return [dbt_cloud_assets]
+ ```
+
+
+
+
+
+
+
+
+
+
+### Ops (dbt Cloud)
+
+
+
dagster_dbt.dbt_cloud_run_op `=` \
+
+
+
+
+ Initiates a run for a dbt Cloud job, then polls until the run completes. If the job
+ fails or is otherwised stopped before succeeding, a dagster.Failure exception will be raised,
+ and this op will fail.
+
+ It requires the use of a ‘dbt_cloud’ resource, which is used to connect to the dbt Cloud API.
+
+ Config Options:
+
+ job_id (int)
+ The integer ID of the relevant dbt Cloud job. You can find this value by going to the details
+ page of your job in the dbt Cloud UI. It will be the final number in the url, e.g.:
+ `https://cloud.getdbt.com/#/accounts/\{account_id}/projects/\{project_id}/jobs/\{job_id}/`
+
+ poll_interval (float)
+ The time (in seconds) that will be waited between successive polls. Defaults to `10`.
+
+ poll_timeout (float)
+ The maximum time (in seconds) that will waited before this operation is timed out. By
+ default, this will never time out.
+
+ yield_materializations (bool)
+ If True, materializations corresponding to the results of the dbt operation will be
+ yielded when the solid executes. Defaults to `True`.
+
+ rasset_key_prefix (float)
+ If provided and yield_materializations is True, these components will be used to ”
+ prefix the generated asset keys. Defaults to [“dbt”].
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_dbt import dbt_cloud_resource, dbt_cloud_run_op
+
+ my_dbt_cloud_resource = dbt_cloud_resource.configured(
+ {"auth_token": {"env": "DBT_CLOUD_AUTH_TOKEN"}, "account_id": 77777}
+ )
+ run_dbt_nightly_sync = dbt_cloud_run_op.configured(
+ {"job_id": 54321}, name="run_dbt_nightly_sync"
+ )
+
+ @job(resource_defs={"dbt_cloud": my_dbt_cloud_resource})
+ def dbt_cloud():
+ run_dbt_nightly_sync()
+ ```
+
+
+
+
+
+
+
+
+
+
+### Resources (dbt Cloud)
+
+
+
class dagster_dbt.DbtCloudClientResource
+
+ This resource helps interact with dbt Cloud connectors.
+
+
+
+
+
+
+#### Deprecated (dbt Cloud)
+
+
+
dagster_dbt.dbt_cloud_resource ResourceDefinition
+
+
+
+
+ This resource allows users to programatically interface with the dbt Cloud Administrative REST
+ API (v2) to launch jobs and monitor their progress. This currently implements only a subset of
+ the functionality exposed by the API.
+
+ For a complete set of documentation on the dbt Cloud Administrative REST API, including expected
+ response JSON schemae, see the [dbt Cloud API Docs](https://docs.getdbt.com/dbt-cloud/api-v2).
+
+ To configure this resource, we recommend using the [configured](https://legacy-docs.dagster.io/concepts/configuration/configured) method.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_dbt import dbt_cloud_resource
+
+ my_dbt_cloud_resource = dbt_cloud_resource.configured(
+ {
+ "auth_token": {"env": "DBT_CLOUD_AUTH_TOKEN"},
+ "account_id": {"env": "DBT_CLOUD_ACCOUNT_ID"},
+ }
+ )
+
+ @job(resource_defs={"dbt_cloud": my_dbt_cloud_resource})
+ def my_dbt_cloud_job():
+ ...
+ ```
+
+
+
+
+
+
+
+
+
+
+## Errors
+
+
+
exception dagster_dbt.DagsterDbtError
+
+ The base exception of the `dagster-dbt` library.
+
+
+
+
+
exception dagster_dbt.DagsterDbtCliRuntimeError
+
+ Represents an error while executing a dbt CLI command.
+
+
+
+
+
+
+
+
+## Utils
+
+
+
dagster_dbt.default_group_from_dbt_resource_props
+
+
+ Get the group name for a dbt node.
+
+ If a Dagster group is configured in the metadata for the node, use that.
+
+ Otherwise, if a dbt group is configured for the node, use that.
+
+
+
+
+ Get the group name for a dbt node.
+
+ Has the same behavior as the default_group_from_dbt_resource_props, except for that, if no group can be determined
+ from config or metadata, falls back to using the subdirectory of the models directory that the
+ source file is in.
+
+ Parameters: dbt_resource_props (Mapping[str, Any]) – A dictionary representing the dbt resource.
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to Delta Lake.
+
+ Examples:
+
+ ```python
+ from dagster_deltalake import DeltaLakeIOManager
+ from dagster_deltalake_pandas import DeltaLakePandasTypeHandler
+
+ class MyDeltaLakeIOManager(DeltaLakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DeltaLakePandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema (parent folder) in Delta Lake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDeltaLakeIOManager()}
+ )
+ ```
+ If you do not provide a schema, Dagster will determine a schema based on the assets and ops using
+ the I/O Manager. For assets, the schema will be determined from the asset key, as in the above example.
+ For ops, the schema can be specified by including a “schema” entry in output metadata. If none
+ of these is provided, the schema will default to “public”.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_deltalake_pandas.DeltaLakePandasTypeHandler
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-deltalake-polars.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-deltalake-polars.mdx
new file mode 100644
index 0000000000000..0133088e962ea
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-deltalake-polars.mdx
@@ -0,0 +1,85 @@
+---
+title: 'deltalake + polars (dagster-deltalake-polars)'
+title_meta: 'deltalake + polars (dagster-deltalake-polars) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'deltalake + polars (dagster-deltalake-polars) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Deltalake + Polars (dagster-deltalake-polars)
+
+This library provides an integration with the [Delta Lake](https://delta.io/) storage framework.
+
+Related guides:
+
+ - [Using Dagster with Delta Lake guide](https://docs.dagster.io/integrations/libraries/deltalake/)
+ - [DeltaLake I/O manager reference](https://docs.dagster.io/integrations/libraries/deltalake/reference)
+
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to Delta Lake.
+
+ Examples:
+
+ ```python
+ from dagster_deltalake import DeltaLakeIOManager
+ from dagster_deltalake_pandas import DeltaLakePandasTypeHandler
+
+ class MyDeltaLakeIOManager(DeltaLakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DeltaLakePandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema (parent folder) in Delta Lake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDeltaLakeIOManager()}
+ )
+ ```
+ If you do not provide a schema, Dagster will determine a schema based on the assets and ops using
+ the I/O Manager. For assets, the schema will be determined from the asset key, as in the above example.
+ For ops, the schema can be specified by including a “schema” entry in output metadata. If none
+ of these is provided, the schema will default to “public”.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_deltalake_polars.DeltaLakePolarsTypeHandler
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-deltalake.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-deltalake.mdx
new file mode 100644
index 0000000000000..9b97c8e2767ca
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-deltalake.mdx
@@ -0,0 +1,167 @@
+---
+title: 'delta lake (dagster-deltalake)'
+title_meta: 'delta lake (dagster-deltalake) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'delta lake (dagster-deltalake) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Delta Lake (dagster-deltalake)
+
+This library provides an integration with the [Delta Lake](https://delta.io/) storage framework.
+
+Related Guides:
+
+ - [Using Dagster with Delta Lake tutorial](https://docs.dagster.io/integrations/libraries/deltalake/)
+ - [Delta Lake reference](https://docs.dagster.io/integrations/libraries/deltalake/reference)
+
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to Delta Lake.
+
+ Examples:
+
+ ```python
+ from dagster_deltalake import DeltaLakeIOManager
+ from dagster_deltalake_pandas import DeltaLakePandasTypeHandler
+
+ class MyDeltaLakeIOManager(DeltaLakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DeltaLakePandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema (parent folder) in Delta Lake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDeltaLakeIOManager()}
+ )
+ ```
+ If you do not provide a schema, Dagster will determine a schema based on the assets and ops using
+ the I/O Manager. For assets, the schema will be determined from the asset key, as in the above example.
+ For ops, the schema can be specified by including a “schema” entry in output metadata. If none
+ of these is provided, the schema will default to “public”.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to Delta Lake.
+
+ Examples:
+
+ ```python
+ from dagster_deltalake import DeltaLakeIOManager
+ from dagster_deltalake_pandas import DeltaLakePandasTypeHandler
+
+ class MyDeltaLakeIOManager(DeltaLakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DeltaLakePandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema (parent folder) in Delta Lake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDeltaLakeIOManager()}
+ )
+ ```
+ If you do not provide a schema, Dagster will determine a schema based on the assets and ops using
+ the I/O Manager. For assets, the schema will be determined from the asset key, as in the above example.
+ For ops, the schema can be specified by including a “schema” entry in output metadata. If none
+ of these is provided, the schema will default to “public”.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-dlt.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-dlt.mdx
new file mode 100644
index 0000000000000..9f36d071c8937
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-dlt.mdx
@@ -0,0 +1,359 @@
+---
+title: 'dlt (dagster-dlt)'
+title_meta: 'dlt (dagster-dlt) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dlt (dagster-dlt) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# dlt (dagster-dlt)
+
+This library provides a Dagster integration with [dlt](https://dlthub.com).
+
+For more information on getting started, see the [dlt & Dagster](https://docs.dagster.io/integrations/libraries/dlt/) documentation.
+
+
+
+
+## Assets
+
+
+
@dagster_dlt.dlt_assets
+
+
+ Asset Factory for using data load tool (dlt).
+
+ Parameters:
+ - dlt_source (DltSource) – The DltSource to be ingested.
+ - dlt_pipeline (Pipeline) – The dlt Pipeline defining the destination parameters.
+ - name (Optional[str], optional) – The name of the op.
+ - group_name (Optional[str], optional) – The name of the asset group.
+ - dagster_dlt_translator ([*DagsterDltTranslator*](#dagster_dlt.DagsterDltTranslator), optional) – Customization object for defining asset parameters from dlt resources.
+ - partitions_def (Optional[[*PartitionsDefinition*](../partitions.mdx#dagster.PartitionsDefinition)]) – Optional partitions definition.
+ - op_tags (Optional[Mapping[str, Any]]) – The tags for the underlying op.
+
+
+ Examples:
+
+ Loading Hubspot data to Snowflake with an auto materialize policy using the dlt verified source:
+
+ ```python
+ from dagster_dlt import DagsterDltResource, DagsterDltTranslator, dlt_assets
+
+
+ class HubspotDagsterDltTranslator(DagsterDltTranslator):
+ @public
+ def get_auto_materialize_policy(self, resource: DltResource) -> Optional[AutoMaterializePolicy]:
+ return AutoMaterializePolicy.eager().with_rules(
+ AutoMaterializeRule.materialize_on_cron("0 0 * * *")
+ )
+
+
+ @dlt_assets(
+ dlt_source=hubspot(include_history=True),
+ dlt_pipeline=pipeline(
+ pipeline_name="hubspot",
+ dataset_name="hubspot",
+ destination="snowflake",
+ progress="log",
+ ),
+ name="hubspot",
+ group_name="hubspot",
+ dagster_dlt_translator=HubspotDagsterDltTranslator(),
+ )
+ def hubspot_assets(context: AssetExecutionContext, dlt: DagsterDltResource):
+ yield from dlt.run(context=context)
+ ```
+ Loading Github issues to snowflake:
+
+ ```python
+ from dagster_dlt import DagsterDltResource, dlt_assets
+
+
+ @dlt_assets(
+ dlt_source=github_reactions(
+ "dagster-io", "dagster", items_per_page=100, max_items=250
+ ),
+ dlt_pipeline=pipeline(
+ pipeline_name="github_issues",
+ dataset_name="github",
+ destination="snowflake",
+ progress="log",
+ ),
+ name="github",
+ group_name="github",
+ )
+ def github_reactions_dagster_assets(context: AssetExecutionContext, dlt: DagsterDltResource):
+ yield from dlt.run(context=context)
+ ```
+
+
+
+
+
+
dagster_dlt.build_dlt_asset_specs
+
+
+ Build a list of asset specs from a dlt source and pipeline.
+
+ Parameters:
+ - dlt_source (DltSource) – dlt source object
+ - dlt_pipeline (Pipeline) – dlt pipeline object
+ - dagster_dlt_translator (Optional[[*DagsterDltTranslator*](#dagster_dlt.DagsterDltTranslator)]) – Allows customizing how to
+
+
+ Returns: List[AssetSpec] list of asset specs from dlt source and pipeline
+
+
+
+
+
+
class dagster_dlt.DagsterDltTranslator
+
+
+
+
get_asset_key
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).key` instead..
+
+ :::
+
+ Defines asset key for a given dlt resource key and dataset name.
+
+ This method can be overridden to provide custom asset key for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: AssetKey of Dagster asset derived from dlt resource
+
+
+
+
+
+
get_auto_materialize_policy
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).auto_materialize_policy` instead..
+
+ :::
+
+ Defines resource specific auto materialize policy.
+
+ This method can be overridden to provide custom auto materialize policy for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: The auto-materialize policy for a resourceReturn type: Optional[AutoMaterializePolicy]
+
+
+
+
+
+
get_automation_condition
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).automation_condition` instead..
+
+ :::
+
+ Defines resource specific automation condition.
+
+ This method can be overridden to provide custom automation condition for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: The automation condition for a resourceReturn type: Optional[[AutomationCondition](../assets.mdx#dagster.AutomationCondition)]
+
+
+
+
+
+
get_deps_asset_keys
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Iterate over `DagsterDltTranslator.get_asset_spec(...).deps` to access `AssetDep.asset_key` instead..
+
+ :::
+
+ Defines upstream asset dependencies given a dlt resource.
+
+ Defaults to a concatenation of resource.source_name and resource.name.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: The Dagster asset keys upstream of dlt_resource_key.Return type: Iterable[[AssetKey](../assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
get_description
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).description` instead..
+
+ :::
+
+ A method that takes in a dlt resource returns the Dagster description of the resource.
+
+ This method can be overridden to provide a custom description for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: The Dagster description for the dlt resource.Return type: Optional[str]
+
+
+
+
+
+
get_group_name
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).group_name` instead..
+
+ :::
+
+ A method that takes in a dlt resource and returns the Dagster group name of the resource.
+
+ This method can be overridden to provide a custom group name for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: A Dagster group name for the dlt resource.Return type: Optional[str]
+
+
+
+
+
+
get_kinds
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).kinds` instead..
+
+ :::
+
+ A method that takes in a dlt resource and returns the kinds which should be
+ attached. Defaults to the destination type and “dlt”.
+
+ This method can be overridden to provide custom kinds for a dlt resource.
+
+ Parameters:
+ - resource (DltResource) – dlt resource
+ - destination (Destination) – dlt destination
+
+
+ Returns: The kinds of the asset.Return type: Set[str]
+
+
+
+
+
+
get_metadata
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).metadata` instead..
+
+ :::
+
+ Defines resource specific metadata.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: The custom metadata entries for this resource.Return type: Mapping[str, Any]
+
+
+
+
+
+
get_owners
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).owners` instead..
+
+ :::
+
+ A method that takes in a dlt resource and returns the Dagster owners of the resource.
+
+ This method can be overridden to provide custom owners for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns: A sequence of Dagster owners for the dlt resource.Return type: Optional[Sequence[str]]
+
+
+
+
+
+
get_tags
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterDltTranslator.get_asset_spec(...).tags` instead..
+
+ :::
+
+ A method that takes in a dlt resource and returns the Dagster tags of the structure.
+
+ This method can be overridden to provide custom tags for a dlt resource.
+
+ Parameters: resource (DltResource) – dlt resourceReturns:
+ A dictionary representing the Dagster tags for the
+ dlt resource.
+
+ Return type: Optional[Mapping[str, str]]
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Resources
+
+
+
class dagster_dlt.DagsterDltResource
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+
+
run
+
+
+ Runs the dlt pipeline with subset support.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – Asset or op execution context
+ - dlt_source (Optional[DltSource]) – optional dlt source if resource is used from an @op
+ - dlt_pipeline (Optional[Pipeline]) – optional dlt pipeline if resource is used from an @op
+ - dagster_dlt_translator (Optional[[*DagsterDltTranslator*](#dagster_dlt.DagsterDltTranslator)]) – optional dlt translator if resource is used from an @op
+ - **kwargs (dict[str, Any]) – Keyword args passed to pipeline run method
+
+
+ Returns: An iterator of MaterializeResult or AssetMaterializationReturn type: DltEventIterator[DltEventType]
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-docker.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-docker.mdx
new file mode 100644
index 0000000000000..7747673692f4a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-docker.mdx
@@ -0,0 +1,209 @@
+---
+title: 'orchestration on docker'
+title_meta: 'orchestration on docker API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'orchestration on docker Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Orchestration on Docker
+
+
+
+
+
+
+
+# APIs
+
+
+
dagster_docker.DockerRunLauncher RunLauncher
+
+
+
+
+ Launches runs in a Docker container.
+
+
+
+
+
+
+
dagster_docker.docker_executor ExecutorDefinition
+
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Executor which launches steps as Docker containers.
+
+ To use the docker_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster_docker import docker_executor
+
+ from dagster import job
+
+ @job(executor_def=docker_executor)
+ def docker_job():
+ pass
+ ```
+ Then you can configure the executor with run config as follows:
+
+ ```YAML
+ execution:
+ config:
+ registry: ...
+ network: ...
+ networks: ...
+ container_kwargs: ...
+ ```
+ If you’re using the DockerRunLauncher, configuration set on the containers created by the run
+ launcher will also be set on the containers that are created for each step.
+
+
+
+
+
+
+
+
+## Ops
+
+
+
dagster_docker.docker_container_op `=` \
+
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ An op that runs a Docker container using the docker Python API.
+
+ Contrast with the docker_executor, which runs each Dagster op in a Dagster job in its
+ own Docker container.
+
+ This op may be useful when:
+ - You need to orchestrate a command that isn’t a Dagster op (or isn’t written in Python)
+ - You want to run the rest of a Dagster job using a specific executor, and only a single
+
+
+ For example:
+
+ ```python
+ from dagster_docker import docker_container_op
+
+ from dagster import job
+
+ first_op = docker_container_op.configured(
+ {
+ "image": "busybox",
+ "command": ["echo HELLO"],
+ },
+ name="first_op",
+ )
+ second_op = docker_container_op.configured(
+ {
+ "image": "busybox",
+ "command": ["echo GOODBYE"],
+ },
+ name="second_op",
+ )
+
+ @job
+ def full_job():
+ second_op(first_op())
+ ```
+ You can create your own op with the same implementation by calling the execute_docker_container function
+ inside your own op.
+
+
+
+
+
+
+
dagster_docker.execute_docker_container
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This function is a utility for executing a Docker container from within a Dagster op.
+
+ Parameters:
+ - image (str) – The image to use for the launched Docker container.
+ - entrypoint (Optional[Sequence[str]]) – The ENTRYPOINT to run in the launched Docker
+ - command (Optional[Sequence[str]]) – The CMD to run in the launched Docker container.
+ - networks (Optional[Sequence[str]]) – Names of the Docker networks to which to connect the
+ - registry – (Optional[Mapping[str, str]]): Information for using a non local/public Docker
+ - env_vars (Optional[Sequence[str]]) – List of environemnt variables to include in the launched
+ - container_kwargs (Optional[Dict[str[Any]]]) – key-value pairs that can be passed into
+
+
+
+
+
+
+
+
+
+
+
+## Pipes
+
+
+
class dagster_docker.PipesDockerClient
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A pipes client that runs external processes in docker containers.
+
+ By default context is injected via environment variables and messages are parsed out of the
+ log stream, with other logs forwarded to stdout of the orchestration process.
+
+ Parameters:
+ - env (Optional[Mapping[str, str]]) – An optional dict of environment variables to pass to the
+ - register (Optional[Mapping[str, str]]) – An optional dict of registry credentials to login to
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+
+
+
+
+
+
+
+
class dagster_docker.PipesDockerLogsMessageReader
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb-pandas.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb-pandas.mdx
new file mode 100644
index 0000000000000..a5f39aca185f0
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb-pandas.mdx
@@ -0,0 +1,218 @@
+---
+title: 'duckdb + pandas (dagster-duckdb-pandas)'
+title_meta: 'duckdb + pandas (dagster-duckdb-pandas) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'duckdb + pandas (dagster-duckdb-pandas) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# DuckDB + Pandas (dagster-duckdb-pandas)
+
+This library provides an integration with the [DuckDB](https://duckdb.org/) database and Pandas data processing library.
+
+Related guides:
+
+ - [Using Dagster with DuckDB guide](https://docs.dagster.io/integrations/libraries/duckdb/)
+ - [DuckDB I/O manager reference](https://docs.dagster.io/integrations/libraries/duckdb/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes Pandas DataFrames to DuckDB. When
+ using the DuckDBPandasIOManager, any inputs and outputs without type annotations will be loaded
+ as Pandas DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_pandas import DuckDBPandasIOManager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPandasIOManager(database="my_db.duckdb")}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPandasIOManager(database="my_db.duckdb", schema="my_schema")}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_duckdb_pandas.DuckDBPandasTypeHandler
+
+
+ Stores and loads Pandas DataFrames in DuckDB.
+
+ To use this type handler, return it from the `type_handlers` method of an I/O manager that inherits from ``DuckDBIOManager`.
+
+ Example:
+
+ ```python
+ from dagster_duckdb import DuckDBIOManager
+ from dagster_duckdb_pandas import DuckDBPandasTypeHandler
+
+ class MyDuckDBIOManager(DuckDBIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DuckDBPandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb")}
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes Pandas DataFrames to DuckDB. When
+ using the duckdb_pandas_io_manager, any inputs and outputs without type annotations will be loaded
+ as Pandas DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_pandas import duckdb_pandas_io_manager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_pandas_io_manager.configured({"database": "my_db.duckdb"})}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_pandas_io_manager.configured({"database": "my_db.duckdb", "schema": "my_schema"})}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb-polars.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb-polars.mdx
new file mode 100644
index 0000000000000..ca93e54436e7f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb-polars.mdx
@@ -0,0 +1,221 @@
+---
+title: 'duckdb + polars (dagster-duckdb-polars)'
+title_meta: 'duckdb + polars (dagster-duckdb-polars) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'duckdb + polars (dagster-duckdb-polars) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+# DuckDB + Polars (dagster-duckdb-polars)
+
+This library provides an integration with the [DuckDB](https://duckdb.org/) database and Polars data processing library.
+
+Related guides:
+
+ - [Using Dagster with DuckDB guide](https://docs.dagster.io/integrations/libraries/duckdb/)
+ - [DuckDB I/O manager reference](https://docs.dagster.io/integrations/libraries/duckdb/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes Polars DataFrames to DuckDB. When
+ using the DuckDBPolarsIOManager, any inputs and outputs without type annotations will be loaded
+ as Polars DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_polars import DuckDBPolarsIOManager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pl.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPolarsIOManager(database="my_db.duckdb")}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPolarsIOManager(database="my_db.duckdb", schema="my_schema")}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pl.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pl.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pl.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ >
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pl.DataFrame) -> pl.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
+
+
class dagster_duckdb_polars.DuckDBPolarsTypeHandler
+
+
+ Stores and loads Polars DataFrames in DuckDB.
+
+ To use this type handler, return it from the `type_handlers` method of an I/O manager that inherits from ``DuckDBIOManager`.
+
+ Example:
+
+ ```python
+ from dagster_duckdb import DuckDBIOManager
+ from dagster_duckdb_polars import DuckDBPolarsTypeHandler
+
+ class MyDuckDBIOManager(DuckDBIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DuckDBPolarsTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pl.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb")}
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes polars dataframes to DuckDB. When
+ using the duckdb_polars_io_manager, any inputs and outputs without type annotations will be loaded
+ as Polars DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_polars import duckdb_polars_io_manager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pl.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_polars_io_manager.configured({"database": "my_db.duckdb"})}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_polars_io_manager.configured({"database": "my_db.duckdb", "schema": "my_schema"})}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pl.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pl.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pl.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pl.DataFrame) -> pl.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb-pyspark.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb-pyspark.mdx
new file mode 100644
index 0000000000000..8248340b46909
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb-pyspark.mdx
@@ -0,0 +1,218 @@
+---
+title: 'duckdb + pyspark (dagster-duckdb-pyspark)'
+title_meta: 'duckdb + pyspark (dagster-duckdb-pyspark) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'duckdb + pyspark (dagster-duckdb-pyspark) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# DuckDB + PySpark (dagster-duckdb-pyspark)
+
+This library provides an integration with the [DuckDB](https://duckdb.org/) database and PySpark data processing library.
+
+Related guides:
+
+ - [Using Dagster with DuckDB guide](https://docs.dagster.io/integrations/libraries/duckdb/)
+ - [DuckDB I/O manager reference](https://docs.dagster.io/integrations/libraries/duckdb/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to DuckDB. When
+ using the DuckDBPySparkIOManager, any inputs and outputs without type annotations will be loaded
+ as PySpark DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_pyspark import DuckDBPySparkIOManager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pyspark.sql.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPySparkIOManager(database="my_db.duckdb")}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": DuckDBPySparkIOManager(database="my_db.duckdb", schema="my_schema")}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pyspark.sql.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pyspark.sql.DataFrame) -> pyspark.sql.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_duckdb_pyspark.DuckDBPySparkTypeHandler
+
+
+ Stores PySpark DataFrames in DuckDB.
+
+ To use this type handler, return it from the `type_handlers` method of an I/O manager that inherits from ``DuckDBIOManager`.
+
+ Example:
+
+ ```python
+ from dagster_duckdb import DuckDBIOManager
+ from dagster_duckdb_pyspark import DuckDBPySparkTypeHandler
+
+ class MyDuckDBIOManager(DuckDBIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DuckDBPySparkTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pyspark.sql.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb")}
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to DuckDB. When
+ using the duckdb_pyspark_io_manager, any inputs and outputs without type annotations will be loaded
+ as PySpark DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb_pyspark import duckdb_pyspark_io_manager
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in DuckDB
+ )
+ def my_table() -> pyspark.sql.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_pyspark_io_manager.configured({"database": "my_db.duckdb"})}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": duckdb_pyspark_io_manager.configured({"database": "my_db.duckdb", "schema": "my_schema"})}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pyspark.sql.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pyspark.sql.DataFrame) -> pyspark.sql.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb.mdx
new file mode 100644
index 0000000000000..7d1f0a6b5953f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-duckdb.mdx
@@ -0,0 +1,233 @@
+---
+title: 'duckdb (dagster-duckdb)'
+title_meta: 'duckdb (dagster-duckdb) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'duckdb (dagster-duckdb) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# DuckDB (dagster-duckdb)
+
+This library provides an integration with the [DuckDB](https://duckdb.org/) database.
+
+Related Guides:
+
+ - [Using Dagster with DuckDB guide](https://docs.dagster.io/integrations/libraries/duckdb/)
+ - [DuckDB I/O manager reference](https://docs.dagster.io/integrations/libraries/duckdb/reference)
+
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to DuckDB.
+
+ Examples:
+
+ ```python
+ from dagster_duckdb import DuckDBIOManager
+ from dagster_duckdb_pandas import DuckDBPandasTypeHandler
+
+ class MyDuckDBIOManager(DuckDBIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [DuckDBPandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb")}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb", schema="my_schema")}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ Set DuckDB configuration options using the connection_config field. See
+ [https://duckdb.org/docs/sql/configuration.html](https://duckdb.org/docs/sql/configuration.html) for all available settings.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={"io_manager": MyDuckDBIOManager(database="my_db.duckdb",
+ connection_config={"arrow_large_buffer_size": True})}
+ )
+ ```
+
+
+
+
+
+
dagster_duckdb.DuckDBResource ResourceDefinition
+
+
+
+
+ Resource for interacting with a DuckDB database.
+
+ Examples:
+
+ ```python
+ from dagster import Definitions, asset
+ from dagster_duckdb import DuckDBResource
+
+ @asset
+ def my_table(duckdb: DuckDBResource):
+ with duckdb.get_connection() as conn:
+ conn.execute("SELECT * from MY_SCHEMA.MY_TABLE")
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={"duckdb": DuckDBResource(database="path/to/db.duckdb")}
+ )
+ ```
+
+
+
+ Builds an IO manager definition that reads inputs from and writes outputs to DuckDB.
+
+ Parameters:
+ - type_handlers (Sequence[DbTypeHandler]) – Each handler defines how to translate between
+ - default_load_type (Type) – When an input has no type annotation, load it as this type.
+
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_duckdb import build_duckdb_io_manager
+ from dagster_duckdb_pandas import DuckDBPandasTypeHandler
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ duckdb_io_manager = build_duckdb_io_manager([DuckDBPandasTypeHandler()])
+
+ defs = Definitions(
+ assets=[my_table]
+ resources={"io_manager" duckdb_io_manager.configured({"database": "my_db.duckdb"})}
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the DuckDB I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={"io_manager" duckdb_io_manager.configured(
+ {"database": "my_db.duckdb", "schema": "my_schema"} # will be used as the schema
+ )}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in duckdb
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in duckdb
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame):
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-fivetran.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-fivetran.mdx
new file mode 100644
index 0000000000000..41c39bfb989de
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-fivetran.mdx
@@ -0,0 +1,541 @@
+---
+title: 'fivetran (dagster-fivetran)'
+title_meta: 'fivetran (dagster-fivetran) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'fivetran (dagster-fivetran) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Fivetran (dagster-fivetran)
+
+This library provides a Dagster integration with [Fivetran](https://www.fivetran.com/).
+
+
+
+
+## Assets (Fivetran API)
+
+
+
class dagster_fivetran.FivetranWorkspace
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This class represents a Fivetran workspace and provides utilities
+ to interact with Fivetran APIs.
+
+
+
sync_and_poll
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Executes a sync and poll process to materialize Fivetran assets.
+ This method can only be used in the context of an asset execution.
+
+ Parameters: context ([*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)) – The execution context
+ from within @fivetran_assets.Returns:
+ An iterator of MaterializeResult
+ or AssetMaterialization.
+
+ Return type: Iterator[Union[[AssetMaterialization](../ops.mdx#dagster.AssetMaterialization), [MaterializeResult](../assets.mdx#dagster.MaterializeResult)]]
+
+
+
+
+
+
+
+
+
+
class dagster_fivetran.DagsterFivetranTranslator
+
+ Translator class which converts a FivetranConnectorTableProps object into AssetSpecs.
+ Subclass this class to implement custom logic on how to translate Fivetran content into asset spec.
+
+
+
+
+
@dagster_fivetran.fivetran_assets
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Create a definition for how to sync the tables of a given Fivetran connector.
+
+ Parameters:
+ - connector_id (str) – The Fivetran Connector ID. You can retrieve this value from the
+ - workspace ([*FivetranWorkspace*](#dagster_fivetran.FivetranWorkspace)) – The Fivetran workspace to fetch assets from.
+ - name (Optional[str], optional) – The name of the op.
+ - group_name (Optional[str], optional) – The name of the asset group.
+ - dagster_fivetran_translator (Optional[[*DagsterFivetranTranslator*](#dagster_fivetran.DagsterFivetranTranslator)], optional) – The translator to use
+
+
+ Examples:
+
+ Sync the tables of a Fivetran connector:
+
+ ```python
+ from dagster_fivetran import FivetranWorkspace, fivetran_assets
+
+ import dagster as dg
+
+ fivetran_workspace = FivetranWorkspace(
+ account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"),
+ api_key=dg.EnvVar("FIVETRAN_API_KEY"),
+ api_secret=dg.EnvVar("FIVETRAN_API_SECRET"),
+ )
+
+ @fivetran_assets(
+ connector_id="fivetran_connector_id",
+ name="fivetran_connector_id",
+ group_name="fivetran_connector_id",
+ workspace=fivetran_workspace,
+ )
+ def fivetran_connector_assets(context: dg.AssetExecutionContext, fivetran: FivetranWorkspace):
+ yield from fivetran.sync_and_poll(context=context)
+
+ defs = dg.Definitions(
+ assets=[fivetran_connector_assets],
+ resources={"fivetran": fivetran_workspace},
+ )
+ ```
+ Sync the tables of a Fivetran connector with a custom translator:
+
+ ```python
+ from dagster_fivetran import (
+ DagsterFivetranTranslator,
+ FivetranConnectorTableProps,
+ FivetranWorkspace,
+ fivetran_assets
+ )
+
+ import dagster as dg
+ from dagster._core.definitions.asset_spec import replace_attributes
+
+ class CustomDagsterFivetranTranslator(DagsterFivetranTranslator):
+ def get_asset_spec(self, props: FivetranConnectorTableProps) -> dg.AssetSpec:
+ default_spec = super().get_asset_spec(props)
+ return default_spec.replace_attributes(
+ key=default_spec.key.with_prefix("my_prefix"),
+ )
+
+
+ fivetran_workspace = FivetranWorkspace(
+ account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"),
+ api_key=dg.EnvVar("FIVETRAN_API_KEY"),
+ api_secret=dg.EnvVar("FIVETRAN_API_SECRET"),
+ )
+
+ @fivetran_assets(
+ connector_id="fivetran_connector_id",
+ name="fivetran_connector_id",
+ group_name="fivetran_connector_id",
+ workspace=fivetran_workspace,
+ dagster_fivetran_translator=CustomDagsterFivetranTranslator(),
+ )
+ def fivetran_connector_assets(context: dg.AssetExecutionContext, fivetran: FivetranWorkspace):
+ yield from fivetran.sync_and_poll(context=context)
+
+ defs = dg.Definitions(
+ assets=[fivetran_connector_assets],
+ resources={"fivetran": fivetran_workspace},
+ )
+ ```
+
+
+
+
+
+
dagster_fivetran.load_fivetran_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Fivetran content in the workspace.
+
+ Parameters:
+ - workspace ([*FivetranWorkspace*](#dagster_fivetran.FivetranWorkspace)) – The Fivetran workspace to fetch assets from.
+ - dagster_fivetran_translator (Optional[[*DagsterFivetranTranslator*](#dagster_fivetran.DagsterFivetranTranslator)], optional) – The translator to use
+
+
+ Returns: The set of assets representing the Fivetran content in the workspace.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+ Examples:
+
+ Loading the asset specs for a given Fivetran workspace:
+
+ ```python
+ from dagster_fivetran import FivetranWorkspace, load_fivetran_asset_specs
+
+ import dagster as dg
+
+ fivetran_workspace = FivetranWorkspace(
+ account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"),
+ api_key=dg.EnvVar("FIVETRAN_API_KEY"),
+ api_secret=dg.EnvVar("FIVETRAN_API_SECRET"),
+ )
+
+ fivetran_specs = load_fivetran_asset_specs(fivetran_workspace)
+ defs = dg.Definitions(assets=[*fivetran_specs], resources={"fivetran": fivetran_workspace}
+ ```
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ The list of AssetsDefinition for all connectors in the Fivetran workspace.
+
+ Parameters:
+ - workspace ([*FivetranWorkspace*](#dagster_fivetran.FivetranWorkspace)) – The Fivetran workspace to fetch assets from.
+ - dagster_fivetran_translator (Optional[[*DagsterFivetranTranslator*](#dagster_fivetran.DagsterFivetranTranslator)], optional) – The translator to use
+
+
+ Returns: The list of AssetsDefinition for all connectors in the Fivetran workspace.Return type: List[[AssetsDefinition](../assets.mdx#dagster.AssetsDefinition)]
+ Examples:
+
+ Sync the tables of a Fivetran connector:
+
+ ```python
+ from dagster_fivetran import FivetranWorkspace, build_fivetran_assets_definitions
+
+ import dagster as dg
+
+ fivetran_workspace = FivetranWorkspace(
+ account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"),
+ api_key=dg.EnvVar("FIVETRAN_API_KEY"),
+ api_secret=dg.EnvVar("FIVETRAN_API_SECRET"),
+ )
+
+ fivetran_assets = build_fivetran_assets_definitions(workspace=workspace)
+
+ defs = dg.Definitions(
+ assets=[*fivetran_assets],
+ resources={"fivetran": fivetran_workspace},
+ )
+ ```
+ Sync the tables of a Fivetran connector with a custom translator:
+
+ ```python
+ from dagster_fivetran import (
+ DagsterFivetranTranslator,
+ FivetranConnectorTableProps,
+ FivetranWorkspace,
+ build_fivetran_assets_definitions
+ )
+
+ import dagster as dg
+ from dagster._core.definitions.asset_spec import replace_attributes
+
+ class CustomDagsterFivetranTranslator(DagsterFivetranTranslator):
+ def get_asset_spec(self, props: FivetranConnectorTableProps) -> dg.AssetSpec:
+ default_spec = super().get_asset_spec(props)
+ return default_spec.replace_attributes(
+ key=default_spec.key.with_prefix("my_prefix"),
+ )
+
+
+ fivetran_workspace = FivetranWorkspace(
+ account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"),
+ api_key=dg.EnvVar("FIVETRAN_API_KEY"),
+ api_secret=dg.EnvVar("FIVETRAN_API_SECRET"),
+ )
+
+ fivetran_assets = build_fivetran_assets_definitions(
+ workspace=workspace,
+ dagster_fivetran_translator=CustomDagsterFivetranTranslator()
+ )
+
+ defs = dg.Definitions(
+ assets=[*fivetran_assets],
+ resources={"fivetran": fivetran_workspace},
+ )
+ ```
+
+
+
+
+
+
class dagster_fivetran.fivetran_event_iterator.FivetranEventIterator
+
+
+ A wrapper around an iterator of Fivetran events which contains additional methods for
+ post-processing the events, such as fetching column metadata.
+
+
+
fetch_column_metadata
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Fetches column metadata for each table synced with the Fivetran API.
+
+ Retrieves the column schema for each destination table.
+
+ Returns: An iterator of Dagster events with column metadata attached.Return type: [FivetranEventIterator](#dagster_fivetran.fivetran_event_iterator.FivetranEventIterator)
+
+
+
+ This resource allows users to programatically interface with the Fivetran REST API to launch
+ syncs and monitor their progress. This currently implements only a subset of the functionality
+ exposed by the API.
+
+ For a complete set of documentation on the Fivetran REST API, including expected response JSON
+ schemae, see the [Fivetran API Docs](https://fivetran.com/docs/rest-api/connectors).
+
+ To configure this resource, we recommend using the [configured](https://legacy-docs.dagster.io/concepts/configuration/configured) method.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_fivetran import fivetran_resource
+
+ my_fivetran_resource = fivetran_resource.configured(
+ {
+ "api_key": {"env": "FIVETRAN_API_KEY"},
+ "api_secret": {"env": "FIVETRAN_API_SECRET"},
+ }
+ )
+
+ @job(resource_defs={"fivetran":my_fivetran_resource})
+ def my_fivetran_job():
+ ...
+ ```
+
+
+
+ Loads Fivetran connector assets from a configured FivetranResource instance. This fetches information
+ about defined connectors at initialization time, and will error on workspace load if the Fivetran
+ instance is not reachable.
+
+ Parameters:
+ - fivetran ([*ResourceDefinition*](../resources.mdx#dagster.ResourceDefinition)) – A FivetranResource configured with the appropriate connection
+ - key_prefix (Optional[CoercibleToAssetKeyPrefix]) – A prefix for the asset keys created.
+ - connector_to_group_fn (Optional[Callable[[str], Optional[str]]]) – Function which returns an asset
+ - io_manager_key (Optional[str]) – The IO manager key to use for all assets. Defaults to “io_manager”.
+ - connector_to_io_manager_key_fn (Optional[Callable[[str], Optional[str]]]) – Function which returns an
+ - connector_filter (Optional[Callable[[FivetranConnectorMetadata], bool]]) – Optional function which takes
+ - connector_to_asset_key_fn (Optional[Callable[[FivetranConnectorMetadata, str], [*AssetKey*](../assets.mdx#dagster.AssetKey)]]) – Optional function
+ - destination_ids (Optional[List[str]]) – A list of destination IDs to fetch connectors from. If None, all destinations
+ - poll_interval (float) – The time (in seconds) that will be waited between successive polls.
+ - poll_timeout (Optional[float]) – The maximum time that will waited before this operation is
+ - fetch_column_metadata (bool) – If True, will fetch column schema information for each table in the connector.
+
+
+ Examples:
+
+ Loading all Fivetran connectors as assets:
+
+ ```python
+ from dagster_fivetran import fivetran_resource, load_assets_from_fivetran_instance
+
+ fivetran_instance = fivetran_resource.configured(
+ {
+ "api_key": "some_key",
+ "api_secret": "some_secret",
+ }
+ )
+ fivetran_assets = load_assets_from_fivetran_instance(fivetran_instance)
+ ```
+ Filtering the set of loaded connectors:
+
+ ```python
+ from dagster_fivetran import fivetran_resource, load_assets_from_fivetran_instance
+
+ fivetran_instance = fivetran_resource.configured(
+ {
+ "api_key": "some_key",
+ "api_secret": "some_secret",
+ }
+ )
+ fivetran_assets = load_assets_from_fivetran_instance(
+ fivetran_instance,
+ connector_filter=lambda meta: "snowflake" in meta.name,
+ )
+ ```
+
+
+
+
+
+
+
dagster_fivetran.build_fivetran_assets
+
+
+ Build a set of assets for a given Fivetran connector.
+
+ Returns an AssetsDefinition which connects the specified `asset_keys` to the computation that
+ will update them. Internally, executes a Fivetran sync for a given `connector_id`, and
+ polls until that sync completes, raising an error if it is unsuccessful. Requires the use of the
+ [`fivetran_resource`](#dagster_fivetran.fivetran_resource), which allows it to communicate with the
+ Fivetran API.
+
+ Parameters:
+ - connector_id (str) – The Fivetran Connector ID that this op will sync. You can retrieve this
+ - destination_tables (List[str]) – schema_name.table_name for each table that you want to be
+ - poll_interval (float) – The time (in seconds) that will be waited between successive polls.
+ - poll_timeout (Optional[float]) – The maximum time that will waited before this operation is
+ - io_manager_key (Optional[str]) – The io_manager to be used to handle each of these assets.
+ - asset_key_prefix (Optional[List[str]]) – A prefix for the asset keys inside this asset.
+ - metadata_by_table_name (Optional[Mapping[str, RawMetadataMapping]]) – A mapping from destination
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. This
+ - infer_missing_tables (bool) – If True, will create asset materializations for tables specified
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset. Frameworks may expect and
+ - fetch_column_metadata (bool) – If True, will fetch column schema information for each table in the connector.
+
+
+ Examples:
+
+ Basic example:
+
+ >
+
+ ```python
+ from dagster import AssetKey, repository, with_resources
+
+ from dagster_fivetran import fivetran_resource
+ from dagster_fivetran.assets import build_fivetran_assets
+
+ my_fivetran_resource = fivetran_resource.configured(
+ {
+ "api_key": {"env": "FIVETRAN_API_KEY"},
+ "api_secret": {"env": "FIVETRAN_API_SECRET"},
+ }
+ )
+ ```
+
+
+Attaching metadata:
+
+>
+
+ ```python
+ fivetran_assets = build_fivetran_assets(
+ connector_id="foobar",
+ table_names=["schema1.table1", "schema2.table2"],
+ metadata_by_table_name={
+ "schema1.table1": {
+ "description": "This is a table that contains foo and bar",
+ },
+ "schema2.table2": {
+ "description": "This is a table that contains baz and quux",
+ },
+ },
+ )
+ ```
+
+
+
+
+
+
+
+
dagster_fivetran.fivetran_sync_op `=` \
+
+
+
+
+ Executes a Fivetran sync for a given `connector_id`, and polls until that sync
+ completes, raising an error if it is unsuccessful. It outputs a FivetranOutput which contains
+ the details of the Fivetran connector after the sync successfully completes, as well as details
+ about which tables the sync updates.
+
+ It requires the use of the [`fivetran_resource`](#dagster_fivetran.fivetran_resource), which allows it to
+ communicate with the Fivetran API.
+
+ Examples:
+
+ ```python
+ from dagster import job
+ from dagster_fivetran import fivetran_resource, fivetran_sync_op
+
+ my_fivetran_resource = fivetran_resource.configured(
+ {
+ "api_key": {"env": "FIVETRAN_API_KEY"},
+ "api_secret": {"env": "FIVETRAN_API_SECRET"},
+ }
+ )
+
+ sync_foobar = fivetran_sync_op.configured({"connector_id": "foobar"}, name="sync_foobar")
+
+ @job(resource_defs={"fivetran": my_fivetran_resource})
+ def my_simple_fivetran_job():
+ sync_foobar()
+
+ @job(resource_defs={"fivetran": my_fivetran_resource})
+ def my_composed_fivetran_job():
+ final_foobar_state = sync_foobar(start_after=some_op())
+ other_op(final_foobar_state)
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-gcp-pandas.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-gcp-pandas.mdx
new file mode 100644
index 0000000000000..3de869ce23903
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-gcp-pandas.mdx
@@ -0,0 +1,256 @@
+---
+title: 'gcp + pandas (dagster-gcp-pandas)'
+title_meta: 'gcp + pandas (dagster-gcp-pandas) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'gcp + pandas (dagster-gcp-pandas) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# GCP + Pandas (dagster-gcp-pandas)
+
+
+
+
+## Google BigQuery
+
+This library provides an integration with the [BigQuery](https://cloud.google.com/bigquery) database and Pandas data processing library.
+
+Related Guides:
+
+ - [Using Dagster with BigQuery](https://docs.dagster.io/integrations/libraries/gcp/bigquery/)
+ - [BigQuery I/O manager reference](https://docs.dagster.io/integrations/libraries/gcp/bigquery/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes pandas DataFrames to BigQuery.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_gcp_pandas import BigQueryPandasIOManager
+ from dagster import Definitions, EnvVar
+
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": BigQueryPandasIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": BigQueryPandasIOManager(project=EnvVar("GCP_PROJECT", dataset="my_dataset")
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the “gcp_credentials” configuration.
+ Dagster will store this key in a temporary file and set GOOGLE_APPLICATION_CREDENTIALS to point to the file.
+ After the run completes, the file will be deleted, and GOOGLE_APPLICATION_CREDENTIALS will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded key with this shell command: cat $GOOGLE_APPLICATION_CREDENTIALS | base64
+
+
+
+
+
+
+
class dagster_gcp_pandas.BigQueryPandasTypeHandler
+
+
+ Plugin for the BigQuery I/O Manager that can store and load Pandas DataFrames as BigQuery tables.
+
+ Examples:
+
+ ```python
+ from dagster_gcp import BigQueryIOManager
+ from dagster_bigquery_pandas import BigQueryPandasTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MyBigQueryIOManager(BigQueryIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [BigQueryPandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_dataset"] # my_dataset will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MyBigQueryIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes pandas DataFrames to BigQuery.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_gcp_pandas import bigquery_pandas_io_manager
+ from dagster import Definitions
+
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": bigquery_pandas_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ })
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": bigquery_pandas_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ "dataset": "my_dataset"
+ })
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the “gcp_credentials” configuration.
+ Dagster will store this key in a temporary file and set GOOGLE_APPLICATION_CREDENTIALS to point to the file.
+ After the run completes, the file will be deleted, and GOOGLE_APPLICATION_CREDENTIALS will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded key with this shell command: cat $GOOGLE_APPLICATION_CREDENTIALS | base64
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-gcp-pyspark.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-gcp-pyspark.mdx
new file mode 100644
index 0000000000000..297d09408566a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-gcp-pyspark.mdx
@@ -0,0 +1,256 @@
+---
+title: 'gcp + pyspark (dagster-gcp-pyspark)'
+title_meta: 'gcp + pyspark (dagster-gcp-pyspark) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'gcp + pyspark (dagster-gcp-pyspark) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# GCP + PySpark (dagster-gcp-pyspark)
+
+
+
+
+## Google BigQuery
+
+This library provides an integration with the [BigQuery](https://cloud.google.com/bigquery) database and PySpark data processing library.
+
+Related Guides:
+
+ - [Using Dagster with BigQuery](https://docs.dagster.io/integrations/libraries/gcp/bigquery/)
+ - [BigQuery I/O manager reference](https://docs.dagster.io/integrations/libraries/gcp/bigquery/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to BigQuery.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_gcp_pyspark import BigQueryPySparkIOManager
+ from dagster import Definitions, EnvVar
+
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pyspark.sql.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": BigQueryPySparkIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": BigQueryPySparkIOManager(project=EnvVar("GCP_PROJECT", dataset="my_dataset")
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pyspark.sql.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pyspark.sql.DataFrame) -> pyspark.sql.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the “gcp_credentials” configuration.
+ Dagster will store this key in a temporary file and set GOOGLE_APPLICATION_CREDENTIALS to point to the file.
+ After the run completes, the file will be deleted, and GOOGLE_APPLICATION_CREDENTIALS will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded key with this shell command: cat $GOOGLE_APPLICATION_CREDENTIALS | base64
+
+
+
+
+
+
+
class dagster_gcp_pyspark.BigQueryPySparkTypeHandler
+
+
+ Plugin for the BigQuery I/O Manager that can store and load PySpark DataFrames as BigQuery tables.
+
+ Examples:
+
+ ```python
+ from dagster_gcp import BigQueryIOManager
+ from dagster_bigquery_pandas import BigQueryPySparkTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MyBigQueryIOManager(BigQueryIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [BigQueryPySparkTypeHandler()]
+
+ @asset(
+ key_prefix=["my_dataset"] # my_dataset will be used as the dataset in BigQuery
+ )
+ def my_table() -> pyspark.sql.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MyBigQueryIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to BigQuery.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_gcp_pyspark import bigquery_pyspark_io_manager
+ from dagster import Definitions
+
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": bigquery_pyspark_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ })
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": bigquery_pandas_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ "dataset": "my_dataset"
+ })
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pyspark.sql.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pyspark.sql.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pyspark.sql.DataFrame) -> pyspark.sql.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the “gcp_credentials” configuration.
+ Dagster will store this key in a temporary file and set GOOGLE_APPLICATION_CREDENTIALS to point to the file.
+ After the run completes, the file will be deleted, and GOOGLE_APPLICATION_CREDENTIALS will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded key with this shell command: cat $GOOGLE_APPLICATION_CREDENTIALS | base64
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-gcp.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-gcp.mdx
new file mode 100644
index 0000000000000..5e30584bb9092
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-gcp.mdx
@@ -0,0 +1,820 @@
+---
+title: 'gcp (dagster-gcp)'
+title_meta: 'gcp (dagster-gcp) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'gcp (dagster-gcp) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+ Resource for interacting with Google BigQuery.
+
+ Examples:
+
+ ```python
+ from dagster import Definitions, asset
+ from dagster_gcp import BigQueryResource
+
+ @asset
+ def my_table(bigquery: BigQueryResource):
+ with bigquery.get_client() as client:
+ client.query("SELECT * FROM my_dataset.my_table")
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "bigquery": BigQueryResource(project="my-project")
+ }
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+### BigQuery I/O Manager
+
+
+
dagster_gcp.BigQueryIOManager IOManagerDefinition
+
+
+
+
+ Base class for an I/O manager definition that reads inputs from and writes outputs to BigQuery.
+
+ Examples:
+
+ ```python
+ from dagster_gcp import BigQueryIOManager
+ from dagster_bigquery_pandas import BigQueryPandasTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MyBigQueryIOManager(BigQueryIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [BigQueryPandasTypeHandler()]
+
+ @asset(
+ key_prefix=["my_dataset"] # my_dataset will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MyBigQueryIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MyBigQueryIOManager(project=EnvVar("GCP_PROJECT"), dataset="my_dataset")
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata `columns` to the
+ [`In`](../ops.mdx#dagster.In) or [`AssetIn`](../assets.mdx#dagster.AssetIn).
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the `gcp_credentials` configuration.
+ Dagster will store this key in a temporary file and set `GOOGLE_APPLICATION_CREDENTIALS` to point to the file.
+ After the run completes, the file will be deleted, and `GOOGLE_APPLICATION_CREDENTIALS` will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded with this shell command: `cat $GOOGLE_APPLICATION_CREDENTIALS | base64`
+
+
+
+
+
+
+
+
+
+
+
+### BigQuery Ops
+
+
+
dagster_gcp.bq_create_dataset
+
+
+ BigQuery Create Dataset.
+
+ This op encapsulates creating a BigQuery dataset.
+
+ Expects a BQ client to be provisioned in resources as context.resources.bigquery.
+
+
+
+
+
+
+
dagster_gcp.bq_delete_dataset
+
+
+ BigQuery Delete Dataset.
+
+ This op encapsulates deleting a BigQuery dataset.
+
+ Expects a BQ client to be provisioned in resources as context.resources.bigquery.
+
+
+
+
+
+
+
dagster_gcp.bq_op_for_queries
+
+
+ Executes BigQuery SQL queries.
+
+ Expects a BQ client to be provisioned in resources as context.resources.bigquery.
+
+
+
+
+
+
+
dagster_gcp.import_df_to_bq
+
+
+
+
+
+
+
dagster_gcp.import_file_to_bq
+
+
+
+
+
+
+
dagster_gcp.import_gcs_paths_to_bq
+
+
+
+
+
+
+
+
+
+
+
+### Data Freshness
+
+
+
dagster_gcp.fetch_last_updated_timestamps
+
+
+ Get the last updated timestamps of a list BigQuery table.
+
+ Note that this only works on BigQuery tables, and not views.
+
+ Parameters:
+ - client (bigquery.Client) – The BigQuery client.
+ - dataset_id (str) – The BigQuery dataset ID.
+ - table_ids (Sequence[str]) – The table IDs to get the last updated timestamp for.
+
+
+ Returns: A mapping of table IDs to their last updated timestamps (UTC).Return type: Mapping[str, datetime]
+
+
+
+
+
+
+
+
+
+
+### Other
+
+
+
class dagster_gcp.BigQueryError
+
+
+
+
+
+
+
+
+
+
+
+## GCS
+
+
+
+
+### GCS Resource
+
+
+
dagster_gcp.GCSResource ResourceDefinition
+
+
+
+
+ Resource for interacting with Google Cloud Storage.
+
+ Example:
+
+ ```default
+ @asset
+ def my_asset(gcs: GCSResource):
+ with gcs.get_client() as client:
+ # client is a google.cloud.storage.Client
+ ...
+ ```
+
+
+
+ Persistent IO manager using GCS for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for GCS and the backing bucket.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at `\/\`. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of `/my/base/path`, an asset with key
+ `AssetKey(["one", "two", "three"])` would be stored in a file called `three` in a directory
+ with path `/my/base/path/one/two/`.
+
+ Example usage:
+
+ 1. Attach this IO manager to a set of assets.
+ ```python
+ from dagster import asset, Definitions
+ from dagster_gcp.gcs import GCSPickleIOManager, GCSResource
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": GCSPickleIOManager(
+ gcs_bucket="my-cool-bucket",
+ gcs_prefix="my-cool-prefix",
+ gcs=GCSResource(project="my-cool-project")
+ ),
+
+ }
+ )
+ ```
+ 2. Attach this IO manager to your job to make it available to your ops.
+ ```python
+ from dagster import job
+ from dagster_gcp.gcs import GCSPickleIOManager, GCSResource
+
+ @job(
+ resource_defs={
+ "io_manager": GCSPickleIOManager(
+ gcs=GCSResource(project="my-cool-project")
+ gcs_bucket="my-cool-bucket",
+ gcs_prefix="my-cool-prefix"
+ ),
+ }
+ )
+ def my_job():
+ ...
+ ```
+
+
+
+
+
+
+
+
+
+
+### GCS Sensor
+
+
+
dagster_gcp.gcs.sensor.get_gcs_keys
+
+
+ Return a list of updated keys in a GCS bucket.
+
+ Parameters:
+ - bucket (str) – The name of the GCS bucket.
+ - prefix (Optional[str]) – The prefix to filter the keys by.
+ - since_key (Optional[str]) – The key to start from. If provided, only keys updated after this key will be returned.
+ - gcs_session (Optional[google.cloud.storage.client.Client]) – A GCS client session. If not provided, a new session will be created.
+
+
+ Returns: A list of keys in the bucket, sorted by update time, that are newer than the since_key.Return type: List[str]
+ Example:
+
+ ```python
+ @resource
+ def google_cloud_storage_client(context):
+ return storage.Client().from_service_account_json("my-service-account.json")
+
+ @sensor(job=my_job, required_resource_keys={"google_cloud_storage_client"})
+ def my_gcs_sensor(context):
+ since_key = context.cursor or None
+ new_gcs_keys = get_gcs_keys(
+ "my-bucket",
+ prefix="data",
+ since_key=since_key,
+ gcs_session=context.resources.google_cloud_storage_client
+ )
+
+ if not new_gcs_keys:
+ return SkipReason("No new gcs files found for bucket 'my-bucket'.")
+
+ for gcs_key in new_gcs_keys:
+ yield RunRequest(run_key=gcs_key, run_config={
+ "ops": {
+ "gcs_files": {
+ "config": {
+ "gcs_key": gcs_key
+ }
+ }
+ }
+ })
+
+ last_key = new_gcs_keys[-1]
+ context.update_cursor(last_key)
+ ```
+
+
+
+ FileManager that provides abstract access to GCS.
+
+
+
+
+
+
+
+
+
+
+
+### GCS Compute Log Manager
+
+
+
class dagster_gcp.gcs.GCSComputeLogManager
+
+
+ Logs op compute function stdout and stderr to GCS.
+
+ Users should not instantiate this class directly. Instead, use a YAML block in `dagster.yaml`
+ such as the following:
+
+ ```YAML
+ compute_logs:
+ module: dagster_gcp.gcs.compute_log_manager
+ class: GCSComputeLogManager
+ config:
+ bucket: "mycorp-dagster-compute-logs"
+ local_dir: "/tmp/cool"
+ prefix: "dagster-test-"
+ upload_interval: 30
+ ```
+ There are more configuration examples in the instance documentation guide: [https://docs.dagster.io/guides/deploy/dagster-instance-configuration#compute-log-storage](https://docs.dagster.io/guides/deploy/dagster-instance-configuration#compute-log-storage)
+
+ Parameters:
+ - bucket (str) – The name of the GCS bucket to which to log.
+ - local_dir (Optional[str]) – Path to the local directory in which to stage logs. Default:
+ - prefix (Optional[str]) – Prefix for the log file keys.
+ - json_credentials_envvar (Optional[str]) – Environment variable that contains the JSON with a private key
+ - upload_interval – (Optional[int]): Interval in seconds to upload partial log files to GCS. By default, will only upload when the capture is complete.
+ - show_url_only – (Optional[bool]): Only show the URL of the log file in the UI, instead of fetching and displaying the full content. Default False.
+ - inst_data (Optional[[*ConfigurableClassData*](../internals.mdx#dagster._serdes.ConfigurableClassData)]) – Serializable representation of the compute
+
+
+
+
+
+
+
+
+
+
+
+
+## Dataproc
+
+
+
+
+### Dataproc Resource
+
+
+
dagster_gcp.DataprocResource ResourceDefinition
+
+
+
+
+ Resource for connecting to a Dataproc cluster.
+
+ Example:
+
+ ```default
+ @asset
+ def my_asset(dataproc: DataprocResource):
+ with dataproc.get_client() as client:
+ # client is a dagster_gcp.DataprocClient
+ ...
+ ```
+
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Please use GCSPickleIOManager instead..
+
+ :::
+
+ Renamed to GCSPickleIOManager. See GCSPickleIOManager for documentation.
+
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Builds an I/O manager definition that reads inputs from and writes outputs to BigQuery.
+
+ Parameters:
+ - type_handlers (Sequence[DbTypeHandler]) – Each handler defines how to translate between
+ - default_load_type (Type) – When an input has no type annotation, load it as this type.
+
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_gcp import build_bigquery_io_manager
+ from dagster_bigquery_pandas import BigQueryPandasTypeHandler
+ from dagster import Definitions
+
+ @asset(
+ key_prefix=["my_prefix"],
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ @asset(
+ key_prefix=["my_dataset"] # my_dataset will be used as the dataset in BigQuery
+ )
+ def my_second_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ bigquery_io_manager = build_bigquery_io_manager([BigQueryPandasTypeHandler()])
+
+ defs = Definitions(
+ assets=[my_table, my_second_table],
+ resources={
+ "io_manager": bigquery_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ })
+ }
+ )
+ ```
+ You can set a default dataset to store the assets using the `dataset` configuration value of the BigQuery I/O
+ Manager. This dataset will be used if no other dataset is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": bigquery_io_manager.configured({
+ "project" : {"env": "GCP_PROJECT"}
+ "dataset": "my_dataset"
+ })
+ }
+ )
+ ```
+ On individual assets, you an also specify the dataset where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ # note that the key needs to be "schema"
+ metadata={"schema": "my_dataset"} # will be used as the dataset in BigQuery
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the dataset can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the dataset will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata `columns` to the
+ [`In`](../ops.mdx#dagster.In) or [`AssetIn`](../assets.mdx#dagster.AssetIn).
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the `gcp_credentials` configuration.
+ Dagster willstore this key in a temporary file and set `GOOGLE_APPLICATION_CREDENTIALS` to point to the file.
+ After the run completes, the file will be deleted, and `GOOGLE_APPLICATION_CREDENTIALS` will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded with this shell command: `cat $GOOGLE_APPLICATION_CREDENTIALS | base64`
+
+
+
+
+ Persistent IO manager using GCS for storage.
+
+ Serializes objects via pickling. Suitable for objects storage for distributed executors, so long
+ as each execution node has network connectivity and credentials for GCS and the backing bucket.
+
+ Assigns each op output to a unique filepath containing run ID, step key, and output name.
+ Assigns each asset to a single filesystem path, at `\/\`. If the asset key
+ has multiple components, the final component is used as the name of the file, and the preceding
+ components as parent directories under the base_dir.
+
+ Subsequent materializations of an asset will overwrite previous materializations of that asset.
+ With a base directory of `/my/base/path`, an asset with key
+ `AssetKey(["one", "two", "three"])` would be stored in a file called `three` in a directory
+ with path `/my/base/path/one/two/`.
+
+ Example usage:
+
+ 1. Attach this IO manager to a set of assets.
+ ```python
+ from dagster import Definitions, asset
+ from dagster_gcp.gcs import gcs_pickle_io_manager, gcs_resource
+
+ @asset
+ def asset1():
+ # create df ...
+ return df
+
+ @asset
+ def asset2(asset1):
+ return asset1[:5]
+
+ defs = Definitions(
+ assets=[asset1, asset2],
+ resources={
+ "io_manager": gcs_pickle_io_manager.configured(
+ {"gcs_bucket": "my-cool-bucket", "gcs_prefix": "my-cool-prefix"}
+ ),
+ "gcs": gcs_resource.configured({"project": "my-cool-project"}),
+ },
+ )
+ ```
+ 2. Attach this IO manager to your job to make it available to your ops.
+ ```python
+ from dagster import job
+ from dagster_gcp.gcs import gcs_pickle_io_manager, gcs_resource
+
+ @job(
+ resource_defs={
+ "io_manager": gcs_pickle_io_manager.configured(
+ {"gcs_bucket": "my-cool-bucket", "gcs_prefix": "my-cool-prefix"}
+ ),
+ "gcs": gcs_resource.configured({"project": "my-cool-project"}),
+ },
+ )
+ def my_job():
+ ...
+ ```
+
+
+
+
+
+
dagster_gcp.gcs_file_manager ResourceDefinition
+
+
+ FileManager that provides abstract access to GCS.
+
+ Implements the [`FileManager`](../internals.mdx#dagster._core.storage.file_manager.FileManager) API.
+
+
+
+
+
+
+
dagster_gcp.dataproc_resource ResourceDefinition
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-ge.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-ge.mdx
new file mode 100644
index 0000000000000..8100692178ada
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-ge.mdx
@@ -0,0 +1,55 @@
+---
+title: 'great expectations (dagster-ge)'
+title_meta: 'great expectations (dagster-ge) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'great expectations (dagster-ge) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+# Great Expectations (dagster-ge)
+
+
+
+
dagster_ge.ge_validation_op_factory
+
+ Generates ops for interacting with Great Expectations.
+
+ Parameters:
+
+ - - name (str) – the name of the op
+ - datasource_name (str) – the name of your DataSource, see your great_expectations.yml
+ - data_connector_name (str) – the name of the data connector for this datasource. This should
+ - data_asset_name (str) – the name of the data asset that this op will be validating.
+ - suite_name (str) – the name of your expectation suite, see your great_expectations.yml
+ - batch_identifier_fn (dict) – A dicitonary of batch identifiers to uniquely identify this
+ - input_dagster_type ([*DagsterType*](../types.mdx#dagster.DagsterType)) – the Dagster type used to type check the input to the op.
+ - runtime_method_type (str) – how GE should interperet the op input. One of (“batch_data”,
+ extra_kwargs (Optional[dict]) –
+
+ adds extra kwargs to the invocation of ge_data_context’s
+ get_validator method. If not set, input will be:
+
+ >
+
+ ```default
+ {
+ "datasource_name": datasource_name,
+ "data_connector_name": data_connector_name,
+ "data_asset_name": data_asset_name,
+ "runtime_parameters": {
+ "":
+ },
+ "batch_identifiers": batch_identifiers,
+ "expectation_suite_name": suite_name,
+ }
+ ```
+
+
+ Returns: An op that takes in a set of data and yields both an expectation with relevant metadata and
+ an output with all the metadata (for user processing)
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-github.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-github.mdx
new file mode 100644
index 0000000000000..47e4cb0e0281b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-github.mdx
@@ -0,0 +1,417 @@
+---
+title: 'github (dagster-github)'
+title_meta: 'github (dagster-github) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'github (dagster-github) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# GitHub (dagster-github)
+
+This library provides an integration with GitHub Apps, to support performing various automation
+operations within your github repositories and with the tighter permissions scopes that github apps
+allow for vs using a personal token.
+
+Presently, it provides a thin wrapper on the [github v4 graphql API](https://developer.github.com/v4/).
+
+To use this integration, you’ll first need to create a GitHub App for it.
+
+1. Create App: Follow the instructions in
+ [https://developer.github.com/apps/quickstart-guides/setting-up-your-development-environment/](https://developer.github.com/apps/quickstart-guides/setting-up-your-development-environment/), You will end up with a private key and App ID, which will be used when configuring the
+ `dagster-github` resource. Note you will need to grant your app the relevent permissions
+ for the API requests you want to make, for example to post issues it will need read/write access
+ for the issues repository permission, more info on GitHub application permissions can be found
+ [here](https://developer.github.com/v3/apps/permissions/)
+2. Install App: Follow the instructions in
+ [https://developer.github.com/apps/quickstart-guides/setting-up-your-development-environment/#step-7-install-the-app-on-your-account](https://developer.github.com/apps/quickstart-guides/setting-up-your-development-environment/#step-7-install-the-app-on-your-account)
+3. Find your installation_id: You can pull this from the GitHub app administration page,
+ `https://github.com/apps//installations/`. Note if your app is
+ installed more than once you can also programatically retrieve these IDs.
+Sharing your App ID and Installation ID is fine, but make sure that the Private Key for your app is
+stored securily.
+
+
+
+
+## Posting Issues
+
+Now, you can create issues in GitHub from Dagster with the GitHub resource:
+
+ ```python
+ import os
+
+ from dagster import job, op
+ from dagster_github import GithubResource
+
+
+ @op
+ def github_op(github: GithubResource):
+ github.get_client().create_issue(
+ repo_name='dagster',
+ repo_owner='dagster-io',
+ title='Dagster\'s first github issue',
+ body='this open source thing seems like a pretty good idea',
+ )
+
+ @job(resource_defs={
+ 'github': GithubResource(
+ github_app_id=os.getenv('GITHUB_APP_ID'),
+ github_app_private_rsa_key=os.getenv('GITHUB_PRIVATE_KEY'),
+ github_installation_id=os.getenv('GITHUB_INSTALLATION_ID')
+ )})
+ def github_job():
+ github_op()
+
+ github_job.execute_in_process()
+ ```
+Run the above code, and you’ll see the issue appear in GitHub:
+
+GitHub enterprise users can provide their hostname in the run config. Provide `github_hostname`
+as part of your github config like below.
+
+ ```python
+ GithubResource(
+ github_app_id=os.getenv('GITHUB_APP_ID'),
+ github_app_private_rsa_key=os.getenv('GITHUB_PRIVATE_KEY'),
+ github_installation_id=os.getenv('GITHUB_INSTALLATION_ID'),
+ github_hostname=os.getenv('GITHUB_HOSTNAME'),
+ )
+ ```
+By provisioning `GithubResource` as a Dagster resource, you can post to GitHub from
+within any asset or op execution.
+
+
+
+ A client for interacting with the GitHub API.
+
+ This client handles authentication and provides methods for making requests
+ to the GitHub API using an authenticated session.
+
+
+
client
+
+
+ The HTTP session used for making requests.
+
+ Type: requests.Session
+
+
+
+
+
+
app_id
+
+
+ The GitHub App ID.
+
+ Type: int
+
+
+
+
+
+
app_private_rsa_key
+
+
+ The private RSA key for the GitHub App.
+
+ Type: str
+
+
+
+
+
+
default_installation_id
+
+
+ The default installation ID for the GitHub App.
+
+ Type: Optional[int]
+
+
+
+
+
+
hostname
+
+
+ The GitHub hostname, defaults to None.
+
+ Type: Optional[str]
+
+
+
+
+
+
installation_tokens
+
+
+ A dictionary to store installation tokens.
+
+ Type: Dict[Any, Any]
+
+
+
+
+
+
app_token
+
+
+ A dictionary to store the app token.
+
+ Type: Dict[str, Any]
+
+
+
+
+
+
create_issue
+
+
+ Create a new issue in the specified GitHub repository.
+
+ This method first retrieves the repository ID using the provided repository name
+ and owner, then creates a new issue in that repository with the given title and body.
+
+ Parameters:
+ - repo_name (str) – The name of the repository where the issue will be created.
+ - repo_owner (str) – The owner of the repository where the issue will be created.
+ - title (str) – The title of the issue.
+ - body (str) – The body content of the issue.
+ - installation_id (Optional[int]) – The installation ID to use for authentication.
+
+
+ Returns: The response data from the GitHub API containing the created issue details.Return type: Dict[str, Any]Raises: RuntimeError – If there are errors in the response from the GitHub API.
+
+
+
+
+
+
create_pull_request
+
+
+ Create a new pull request in the specified GitHub repository.
+
+ This method creates a pull request from the head reference (branch) to the base reference (branch)
+ in the specified repositories. It uses the provided title and body for the pull request description.
+
+ Parameters:
+ - base_repo_name (str) – The name of the base repository where the pull request will be created.
+ - base_repo_owner (str) – The owner of the base repository.
+ - base_ref_name (str) – The name of the base reference (branch) to which the changes will be merged.
+ - head_repo_name (str) – The name of the head repository from which the changes will be taken.
+ - head_repo_owner (str) – The owner of the head repository.
+ - head_ref_name (str) – The name of the head reference (branch) from which the changes will be taken.
+ - title (str) – The title of the pull request.
+ - body (Optional[str]) – The body content of the pull request. Defaults to None.
+ - maintainer_can_modify (Optional[bool]) – Whether maintainers can modify the pull request. Defaults to None.
+ - draft (Optional[bool]) – Whether the pull request is a draft. Defaults to None.
+ - installation_id (Optional[int]) – The installation ID to use for authentication.
+
+
+ Returns: The response data from the GitHub API containing the created pull request details.Return type: Dict[str, Any]Raises: RuntimeError – If there are errors in the response from the GitHub API.
+
+
+
+
+
+
create_ref
+
+
+ Create a new reference (branch) in the specified GitHub repository.
+
+ This method first retrieves the repository ID and the source reference (branch or tag)
+ using the provided repository name, owner, and source reference. It then creates a new
+ reference (branch) in that repository with the given target name.
+
+ Parameters:
+ - repo_name (str) – The name of the repository where the reference will be created.
+ - repo_owner (str) – The owner of the repository where the reference will be created.
+ - source (str) – The source reference (branch or tag) from which the new reference will be created.
+ - target (str) – The name of the new reference (branch) to be created.
+ - installation_id (Optional[int]) – The installation ID to use for authentication.
+
+
+ Returns: The response data from the GitHub API containing the created reference details.Return type: Dict[str, Any]Raises: RuntimeError – If there are errors in the response from the GitHub API.
+
+
+
+
+
+
execute
+
+
+ Execute a GraphQL query against the GitHub API.
+
+ This method sends a POST request to the GitHub API with the provided GraphQL query
+ and optional variables. It ensures that the appropriate installation token is included
+ in the request headers.
+
+ Parameters:
+ - query (str) – The GraphQL query string to be executed.
+ - variables (Optional[Dict[str, Any]]) – Optional variables to include in the query.
+ - headers (Optional[Dict[str, Any]]) – Optional headers to include in the request.
+ - installation_id (Optional[int]) – The installation ID to use for authentication.
+
+
+ Returns: The response data from the GitHub API.Return type: Dict[str, Any]Raises:
+ - RuntimeError – If no installation ID is provided and no default installation ID is set.
+ - requests.exceptions.HTTPError – If the request to the GitHub API fails.
+
+
+
+
+
+
+
+
get_installations
+
+
+ Retrieve the list of installations for the authenticated GitHub App.
+
+ This method makes a GET request to the GitHub API to fetch the installations
+ associated with the authenticated GitHub App. It ensures that the app token
+ is valid and includes it in the request headers.
+
+ Parameters: headers (Optional[Dict[str, Any]]) – Optional headers to include in the request.Returns: A dictionary containing the installations data.Return type: Dict[str, Any]Raises: requests.exceptions.HTTPError – If the request to the GitHub API fails.
+
+
+
+ A resource configuration class for GitHub integration.
+
+ This class provides configuration fields for setting up a GitHub Application,
+ including the application ID, private RSA key, installation ID, and hostname.
+
+
+
dagster_github.resources.github_app_id
+
+
+ The GitHub Application ID. For more information, see
+ [https://developer.github.com/apps/](https://developer.github.com/apps/).
+
+ Type: int
+
+
+
+ The private RSA key text for the GitHub Application.
+ For more information, see [https://developer.github.com/apps/](https://developer.github.com/apps/).
+
+ Type: str
+
+
+
+
+
+
dagster_github.resources.github_installation_id
+
+
+ The GitHub Application Installation ID.
+ Defaults to None. For more information, see [https://developer.github.com/apps/](https://developer.github.com/apps/).
+
+ Type: Optional[int]
+
+
+
+
+
+
dagster_github.resources.github_hostname
+
+
+ The GitHub hostname. Defaults to api.github.com.
+ For more information, see [https://developer.github.com/apps/](https://developer.github.com/apps/).
+
+ Type: Optional[str]
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-graphql.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-graphql.mdx
new file mode 100644
index 0000000000000..5886dbf4cd5aa
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-graphql.mdx
@@ -0,0 +1,180 @@
+---
+title: 'graphql (dagster-graphql)'
+title_meta: 'graphql (dagster-graphql) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'graphql (dagster-graphql) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# GraphQL (dagster-graphql)
+
+
+
+## Python Client
+
+
+
class dagster_graphql.DagsterGraphQLClient
+
+
+ Official Dagster Python Client for GraphQL.
+
+ Utilizes the gql library to dispatch queries over HTTP to a remote Dagster GraphQL Server
+
+ As of now, all operations on this client are synchronous.
+
+ Intended usage:
+
+ ```python
+ client = DagsterGraphQLClient("localhost", port_number=3000)
+ status = client.get_run_status(**SOME_RUN_ID**)
+ ```
+ Parameters:
+ - hostname (str) – Hostname for the Dagster GraphQL API, like localhost or
+ - port_number (Optional[int]) – Port number to connect to on the host.
+ - transport (Optional[Transport], optional) – A custom transport to use to connect to the
+ - use_https (bool, optional) – Whether to use https in the URL connection string for the
+ - timeout (int) – Number of seconds before requests should time out. Defaults to 60.
+ - headers (Optional[Dict[str, str]]) – Additional headers to include in the request. To use
+
+
+ Raises: ConnectionError – if the client cannot connect to the host.
+
+
get_run_status
+
+
+ Get the status of a given Pipeline Run.
+
+ Parameters: run_id (str) – run id of the requested pipeline run.Raises:
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("PipelineNotFoundError", message) – if the requested run id is not found
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("PythonError", message) – on internal framework errors
+
+
+ Returns: returns a status Enum describing the state of the requested pipeline runReturn type: [DagsterRunStatus](../internals.mdx#dagster.DagsterRunStatus)
+
+
+
+
+
+
reload_repository_location
+
+
+ Reloads a Dagster Repository Location, which reloads all repositories in that repository location.
+
+ This is useful in a variety of contexts, including refreshing the Dagster UI without restarting
+ the server.
+
+ Parameters: repository_location_name (str) – The name of the repository locationReturns: Object with information about the result of the reload requestReturn type: [ReloadRepositoryLocationInfo](#dagster_graphql.ReloadRepositoryLocationInfo)
+
+
+
+
+
+
shutdown_repository_location
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+
+
+ :::
+
+ Shuts down the server that is serving metadata for the provided repository location.
+
+ This is primarily useful when you want the server to be restarted by the compute environment
+ in which it is running (for example, in Kubernetes, the pod in which the server is running
+ will automatically restart when the server is shut down, and the repository metadata will
+ be reloaded)
+
+ Parameters: repository_location_name (str) – The name of the repository locationReturns: Object with information about the result of the reload requestReturn type: ShutdownRepositoryLocationInfo
+
+
+
+
+
+
submit_job_execution
+
+
+ Submits a job with attached configuration for execution.
+
+ Parameters:
+ - job_name (str) – The job’s name
+ - repository_location_name (Optional[str]) – The name of the repository location where
+ - repository_name (Optional[str]) – The name of the repository where the job is located.
+ - run_config (Optional[Union[[*RunConfig*](../config.mdx#dagster.RunConfig), Mapping[str, Any]]]) – This is the run config to execute the job with.
+ - tags (Optional[Dict[str, Any]]) – A set of tags to add to the job execution.
+
+
+ Raises:
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("InvalidStepError", invalid_step_key) – the job has an invalid step
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("InvalidOutputError", body=error_object) – some solid has an invalid output within the job.
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("RunConflict", message) – a DagsterRunConflict occured during execution.
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("PipelineConfigurationInvalid", invalid_step_key) – the run_config is not in the expected format
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("JobNotFoundError", message) – the requested job does not exist
+ - [DagsterGraphQLClientError](#dagster_graphql.DagsterGraphQLClientError)DagsterGraphQLClientError("PythonError", message) – an internal framework error occurred
+
+
+ Returns: run id of the submitted pipeline runReturn type: str
+
+
+
+ This class gives information about an InvalidOutputError from submitting a pipeline for execution
+ from GraphQL.
+
+ Parameters:
+ - step_key (str) – key of the step that failed
+ - invalid_output_name (str) – the name of the invalid output from the given step
+
+
+
+
+
+
+
+
class dagster_graphql.ReloadRepositoryLocationInfo
+
+
+ This class gives information about the result of reloading
+ a Dagster repository location with a GraphQL mutation.
+
+ Parameters:
+ - status ([*ReloadRepositoryLocationStatus*](#dagster_graphql.ReloadRepositoryLocationStatus)) – The status of the reload repository location mutation
+ - failure_type – (Optional[str], optional): the failure type if status == ReloadRepositoryLocationStatus.FAILURE.
+ - message (Optional[str], optional) – the failure message/reason if
+
+
+
+
+
+
+
+
class dagster_graphql.ReloadRepositoryLocationStatus
+
+
+ This enum describes the status of a GraphQL mutation to reload a Dagster repository location.
+
+ Parameters: Enum (str) – can be either ReloadRepositoryLocationStatus.SUCCESS
+ or ReloadRepositoryLocationStatus.FAILURE.
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-k8s.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-k8s.mdx
new file mode 100644
index 0000000000000..f371c6932843e
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-k8s.mdx
@@ -0,0 +1,551 @@
+---
+title: 'kubernetes (dagster-k8s)'
+title_meta: 'kubernetes (dagster-k8s) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'kubernetes (dagster-k8s) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Kubernetes (dagster-k8s)
+
+See also the [Kubernetes deployment guide](https://docs.dagster.io/deploying/kubernetes/).
+
+This library contains utilities for running Dagster with Kubernetes. This includes a Python API
+allowing the webserver to launch runs as Kubernetes Jobs, as well as a Helm chart you can use as the basis
+for a Dagster deployment on a Kubernetes cluster.
+
+
+
+
+
+
+
+# APIs
+
+
+
dagster_k8s.K8sRunLauncher RunLauncher
+
+
+
+
+ RunLauncher that starts a Kubernetes Job for each Dagster job run.
+
+ Encapsulates each run in a separate, isolated invocation of `dagster-graphql`.
+
+ You can configure a Dagster instance to use this RunLauncher by adding a section to your
+ `dagster.yaml` like the following:
+
+ ```yaml
+ run_launcher:
+ module: dagster_k8s.launcher
+ class: K8sRunLauncher
+ config:
+ service_account_name: your_service_account
+ job_image: my_project/dagster_image:latest
+ instance_config_map: dagster-instance
+ postgres_password_secret: dagster-postgresql-secret
+ ```
+
+
+
+
+
+
dagster_k8s.k8s_job_executor ExecutorDefinition
+
+
+
+
+ Executor which launches steps as Kubernetes Jobs.
+
+ To use the k8s_job_executor, set it as the executor_def when defining a job:
+
+ ```python
+ from dagster_k8s import k8s_job_executor
+
+ from dagster import job
+
+ @job(executor_def=k8s_job_executor)
+ def k8s_job():
+ pass
+ ```
+ Then you can configure the executor with run config as follows:
+
+ ```YAML
+ execution:
+ config:
+ job_namespace: 'some-namespace'
+ image_pull_policy: ...
+ image_pull_secrets: ...
+ service_account_name: ...
+ env_config_maps: ...
+ env_secrets: ...
+ env_vars: ...
+ job_image: ... # leave out if using userDeployments
+ max_concurrent: ...
+ ```
+ max_concurrent limits the number of pods that will execute concurrently for one run. By default
+ there is no limit- it will maximally parallel as allowed by the DAG. Note that this is not a
+ global limit.
+
+ Configuration set on the Kubernetes Jobs and Pods created by the K8sRunLauncher will also be
+ set on Kubernetes Jobs and Pods created by the k8s_job_executor.
+
+ Configuration set using tags on a @job will only apply to the run level. For configuration
+ to apply at each step it must be set using tags for each @op.
+
+
+
+
+
+
+
+
+## Ops
+
+
+
dagster_k8s.k8s_job_op `=` \
+
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ An op that runs a Kubernetes job using the k8s API.
+
+ Contrast with the k8s_job_executor, which runs each Dagster op in a Dagster job in its
+ own k8s job.
+
+ This op may be useful when:
+ - You need to orchestrate a command that isn’t a Dagster op (or isn’t written in Python)
+ - You want to run the rest of a Dagster job using a specific executor, and only a single
+
+
+ For example:
+
+ ```python
+ from dagster_k8s import k8s_job_op
+
+ from dagster import job
+
+ first_op = k8s_job_op.configured(
+ {
+ "image": "busybox",
+ "command": ["/bin/sh", "-c"],
+ "args": ["echo HELLO"],
+ },
+ name="first_op",
+ )
+ second_op = k8s_job_op.configured(
+ {
+ "image": "busybox",
+ "command": ["/bin/sh", "-c"],
+ "args": ["echo GOODBYE"],
+ },
+ name="second_op",
+ )
+
+ @job
+ def full_job():
+ second_op(first_op())
+ ```
+ You can create your own op with the same implementation by calling the execute_k8s_job function
+ inside your own op.
+
+ The service account that is used to run this job should have the following RBAC permissions:
+
+ ```YAML
+ rules:
+ - apiGroups: ["batch"]
+ resources: ["jobs", "jobs/status"]
+ verbs: ["*"]
+ # The empty arg "" corresponds to the core API group
+ - apiGroups: [""]
+ resources: ["pods", "pods/log", "pods/status"]
+ verbs: ["*"]'
+ ```
+
+
+
+
+
+
dagster_k8s.execute_k8s_job
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This function is a utility for executing a Kubernetes job from within a Dagster op.
+
+ Parameters:
+ - image (str) – The image in which to launch the k8s job.
+ - command (Optional[List[str]]) – The command to run in the container within the launched
+ - args (Optional[List[str]]) – The args for the command for the container. Default: None.
+ - namespace (Optional[str]) – Override the kubernetes namespace in which to run the k8s job.
+ - image_pull_policy (Optional[str]) – Allows the image pull policy to be overridden, e.g. to
+ - image_pull_secrets (Optional[List[Dict[str, str]]]) – Optionally, a list of dicts, each of
+ - service_account_name (Optional[str]) – The name of the Kubernetes service account under which
+ - env_secrets (Optional[List[str]]) – A list of custom Secret names from which to
+ - env_vars (Optional[List[str]]) – A list of environment variables to inject into the Job.
+ - volume_mounts (Optional[List[[*Permissive*](../config.mdx#dagster.Permissive)]]) – A list of volume mounts to include in the job’s
+ - volumes (Optional[List[[*Permissive*](../config.mdx#dagster.Permissive)]]) – A list of volumes to include in the Job’s Pod. Default: `[]`. See:
+ - labels (Optional[Dict[str, str]]) – Additional labels that should be included in the Job’s Pod. See:
+ - resources (Optional[Dict[str, Any]]) – [https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/](https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/)
+ - scheduler_name (Optional[str]) – Use a custom Kubernetes scheduler for launched Pods. See:
+ - load_incluster_config (bool) – Whether the op is running within a k8s cluster. If `True`,
+ - kubeconfig_file (Optional[str]) – The kubeconfig file from which to load config. Defaults to
+ - timeout (Optional[int]) – Raise an exception if the op takes longer than this timeout in
+ - container_config (Optional[Dict[str, Any]]) – Raw k8s config for the k8s pod’s main container
+ - pod_template_spec_metadata (Optional[Dict[str, Any]]) – Raw k8s config for the k8s pod’s
+ - pod_spec_config (Optional[Dict[str, Any]]) – Raw k8s config for the k8s pod’s pod spec
+ - job_metadata (Optional[Dict[str, Any]]) – Raw k8s config for the k8s job’s metadata
+ - job_spec_config (Optional[Dict[str, Any]]) – Raw k8s config for the k8s job’s job spec
+ - k8s_job_name (Optional[str]) – Overrides the name of the k8s job. If not set, will be set
+ - merge_behavior (Optional[K8sConfigMergeBehavior]) – How raw k8s config set on this op should
+ - delete_failed_k8s_jobs (bool) – Whether to immediately delete failed Kubernetes jobs. If False,
+
+
+
+
+
+
+
+
+
+### Python API
+
+The `K8sRunLauncher` allows webserver instances to be configured to launch new runs by starting
+per-run Kubernetes Jobs. To configure the `K8sRunLauncher`, your `dagster.yaml` should
+include a section like:
+
+ ```yaml
+ run_launcher:
+ module: dagster_k8s.launcher
+ class: K8sRunLauncher
+ config:
+ image_pull_secrets:
+ service_account_name: dagster
+ job_image: "my-company.com/image:latest"
+ dagster_home: "/opt/dagster/dagster_home"
+ postgres_password_secret: "dagster-postgresql-secret"
+ image_pull_policy: "IfNotPresent"
+ job_namespace: "dagster"
+ instance_config_map: "dagster-instance"
+ env_config_maps:
+ - "dagster-k8s-job-runner-env"
+ env_secrets:
+ - "dagster-k8s-some-secret"
+ ```
+
+
+
+
+
+
+### Helm chart
+
+For local dev (e.g., on kind or minikube):
+
+ ```shell
+ helm install \
+ --set dagsterWebserver.image.repository="dagster.io/buildkite-test-image" \
+ --set dagsterWebserver.image.tag="py310-latest" \
+ --set job_runner.image.repository="dagster.io/buildkite-test-image" \
+ --set job_runner.image.tag="py310-latest" \
+ --set imagePullPolicy="IfNotPresent" \
+ dagster \
+ helm/dagster/
+ ```
+Upon installation, the Helm chart will provide instructions for port forwarding
+the Dagster webserver and Flower (if configured).
+
+
+
+
+
+
+
+### Running tests
+
+To run the unit tests:
+
+ ```default
+ pytest -m "not integration"
+ ```
+To run the integration tests, you must have [Docker](https://docs.docker.com/install/),
+[kind](https://kind.sigs.k8s.io/docs/user/quick-start#installation),
+and [helm](https://helm.sh/docs/intro/install/) installed.
+
+On macOS:
+
+ ```default
+ brew install kind
+ brew install helm
+ ```
+Docker must be running.
+
+You may experience slow first test runs thanks to image pulls (run `pytest -svv --fulltrace` for
+visibility). Building images and loading them to the kind cluster is slow, and there is
+no visibility into the progress of the load.
+
+NOTE: This process is quite slow, as it requires bootstrapping a local `kind` cluster with
+Docker images and the `dagster-k8s` Helm chart. For faster development, you can either:
+
+1. Keep a warm kind cluster
+2. Use a remote K8s cluster, e.g. via AWS EKS or GCP GKE
+Instructions are below.
+
+
+
+
+#### Faster local development (with kind)
+
+You may find that the kind cluster creation, image loading, and kind cluster creation loop
+is too slow for effective local dev.
+
+You may bypass cluster creation and image loading in the following way. First add the `--no-cleanup`
+flag to your pytest invocation:
+
+ ```shell
+ pytest --no-cleanup -s -vvv -m "not integration"
+ ```
+The tests will run as before, but the kind cluster will be left running after the tests are completed.
+
+For subsequent test runs, you can run:
+
+ ```shell
+ pytest --kind-cluster="cluster-d9971c84d44d47f382a2928c8c161faa" --existing-helm-namespace="dagster-test-95590a" -s -vvv -m "not integration"
+ ```
+This will bypass cluster creation, image loading, and Helm chart installation, for much faster tests.
+
+The kind cluster name and Helm namespace for this command can be found in the logs, or retrieved
+via the respective CLIs, using `kind get clusters` and `kubectl get namespaces`. Note that
+for `kubectl` and `helm` to work correctly with a kind cluster, you should override your
+kubeconfig file location with:
+
+ ```shell
+ kind get kubeconfig --name kind-test > /tmp/kubeconfig
+ export KUBECONFIG=/tmp/kubeconfig
+ ```
+
+
+
+
+
+
+#### Manual kind cluster setup
+
+The test fixtures provided by `dagster-k8s` automate the process described below, but sometimes
+it’s useful to manually configure a kind cluster and load images onto it.
+
+First, ensure you have a Docker image appropriate for your Python version. Run, from the root of
+the repo:
+
+ ```shell
+ ./python_modules/dagster-test/dagster_test/test_project/build.sh 3.7.6 \
+ dagster.io.priv/buildkite-test-image:py310-latest
+ ```
+In the above invocation, the Python majmin version should be appropriate for your desired tests.
+
+Then run the following commands to create the cluster and load the image. Note that there is no
+feedback from the loading process.
+
+ ```shell
+ kind create cluster --name kind-test
+ kind load docker-image --name kind-test dagster.io/dagster-docker-buildkite:py310-latest
+ ```
+If you are deploying the Helm chart with an in-cluster Postgres (rather than an external database),
+and/or with dagster-celery workers (and a RabbitMQ), you’ll also want to have images present for
+rabbitmq and postgresql:
+
+ ```shell
+ docker pull docker.io/bitnami/rabbitmq
+ docker pull docker.io/bitnami/postgresql
+
+ kind load docker-image --name kind-test docker.io/bitnami/rabbitmq:latest
+ kind load docker-image --name kind-test docker.io/bitnami/postgresql:latest
+ ```
+Then you can run pytest as follows:
+
+ ```shell
+ pytest --kind-cluster=kind-test
+ ```
+
+
+
+
+
+
+### Faster local development (with an existing K8s cluster)
+
+If you already have a development K8s cluster available, you can run tests on that cluster vs.
+running locally in `kind`.
+
+For this to work, first build and deploy the test image to a registry available to your cluster.
+For example, with a private ECR repository:
+
+ ```default
+ ./python_modules/dagster-test/dagster_test/test_project/build.sh 3.7.6
+ docker tag dagster-docker-buildkite:latest $AWS_ACCOUNT_ID.dkr.ecr.us-west-2.amazonaws.com/dagster-k8s-tests:2020-04-21T21-04-06
+
+ aws ecr get-login --no-include-email --region us-west-1 | sh
+ docker push $AWS_ACCOUNT_ID.dkr.ecr.us-west-1.amazonaws.com/dagster-k8s-tests:2020-04-21T21-04-06
+ ```
+Then, you can run tests on EKS with:
+
+ ```default
+ export DAGSTER_DOCKER_IMAGE_TAG="2020-04-21T21-04-06"
+ export DAGSTER_DOCKER_REPOSITORY="$AWS_ACCOUNT_ID.dkr.ecr.us-west-2.amazonaws.com"
+ export DAGSTER_DOCKER_IMAGE="dagster-k8s-tests"
+
+ # First run with --no-cleanup to leave Helm chart in place
+ pytest --cluster-provider="kubeconfig" --no-cleanup -s -vvv
+
+ # Subsequent runs against existing Helm chart
+ pytest --cluster-provider="kubeconfig" --existing-helm-namespace="dagster-test-" -s -vvv
+ ```
+
+
+
+### A note about PVCs
+
+Both the Postgres and the RabbitMQ Helm charts will store credentials using Persistent Volume
+Claims, which will outlive test invocations and calls to `helm uninstall`. These must be deleted if
+you want to change credentials. To view your pvcs, run:
+
+ ```default
+ kubectl get pvc
+ ```
+
+
+
+
+
+
+### Testing Redis
+
+The Redis Helm chart installs w/ a randomly-generated password by default; turn this off:
+
+ ```default
+ helm install dagredis stable/redis --set usePassword=false
+ ```
+Then, to connect to your database from outside the cluster execute the following commands:
+
+ ```default
+ kubectl port-forward --namespace default svc/dagredis-master 6379:6379
+ redis-cli -h 127.0.0.1 -p 6379
+ ```
+
+
+
+
+
+## Pipes
+
+
+
class dagster_k8s.PipesK8sClient
+
+
+ A pipes client for launching kubernetes pods.
+
+ By default context is injected via environment variables and messages are parsed out of
+ the pod logs, with other logs forwarded to stdout of the orchestration process.
+
+ The first container within the containers list of the pod spec is expected (or set) to be
+ the container prepared for pipes protocol communication.
+
+ Parameters:
+ - env (Optional[Mapping[str, str]]) – An optional dict of environment variables to pass to the
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – A message reader to use to read messages
+ - load_incluster_config (Optional[bool]) – Whether this client is expected to be running from inside
+ - kubeconfig_file (Optional[str]) – The value to pass as the config_file argument to
+ - kube_context (Optional[str]) – The value to pass as the context argument to
+ - poll_interval (Optional[float]) – How many seconds to wait between requests when
+
+
+
+
run
+
+
+ Publish a kubernetes pod and wait for it to complete, enriched with the pipes protocol.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](../execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](../execution.mdx#dagster.AssetExecutionContext)]) – The execution context.
+ - image (Optional[str]) – The image to set the first container in the pod spec to use.
+ - command (Optional[Union[str, Sequence[str]]]) – The command to set the first container in the pod spec to use.
+ - namespace (Optional[str]) – Which kubernetes namespace to use, defaults to the current namespace if
+ - env (Optional[Mapping[str,str]]) – A mapping of environment variable names to values to set on the first
+ - base_pod_meta (Optional[Mapping[str, Any]]) – Raw k8s config for the k8s pod’s metadata
+ - base_pod_spec (Optional[Mapping[str, Any]]) – Raw k8s config for the k8s pod’s pod spec
+ - extras (Optional[PipesExtras]) – Extra values to pass along as part of the ext protocol.
+ - context_injector (Optional[[*PipesContextInjector*](../pipes.mdx#dagster.PipesContextInjector)]) – Override the default ext protocol context injection.
+ - message_reader (Optional[[*PipesMessageReader*](../pipes.mdx#dagster.PipesMessageReader)]) – Override the default ext protocol message reader.
+ - ignore_containers (Optional[Set]) – Ignore certain containers from waiting for termination. Defaults to
+ - enable_multi_container_logs (bool) – Whether or not to enable multi-container log consumption.
+
+
+ Returns:
+ Wrapper containing results reported by the external
+ process.
+
+ Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster_k8s.PipesK8sPodLogsMessageReader
+
+ Message reader that reads messages from kubernetes pod logs.
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-looker.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-looker.mdx
new file mode 100644
index 0000000000000..43b31816fa5a1
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-looker.mdx
@@ -0,0 +1,511 @@
+---
+title: 'looker (dagster-looker)'
+title_meta: 'looker (dagster-looker) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'looker (dagster-looker) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Looker (dagster-looker)
+
+Dagster allows you to represent your Looker project as assets, alongside other your other
+technologies like dbt and Sling. This allows you to see how your Looker assets are connected to
+your other data assets, and how changes to other data assets might impact your Looker project.
+
+
+
+
+## Looker API
+
+Here, we provide interfaces to manage Looker projects using the Looker API.
+
+
+
+
+### Assets (Looker API)
+
+
+
class dagster_looker.LookerResource
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Represents a connection to a Looker instance and provides methods
+ to interact with the Looker API.
+
+
+
build_defs
+
+
+ :::danger[deprecated]
+ This API will be removed in version 1.9.0.
+ Use dagster_looker.load_looker_asset_specs instead.
+
+ :::
+
+ Returns a Definitions object which will load structures from the Looker instance
+ and translate it into assets, using the provided translator.
+
+ Parameters:
+ - request_start_pdt_builds (Optional[Sequence[[*RequestStartPdtBuild*](#dagster_looker.RequestStartPdtBuild)]]) – A list of
+ - dagster_looker_translator (Optional[[*DagsterLookerApiTranslator*](#dagster_looker.DagsterLookerApiTranslator)]) – The translator to
+
+
+ Returns: A Definitions object which will contain return the Looker structures as assets.Return type: [Definitions](../definitions.mdx#dagster.Definitions)
+
+
+
+
+
+
+
+
+
+
class dagster_looker.DagsterLookerApiTranslator
+
+
+
+
get_asset_key
+
+
+ :::danger[deprecated]
+ This API will be removed in version 1.10.
+ Use `DagsterLookerApiTranslator.get_asset_spec().key` instead.
+
+ :::
+
+
+
+
+
+
+
get_asset_spec
+
+
+
+
+
+
+
+
+
+
+
class dagster_looker.LookerStructureData
+
+
+
+
+
+
+
class dagster_looker.LookerStructureType
+
+
+
+
+
+
+
class dagster_looker.RequestStartPdtBuild
+
+
+ A request to start a PDT build. See [https://developers.looker.com/api/explorer/4.0/types/DerivedTable/RequestStartPdtBuild?sdk=py](https://developers.looker.com/api/explorer/4.0/types/DerivedTable/RequestStartPdtBuild?sdk=py)
+ for documentation on all available fields.
+
+ Parameters:
+ - model_name – The model of the PDT to start building.
+ - view_name – The view name of the PDT to start building.
+ - force_rebuild – Force rebuild of required dependent PDTs, even if they are already materialized.
+ - force_full_incremental – Force involved incremental PDTs to fully re-materialize.
+ - workspace – Workspace in which to materialize selected PDT (‘dev’ or default ‘production’).
+ - source – The source of this request.
+
+
+
+
+
+
+
+
class dagster_looker.LookerFilter
+
+
+ Filters the set of Looker objects to fetch.
+
+ Parameters:
+ - dashboard_folders (Optional[List[List[str]]]) – A list of folder paths to fetch dashboards from.
+ - only_fetch_explores_used_in_dashboards (bool) – If True, only explores used in the fetched dashboards
+
+
+
+
+
+
+
+
dagster_looker.load_looker_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Looker structures.
+
+ Parameters:
+ - looker_resource ([*LookerResource*](#dagster_looker.LookerResource)) – The Looker resource to fetch assets from.
+ - dagster_looker_translator (Optional[Union[[*DagsterLookerApiTranslator*](#dagster_looker.DagsterLookerApiTranslator), Type[[*DagsterLookerApiTranslator*](#dagster_looker.DagsterLookerApiTranslator)]]]) – The translator to use to convert Looker structures into [`dagster.AssetSpec`](../assets.mdx#dagster.AssetSpec).
+
+
+ Returns: The set of AssetSpecs representing the Looker structures.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns the AssetsDefinitions of the executable assets for the given the list of refreshable PDTs.
+
+ Parameters:
+ - resource_key (str) – The resource key to use for the Looker resource.
+ - request_start_pdt_builds (Optional[Sequence[[*RequestStartPdtBuild*](#dagster_looker.RequestStartPdtBuild)]]) – A list of requests to start PDT builds.
+ - dagster_looker_translator (Optional[Union[[*DagsterLookerApiTranslator*](#dagster_looker.DagsterLookerApiTranslator), Type[[*DagsterLookerApiTranslator*](#dagster_looker.DagsterLookerApiTranslator)]]]) – The translator to use to convert Looker structures into [`dagster.AssetSpec`](../assets.mdx#dagster.AssetSpec).
+
+
+ Returns: The AssetsDefinitions of the executable assets for the given the list of refreshable PDTs.Return type: [AssetsDefinition](../assets.mdx#dagster.AssetsDefinition)
+
+
+
+
+
+
+
+
+
+
+## lkml (LookML)
+
+Here, we provide interfaces to manage Looker projects defined a set of locally accessible
+LookML files.
+
+
+
+### Assets (lkml)
+
+
+
dagster_looker.build_looker_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Build a list of asset specs from a set of Looker structures defined in a Looker project.
+
+ Parameters:
+ - project_dir (Path) – The path to the Looker project directory.
+ - dagster_looker_translator (Optional[DagsterLookerTranslator]) – Allows customizing how to
+
+
+ Examples:
+
+ ```python
+ from pathlib import Path
+
+ from dagster import external_assets_from_specs
+ from dagster_looker import build_looker_asset_specs
+
+
+ looker_specs = build_looker_asset_specs(project_dir=Path("my_looker_project"))
+ looker_assets = external_assets_from_specs(looker_specs)
+ ```
+
+
+
+
+
+
class dagster_looker.DagsterLookerLkmlTranslator
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Holds a set of methods that derive Dagster asset definition metadata given a representation
+ of a LookML structure (dashboards, explores, views).
+
+ This class is exposed so that methods can be overridden to customize how Dagster asset metadata
+ is derived.
+
+
+
get_asset_key
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).key` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster asset key that represents the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide a custom asset key for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: The Dagster asset key that represents the LookML structure.Return type: [AssetKey](../assets.mdx#dagster.AssetKey)
+
+
+
+
+
+
get_asset_spec
+
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster asset spec that represents the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide a custom asset spec for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: The Dagster asset spec that represents the LookML structure.Return type: [AssetSpec](../assets.mdx#dagster.AssetSpec)
+
+
+
+
+
+
get_deps
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Iterate over `DagsterLookerLkmlTranslator.get_asset_spec(...).deps` to access `AssetDep.asset_key` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster dependencies of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide custom dependencies for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: The Dagster dependencies for the LookML structure.Return type: Sequence[[AssetKey](../assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
get_description
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).description` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster description of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide a custom description for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: The Dagster description for the LookML structure.Return type: Optional[str]
+
+
+
+
+
+
get_group_name
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).group_name` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster group name of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide a custom group name for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: A Dagster group name for the LookML structure.Return type: Optional[str]
+
+
+
+
+
+
get_metadata
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).metadata` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster metadata of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide custom metadata for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns:
+ A dictionary representing the Dagster metadata for the
+ LookML structure.
+
+ Return type: Optional[Mapping[str, Any]]
+
+
+
+
+
+
get_owners
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).owners` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster owners of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide custom owners for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns: A sequence of Dagster owners for the LookML structure.Return type: Optional[Sequence[str]]
+
+
+
+
+
+
get_tags
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterLookerLkmlTranslator.get_asset_spec(...).tags` instead..
+
+ :::
+
+ A method that takes in a LookML structure (dashboards, explores, views) and
+ returns the Dagster tags of the structure.
+
+ The LookML structure is parsed using `lkml`. You can learn more about this here:
+ [https://lkml.readthedocs.io/en/latest/simple.html](https://lkml.readthedocs.io/en/latest/simple.html).
+
+ You can learn more about LookML dashboards and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/param-lookml-dashboard](https://cloud.google.com/looker/docs/reference/param-lookml-dashboard).
+
+ You can learn more about LookML explores and views and the properties available in this
+ dictionary here: [https://cloud.google.com/looker/docs/reference/lookml-quick-reference](https://cloud.google.com/looker/docs/reference/lookml-quick-reference).
+
+ This method can be overridden to provide custom tags for a LookML structure.
+
+ Parameters: lookml_structure (Tuple[Path, str, Mapping[str, Any]]) – A tuple with the path to file
+ defining a LookML structure, the LookML structure type, and a dictionary
+ representing a LookML structure.Returns:
+ A dictionary representing the Dagster tags for the
+ LookML structure.
+
+ Return type: Optional[Mapping[str, str]]
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-mlflow.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-mlflow.mdx
new file mode 100644
index 0000000000000..d7d247e8e6be0
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-mlflow.mdx
@@ -0,0 +1,87 @@
+---
+title: 'mlflow (dagster-mlflow)'
+title_meta: 'mlflow (dagster-mlflow) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'mlflow (dagster-mlflow) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# MLflow (dagster-mlflow)
+
+
+
dagster_mlflow.mlflow_tracking ResourceDefinition
+
+
+
+
+ This resource initializes an MLflow run that’s used for all steps within a Dagster run.
+
+ This resource provides access to all of mlflow’s methods as well as the mlflow tracking client’s
+ methods.
+
+ Usage:
+
+ 1. Add the mlflow resource to any ops in which you want to invoke mlflow tracking APIs.
+ 2. Add the end_mlflow_on_run_finished hook to your job to end the MLflow run
+ when the Dagster run is finished.
+ Examples:
+
+ ```python
+ from dagster_mlflow import end_mlflow_on_run_finished, mlflow_tracking
+
+ @op(required_resource_keys={"mlflow"})
+ def mlflow_op(context):
+ mlflow.log_params(some_params)
+ mlflow.tracking.MlflowClient().create_registered_model(some_model_name)
+
+ @end_mlflow_on_run_finished
+ @job(resource_defs={"mlflow": mlflow_tracking})
+ def mlf_example():
+ mlflow_op()
+
+ # example using an mlflow instance with s3 storage
+ mlf_example.execute_in_process(run_config={
+ "resources": {
+ "mlflow": {
+ "config": {
+ "experiment_name": my_experiment,
+ "mlflow_tracking_uri": "http://localhost:5000",
+
+ # if want to run a nested run, provide parent_run_id
+ "parent_run_id": an_existing_mlflow_run_id,
+
+ # if you want to resume a run or avoid creating a new run in the resource init,
+ # provide mlflow_run_id
+ "mlflow_run_id": an_existing_mlflow_run_id,
+
+ # env variables to pass to mlflow
+ "env": {
+ "MLFLOW_S3_ENDPOINT_URL": my_s3_endpoint,
+ "AWS_ACCESS_KEY_ID": my_aws_key_id,
+ "AWS_SECRET_ACCESS_KEY": my_secret,
+ },
+
+ # env variables you want to log as mlflow tags
+ "env_to_tag": ["DOCKER_IMAGE_TAG"],
+
+ # key-value tags to add to your experiment
+ "extra_tags": {"super": "experiment"},
+ }
+ }
+ }
+ })
+ ```
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-msteams.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-msteams.mdx
new file mode 100644
index 0000000000000..f346553f84e61
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-msteams.mdx
@@ -0,0 +1,243 @@
+---
+title: 'microsoft teams (dagster-msteams)'
+title_meta: 'microsoft teams (dagster-msteams) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'microsoft teams (dagster-msteams) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+ This resource is for connecting to Microsoft Teams.
+
+ Provides a dagster_msteams.TeamsClient which can be used to
+ interface with the MS Teams API.
+
+ By configuring this resource, you can post messages to MS Teams from any Dagster op,
+ asset, schedule, or sensor:
+
+ Examples:
+
+ ```python
+ import os
+
+ from dagster import op, job, Definitions, EnvVar
+ from dagster_msteams import Card, MSTeamsResource
+
+
+ @op
+ def teams_op(msteams: MSTeamsResource):
+ card = Card()
+ card.add_attachment(text_message="Hello There !!")
+ msteams.get_client().post_message(payload=card.payload)
+
+
+ @job
+ def teams_job():
+ teams_op()
+
+ defs = Definitions(
+ jobs=[teams_job],
+ resources={
+ "msteams": MSTeamsResource(
+ hook_url=EnvVar("TEAMS_WEBHOOK_URL")
+ )
+ }
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+## Sensors
+
+
+
dagster_msteams.teams_on_failure HookDefinition
+
+
+ Create a hook on step failure events that will message the given MS Teams webhook URL.
+
+ Parameters:
+ - message_fn (Optional(Callable[[[*HookContext*](../hooks.mdx#dagster.HookContext)], str])) – Function which takes in the
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your webserver instance. Specify this
+ - webserver_base_url – (Optional[str]): The base url of your webserver instance. Specify this
+
+
+ Examples:
+
+ ```python
+ @teams_on_failure(webserver_base_url="http://localhost:3000")
+ @job(...)
+ def my_job():
+ pass
+ ```
+ ```python
+ def my_message_fn(context: HookContext) -> str:
+ return f"Op {context.op.name} failed!"
+
+ @op
+ def a_op(context):
+ pass
+
+ @job(...)
+ def my_job():
+ a_op.with_hooks(hook_defs={teams_on_failure("#foo", my_message_fn)})
+ ```
+
+
+
+
+
+
dagster_msteams.teams_on_success HookDefinition
+
+
+ Create a hook on step success events that will message the given MS Teams webhook URL.
+
+ Parameters:
+ - message_fn (Optional(Callable[[[*HookContext*](../hooks.mdx#dagster.HookContext)], str])) – Function which takes in the
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your webserver instance. Specify this
+
+
+ Examples:
+
+ ```python
+ @teams_on_success(webserver_base_url="http://localhost:3000")
+ @job(...)
+ def my_job():
+ pass
+ ```
+ ```python
+ def my_message_fn(context: HookContext) -> str:
+ return f"Op {context.op.name} failed!"
+
+ @op
+ def a_op(context):
+ pass
+
+ @job(...)
+ def my_job():
+ a_op.with_hooks(hook_defs={teams_on_success("#foo", my_message_fn)})
+ ```
+
+
+
+
+
+
dagster_msteams.make_teams_on_run_failure_sensor
+
+
+ Create a sensor on run failures that will message the given MS Teams webhook URL.
+
+ Parameters:
+ - hook_url (str) – MS Teams incoming webhook URL.
+ - message_fn (Optional(Callable[[[*RunFailureSensorContext*](../schedules-sensors.mdx#dagster.RunFailureSensorContext)], str])) – Function which
+ - http_proxy – (Optional[str]): Proxy for requests using http protocol.
+ - https_proxy – (Optional[str]): Proxy for requests using https protocol.
+ - timeout – (Optional[float]): Connection timeout in seconds. Defaults to 60.
+ - verify – (Optional[bool]): Whether to verify the servers TLS certificate.
+ - name – (Optional[str]): The name of the sensor. Defaults to “teams_on_run_failure”.
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your webserver instance. Specify this to allow
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](../jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](../graphs.mdx#dagster.GraphDefinition), UnresolvedAssetJobDefinition, [*RepositorySelector*](../schedules-sensors.mdx#dagster.RepositorySelector), [*JobSelector*](../schedules-sensors.mdx#dagster.JobSelector)]]]) – Jobs in the current repository that will be monitored by this sensor. Defaults to None,
+ - monitor_all_code_locations (bool) – If set to True, the sensor will monitor all runs in the
+ - webserver_base_url – (Optional[str]): The base url of your webserver instance. Specify this to allow
+ - monitor_all_repositories (bool) – deprecatedmonitor_all_code_locations instead.) If set to True, the sensor will monitor all runs in the
+
+
+ Examples:
+
+ ```python
+ teams_on_run_failure = make_teams_on_run_failure_sensor(
+ hook_url=os.getenv("TEAMS_WEBHOOK_URL")
+ )
+
+ @repository
+ def my_repo():
+ return [my_job + teams_on_run_failure]
+ ```
+ ```python
+ def my_message_fn(context: RunFailureSensorContext) -> str:
+ return "Job {job_name} failed! Error: {error}".format(
+ job_name=context.dagster_run.job_name,
+ error=context.failure_event.message,
+ )
+
+ teams_on_run_failure = make_teams_on_run_failure_sensor(
+ hook_url=os.getenv("TEAMS_WEBHOOK_URL"),
+ message_fn=my_message_fn,
+ webserver_base_url="http://localhost:3000",
+ )
+ ```
+
+
+
+ This resource is for connecting to Microsoft Teams.
+
+ The resource object is a dagster_msteams.TeamsClient.
+
+ By configuring this resource, you can post messages to MS Teams from any Dagster solid:
+
+ Examples:
+
+ ```python
+ import os
+
+ from dagster import op, job
+ from dagster_msteams import Card, msteams_resource
+
+
+ @op(required_resource_keys={"msteams"})
+ def teams_op(context):
+ card = Card()
+ card.add_attachment(text_message="Hello There !!")
+ context.resources.msteams.post_message(payload=card.payload)
+
+
+ @job(resource_defs={"msteams": msteams_resource})
+ def teams_job():
+ teams_op()
+
+
+ teams_job.execute_in_process(
+ {"resources": {"msteams": {"config": {"hook_url": os.getenv("TEAMS_WEBHOOK_URL")}}}}
+ )
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-mysql.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-mysql.mdx
new file mode 100644
index 0000000000000..6a03098bda46f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-mysql.mdx
@@ -0,0 +1,112 @@
+---
+title: 'mysql (dagster-mysql)'
+title_meta: 'mysql (dagster-mysql) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'mysql (dagster-mysql) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# MySQL (dagster-mysql)
+
+
+
class dagster_mysql.MySQLEventLogStorage
+
+
+ MySQL-backed event log storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ dagster.yaml
+
+ ```YAML
+
+ event_log_storage:
+ module: dagster_mysql.event_log
+ class: MySQLEventLogStorage
+ config:
+ mysql_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { db_name }
+ port: { port }
+
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
+
class dagster_mysql.MySQLRunStorage
+
+
+ MySQL-backed run storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ dagster.yaml
+
+ ```YAML
+
+ run_storage:
+ module: dagster_mysql.run_storage
+ class: MySQLRunStorage
+ config:
+ mysql_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { database }
+ port: { port }
+
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
+
class dagster_mysql.MySQLScheduleStorage
+
+
+ MySQL-backed run storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ dagster.yaml
+
+ ```YAML
+
+ schedule_storage:
+ module: dagster_mysql.schedule_storage
+ class: MySQLScheduleStorage
+ config:
+ mysql_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { db_name }
+ port: { port }
+
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-openai.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-openai.mdx
new file mode 100644
index 0000000000000..18e537b8ef7ca
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-openai.mdx
@@ -0,0 +1,284 @@
+---
+title: 'openai (dagster-openai)'
+title_meta: 'openai (dagster-openai) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'openai (dagster-openai) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# OpenAI (dagster-openai)
+
+The dagster_openai library provides utilities for using OpenAI with Dagster.
+A good place to start with dagster_openai is [the guide](https://docs.dagster.io/integrations/libraries/openai/).
+
+
+
dagster_openai.with_usage_metadata
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This wrapper can be used on any endpoint of the
+ openai library \
+ to log the OpenAI API usage metadata in the asset metadata.
+
+ Examples:
+
+ ```python
+ from dagster import (
+ AssetExecutionContext,
+ AssetKey,
+ AssetSelection,
+ AssetSpec,
+ Definitions,
+ EnvVar,
+ MaterializeResult,
+ asset,
+ define_asset_job,
+ multi_asset,
+ )
+ from dagster_openai import OpenAIResource, with_usage_metadata
+
+
+ @asset(compute_kind="OpenAI")
+ def openai_asset(context: AssetExecutionContext, openai: OpenAIResource):
+ with openai.get_client(context) as client:
+ client.fine_tuning.jobs.create = with_usage_metadata(
+ context=context, output_name="some_output_name", func=client.fine_tuning.jobs.create
+ )
+ client.fine_tuning.jobs.create(model="gpt-3.5-turbo", training_file="some_training_file")
+
+
+ openai_asset_job = define_asset_job(name="openai_asset_job", selection="openai_asset")
+
+
+ @multi_asset(
+ specs=[
+ AssetSpec("my_asset1"),
+ AssetSpec("my_asset2"),
+ ]
+ )
+ def openai_multi_asset(context: AssetExecutionContext, openai: OpenAIResource):
+ with openai.get_client(context, asset_key=AssetKey("my_asset1")) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo", messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+
+ # The materialization of `my_asset1` will include both OpenAI usage metadata
+ # and the metadata added when calling `MaterializeResult`.
+ return (
+ MaterializeResult(asset_key="my_asset1", metadata={"foo": "bar"}),
+ MaterializeResult(asset_key="my_asset2", metadata={"baz": "qux"}),
+ )
+
+
+ openai_multi_asset_job = define_asset_job(
+ name="openai_multi_asset_job", selection=AssetSelection.assets(openai_multi_asset)
+ )
+
+
+ defs = Definitions(
+ assets=[openai_asset, openai_multi_asset],
+ jobs=[openai_asset_job, openai_multi_asset_job],
+ resources={
+ "openai": OpenAIResource(api_key=EnvVar("OPENAI_API_KEY")),
+ },
+ )
+ ```
+
+
+
+
+
+
class dagster_openai.OpenAIResource
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ This resource is wrapper over the
+ [openai library](https://github.com/openai/openai-python).
+
+ By configuring this OpenAI resource, you can interact with OpenAI API
+ and log its usage metadata in the asset metadata.
+
+ Examples:
+
+ ```python
+ import os
+
+ from dagster import AssetExecutionContext, Definitions, EnvVar, asset, define_asset_job
+ from dagster_openai import OpenAIResource
+
+
+ @asset(compute_kind="OpenAI")
+ def openai_asset(context: AssetExecutionContext, openai: OpenAIResource):
+ with openai.get_client(context) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo",
+ messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+
+ openai_asset_job = define_asset_job(name="openai_asset_job", selection="openai_asset")
+
+ defs = Definitions(
+ assets=[openai_asset],
+ jobs=[openai_asset_job],
+ resources={
+ "openai": OpenAIResource(api_key=EnvVar("OPENAI_API_KEY")),
+ },
+ )
+ ```
+
+
get_client
+
+
+ Yields an `openai.Client` for interacting with the OpenAI API.
+
+ By default, in an asset context, the client comes with wrapped endpoints
+ for three API resources, Completions, Embeddings and Chat,
+ allowing to log the API usage metadata in the asset metadata.
+
+ Note that the endpoints are not and cannot be wrapped
+ to automatically capture the API usage metadata in an op context.
+
+ Parameters: context – The `context` object for computing the op or asset in which `get_client` is called.
+ Examples:
+
+ ```python
+ from dagster import (
+ AssetExecutionContext,
+ Definitions,
+ EnvVar,
+ GraphDefinition,
+ OpExecutionContext,
+ asset,
+ define_asset_job,
+ op,
+ )
+ from dagster_openai import OpenAIResource
+
+
+ @op
+ def openai_op(context: OpExecutionContext, openai: OpenAIResource):
+ with openai.get_client(context) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo", messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+
+
+ openai_op_job = GraphDefinition(name="openai_op_job", node_defs=[openai_op]).to_job()
+
+
+ @asset(compute_kind="OpenAI")
+ def openai_asset(context: AssetExecutionContext, openai: OpenAIResource):
+ with openai.get_client(context) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo", messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+
+
+ openai_asset_job = define_asset_job(name="openai_asset_job", selection="openai_asset")
+
+ defs = Definitions(
+ assets=[openai_asset],
+ jobs=[openai_asset_job, openai_op_job],
+ resources={
+ "openai": OpenAIResource(api_key=EnvVar("OPENAI_API_KEY")),
+ },
+ )
+ ```
+
+
+
+
+
+
get_client_for_asset
+
+
+ Yields an `openai.Client` for interacting with the OpenAI.
+
+ When using this method, the OpenAI API usage metadata is automatically
+ logged in the asset materializations associated with the provided `asset_key`.
+
+ By default, the client comes with wrapped endpoints
+ for three API resources, Completions, Embeddings and Chat,
+ allowing to log the API usage metadata in the asset metadata.
+
+ This method can only be called when working with assets,
+ i.e. the provided `context` must be of type `AssetExecutionContext`.
+
+ Parameters:
+ - context – The `context` object for computing the asset in which `get_client` is called.
+ - asset_key – the `asset_key` of the asset for which a materialization should include the metadata.
+
+
+ Examples:
+
+ ```python
+ from dagster import (
+ AssetExecutionContext,
+ AssetKey,
+ AssetSpec,
+ Definitions,
+ EnvVar,
+ MaterializeResult,
+ asset,
+ define_asset_job,
+ multi_asset,
+ )
+ from dagster_openai import OpenAIResource
+
+
+ @asset(compute_kind="OpenAI")
+ def openai_asset(context: AssetExecutionContext, openai: OpenAIResource):
+ with openai.get_client_for_asset(context, context.asset_key) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo", messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+
+
+ openai_asset_job = define_asset_job(name="openai_asset_job", selection="openai_asset")
+
+
+ @multi_asset(specs=[AssetSpec("my_asset1"), AssetSpec("my_asset2")], compute_kind="OpenAI")
+ def openai_multi_asset(context: AssetExecutionContext, openai_resource: OpenAIResource):
+ with openai_resource.get_client_for_asset(context, asset_key=AssetKey("my_asset1")) as client:
+ client.chat.completions.create(
+ model="gpt-3.5-turbo", messages=[{"role": "user", "content": "Say this is a test"}]
+ )
+ return (
+ MaterializeResult(asset_key="my_asset1", metadata={"some_key": "some_value1"}),
+ MaterializeResult(asset_key="my_asset2", metadata={"some_key": "some_value2"}),
+ )
+
+
+ openai_multi_asset_job = define_asset_job(
+ name="openai_multi_asset_job", selection="openai_multi_asset"
+ )
+
+ defs = Definitions(
+ assets=[openai_asset, openai_multi_asset],
+ jobs=[openai_asset_job, openai_multi_asset_job],
+ resources={
+ "openai": OpenAIResource(api_key=EnvVar("OPENAI_API_KEY")),
+ },
+ )
+ ```
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pagerduty.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pagerduty.mdx
new file mode 100644
index 0000000000000..335bc87a41c7f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pagerduty.mdx
@@ -0,0 +1,95 @@
+---
+title: 'pagerduty (dagster-pagerduty)'
+title_meta: 'pagerduty (dagster-pagerduty) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'pagerduty (dagster-pagerduty) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# PagerDuty (dagster-pagerduty)
+
+This library provides an integration with PagerDuty, to support creating alerts from your Dagster
+code.
+
+Presently, it provides a thin wrapper on the [Events API V2](https://v2.developer.pagerduty.com/docs/events-api-v2).
+
+
+
+
+
+
+
+# Getting Started
+
+You can install this library with:
+
+ ```default
+ pip install dagster-pagerduty
+ ```
+To use this integration, you’ll first need to create an Events API V2 PagerDuty integration on a PagerDuty service. There are instructions
+[here](https://support.pagerduty.com/docs/services-and-integrations#section-events-api-v2) for
+creating a new PagerDuty service & integration.
+
+Once your Events API V2 integration is set up, you’ll find an Integration Key (also referred to as a
+“Routing Key”) on the Integrations tab for your service. This key is used to authorize events
+created from the PagerDuty events API.
+
+Once your service/integration is created, you can provision a PagerDuty resource and issue PagerDuty
+alerts from within your ops.
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pandas.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pandas.mdx
new file mode 100644
index 0000000000000..040507ed2c8f8
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pandas.mdx
@@ -0,0 +1,114 @@
+---
+title: 'pandas (dagster-pandas)'
+title_meta: 'pandas (dagster-pandas) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'pandas (dagster-pandas) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Pandas (dagster-pandas)
+
+The dagster_pandas library provides utilities for using pandas with Dagster and for implementing
+validation on pandas DataFrames. A good place to start with dagster_pandas is the [validation
+guide](https://docs.dagster.io/integrations/libraries/pandas/).
+
+
+
+ Constructs a custom pandas dataframe dagster type.
+
+ Parameters:
+ - name (str) – Name of the dagster pandas type.
+ - description (Optional[str]) – A markdown-formatted string, displayed in tooling.
+ - columns (Optional[List[[*PandasColumn*](#dagster_pandas.PandasColumn)]]) – A list of `PandasColumn` objects
+ - metadata_fn (Optional[Callable[[], Union[Dict[str, Union[str, float, int, Dict, [*MetadataValue*](../metadata.mdx#dagster.MetadataValue)]]) – A callable which takes your dataframe and returns a dict with string label keys and
+ - dataframe_constraints (Optional[List[DataFrameConstraint]]) – A list of objects that inherit from
+ - loader (Optional[[*DagsterTypeLoader*](../types.mdx#dagster.DagsterTypeLoader)]) – An instance of a class that
+
+
+
+
+
+
+
+
class dagster_pandas.RowCountConstraint
+
+
+ A dataframe constraint that validates the expected count of rows.
+
+ Parameters:
+ - num_allowed_rows (int) – The number of allowed rows in your dataframe.
+ - error_tolerance (Optional[int]) – The acceptable threshold if you are not completely certain. Defaults to 0.
+
+
+
+
+
+
+
+
class dagster_pandas.StrictColumnsConstraint
+
+
+ A dataframe constraint that validates column existence and ordering.
+
+ Parameters:
+ - strict_column_list (List[str]) – The exact list of columns that your dataframe must have.
+ - enforce_ordering (Optional[bool]) – If true, will enforce that the ordering of column names must match.
+
+
+
+
+
+
+
+
class dagster_pandas.PandasColumn
+
+
+ The main API for expressing column level schemas and constraints for your custom dataframe
+ types.
+
+ Parameters:
+ - name (str) – Name of the column. This must match up with the column name in the dataframe you
+ - is_required (Optional[bool]) – Flag indicating the optional/required presence of the column.
+ - constraints (Optional[List[Constraint]]) – List of constraint objects that indicate the
+
+
+
+
+
+
+
+
dagster_pandas.DataFrame `=` \
+
+
+ Define a type in dagster. These can be used in the inputs and outputs of ops.
+
+ Parameters:
+ - type_check_fn (Callable[[[*TypeCheckContext*](../execution.mdx#dagster.TypeCheckContext), Any], [Union[bool, [*TypeCheck*](../ops.mdx#dagster.TypeCheck)]]]) – The function that defines the type check. It takes the value flowing
+ - key (Optional[str]) –
+
+ The unique key to identify types programmatically.
+ The key property always has a value. If you omit key to the argument
+ to the init function, it instead receives the value of `name`. If
+ neither `key` nor `name` is provided, a `CheckError` is thrown.
+
+ In the case of a generic type such as `List` or `Optional`, this is
+ generated programmatically based on the type parameters.
+
+ - name (Optional[str]) – A unique name given by a user. If `key` is `None`, `key`
+ - description (Optional[str]) – A markdown-formatted string, displayed in tooling.
+ - loader (Optional[[*DagsterTypeLoader*](../types.mdx#dagster.DagsterTypeLoader)]) – An instance of a class that
+ - required_resource_keys (Optional[Set[str]]) – Resource keys required by the `type_check_fn`.
+ - is_builtin (bool) – Defaults to False. This is used by tools to display or
+ - kind (DagsterTypeKind) – Defaults to None. This is used to determine the kind of runtime type
+ - typing_type – Defaults to None. A valid python typing type (e.g. Optional[List[int]]) for the
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pandera.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pandera.mdx
new file mode 100644
index 0000000000000..b4ff01bbad00a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pandera.mdx
@@ -0,0 +1,49 @@
+---
+title: 'pandera (dagster-pandera)'
+title_meta: 'pandera (dagster-pandera) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'pandera (dagster-pandera) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Pandera (dagster-pandera)
+
+The dagster_pandera library allows Dagster users to use dataframe validation library [Pandera](https://github.com/pandera-dev/pandera) for the validation of Pandas dataframes. See [the guide](https://docs.dagster.io/integrations/libraries/pandera/) for details.
+
+
+
dagster_pandera.pandera_schema_to_dagster_type
+
+
+ Convert a Pandera dataframe schema to a DagsterType.
+
+ The generated Dagster type will be given an automatically generated name. The schema’s title
+ property, name property, or class name (in that order) will be used. If neither title or
+ name is defined, a name of the form DagsterPanderaDataframe\ is generated.
+
+ Additional metadata is also extracted from the Pandera schema and attached to the returned
+ DagsterType as a metadata dictionary. The extracted metadata includes:
+
+ - Descriptions on the schema and constituent columns and checks.
+ - Data types for each column.
+ - String representations of all column-wise checks.
+ - String representations of all row-wise (i.e. “wide”) checks.
+
+
+ The returned DagsterType type will call the Pandera schema’s validate() method in its type
+ check function. Validation is done in lazy mode, i.e. pandera will attempt to validate all
+ values in the dataframe, rather than stopping on the first error.
+
+ If validation fails, the returned TypeCheck object will contain two pieces of metadata:
+
+ - num_failures total number of validation errors.
+ - failure_sample a table containing up to the first 10 validation errors.
+
+
+ Parameters: schema (Union[pa.DataFrameSchema, Type[pa.DataFrameModel]])Returns: Dagster Type constructed from the Pandera schema.Return type: [DagsterType](../types.mdx#dagster.DagsterType)
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-papertrail.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-papertrail.mdx
new file mode 100644
index 0000000000000..a0b6559e67613
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-papertrail.mdx
@@ -0,0 +1,37 @@
+---
+title: 'papertrail (dagster-papertrail)'
+title_meta: 'papertrail (dagster-papertrail) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'papertrail (dagster-papertrail) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Papertrail (dagster-papertrail)
+
+This library provides an integration with [Papertrail](https://papertrailapp.com) for logging.
+
+You can easily set up your Dagster job to log to Papertrail. You’ll need an active Papertrail
+account, and have your papertrail URL and port handy.
+
+
+
+ Core class for defining loggers.
+
+ Loggers are job-scoped logging handlers, which will be automatically invoked whenever
+ dagster messages are logged from within a job.
+
+ Parameters:
+ - logger_fn (Callable[[[*InitLoggerContext*](../loggers.mdx#dagster.InitLoggerContext)], logging.Logger]) – User-provided function to
+ - config_schema (Optional[[*ConfigSchema*](../config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of this logger.
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pipes.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pipes.mdx
new file mode 100644
index 0000000000000..e81c8d56333b3
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pipes.mdx
@@ -0,0 +1,1014 @@
+---
+title: 'pipes (dagster-pipes)'
+title_meta: 'pipes (dagster-pipes) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'pipes (dagster-pipes) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Pipes (dagster-pipes)
+
+The `dagster-pipes` library is intended for inclusion in an external process that integrates with Dagster using the [Pipes](https://docs.dagster.io/guides/build/external-pipelines/) protocol. This could be in an environment like Databricks, Kubernetes, or Docker. Using this library, you can write code in the external process that streams metadata back to Dagster.
+
+For a detailed look at the Pipes process, including how to customize it, refer to the [Dagster Pipes details and customization guide](https://docs.dagster.io/guides/build/external-pipelines/dagster-pipes-details-and-customization).
+
+Looking to set up a Pipes client in Dagster? Refer to the [Dagster Pipes API reference](https://docs.dagster.io/api/python-api/libraries/dagster-pipes).
+
+Note: This library isn’t included with `dagster` and must be [installed separately](https://pypi.org/project/dagster-pipes/).
+
+
+
+
+## Context
+
+
+
dagster_pipes.open_dagster_pipes
+
+
+ Initialize the Dagster Pipes context.
+
+ This function should be called near the entry point of a pipes process. It will load injected
+ context information from Dagster and spin up the machinery for streaming messages back to
+ Dagster.
+
+ If the process was not launched by Dagster, this function will emit a warning and return a
+ MagicMock object. This should make all operations on the context no-ops and prevent your code
+ from crashing.
+
+ Parameters:
+ - context_loader (Optional[[*PipesContextLoader*](#dagster_pipes.PipesContextLoader)]) – The context loader to use. Defaults to
+ - message_writer (Optional[[*PipesMessageWriter*](#dagster_pipes.PipesMessageWriter)]) – The message writer to use. Defaults to
+ - params_loader (Optional[[*PipesParamsLoader*](#dagster_pipes.PipesParamsLoader)]) – The params loader to use. Defaults to
+
+
+ Returns: The initialized context.Return type: [PipesContext](#dagster_pipes.PipesContext)
+
+
+
+
+
+
class dagster_pipes.PipesContext
+
+
+ The context for a Dagster Pipes process.
+
+ This class is analogous to [`OpExecutionContext`](../execution.mdx#dagster.OpExecutionContext) on the Dagster side of the Pipes
+ connection. It provides access to information such as the asset key(s) and partition key(s) in
+ scope for the current step. It also provides methods for logging and emitting results that will
+ be streamed back to Dagster.
+
+ This class should not be directly instantiated by the user. Instead it should be initialized by
+ calling [`open_dagster_pipes()`](#dagster_pipes.open_dagster_pipes), which will return the singleton instance of this class.
+ After open_dagster_pipes() has been called, the singleton instance can also be retrieved by
+ calling [`PipesContext.get()`](#dagster_pipes.PipesContext.get).
+
+
+
close
+
+ Close the pipes connection. This will flush all buffered messages to the orchestration
+ process and cause any further attempt to write a message to raise an error. This method is
+ idempotent– subsequent calls after the first have no effect.
+
+
+
+
+
classmethod get
+
+ Get the singleton instance of the context. Raises an error if the context has not been initialized.
+
+
+
+
+
get_extra
+
+
+ Get the value of an extra provided by the user. Raises an error if the extra is not defined.
+
+ Parameters: key (str) – The key of the extra.Returns: The value of the extra.Return type: Any
+
+
+
+
+
+
classmethod is_initialized
+
+ bool: Whether the context has been initialized.
+
+
+
+
+
log_external_stream
+
+
+
+
+
+
+
report_asset_check
+
+
+ Report to Dagster that an asset check has been performed. Streams a payload containing
+ check result information back to Dagster. If no assets or associated checks are in scope, raises an error.
+
+ Parameters:
+ - check_name (str) – The name of the check.
+ - passed (bool) – Whether the check passed.
+ - severity (PipesAssetCheckSeverity) – The severity of the check. Defaults to “ERROR”.
+ - metadata (Optional[Mapping[str, Union[PipesMetadataRawValue, PipesMetadataValue]]]) – Metadata for the check. Defaults to None.
+ - asset_key (Optional[str]) – The asset key for the check. If only a single asset is in
+
+
+
+
+
+
+
+
report_asset_materialization
+
+
+ Report to Dagster that an asset has been materialized. Streams a payload containing
+ materialization information back to Dagster. If no assets are in scope, raises an error.
+
+ Parameters:
+ - metadata (Optional[Mapping[str, Union[PipesMetadataRawValue, PipesMetadataValue]]]) – Metadata for the materialized asset. Defaults to None.
+ - data_version (Optional[str]) – The data version for the materialized asset.
+ - asset_key (Optional[str]) – The asset key for the materialized asset. If only a
+
+
+
+
+
+
+
+
report_custom_message
+
+
+ Send a JSON serializable payload back to the orchestration process. Can be retrieved there
+ using get_custom_messages.
+
+ Parameters: payload (Any) – JSON serializable data.
+
+
+
+
+
+
classmethod set
+
+ Set the singleton instance of the context.
+
+
+
+
+
property asset_key
+
+
+ The AssetKey for the currently scoped asset. Raises an error if 0 or multiple assets
+ are in scope.
+
+ Type: str
+
+
+
+
+
+
property asset_keys
+
+
+ The AssetKeys for the currently scoped assets. Raises an error if no
+ assets are in scope.
+
+ Type: Sequence[str]
+
+
+
+
+
+
property code_version
+
+
+ The code version for the currently scoped asset. Raises an error if 0 or
+ multiple assets are in scope.
+
+ Type: Optional[str]
+
+
+
+
+
+
property code_version_by_asset_key
+
+
+ Mapping of asset key to code version for the currently
+ scoped assets. Raises an error if no assets are in scope.
+
+ Type: Mapping[str, Optional[str]]
+
+
+
+
+
+
property extras
+
+
+ Key-value map for all extras provided by the user.
+
+ Type: Mapping[str, Any]
+
+
+
+
+
+
property is_asset_step
+
+
+ Whether the current step targets assets.
+
+ Type: bool
+
+
+
+
+
+
property is_closed
+
+
+ Whether the context has been closed.
+
+ Type: bool
+
+
+
+
+
+
property is_partition_step
+
+
+ Whether the current step is scoped to one or more partitions.
+
+ Type: bool
+
+
+
+
+
+
property job_name
+
+
+ The job name for the currently executing run. Returns None if the run is
+ not derived from a job.
+
+ Type: Optional[str]
+
+
+
+
+
+
property log
+
+
+ A logger that streams log messages back to Dagster.
+
+ Type: logging.Logger
+
+
+
+
+
+
property partition_key
+
+
+ The partition key for the currently scoped partition. Raises an error if 0 or
+ multiple partitions are in scope.
+
+ Type: str
+
+
+
+
+
+
property partition_key_range
+
+
+ The partition key range for the currently scoped partition or
+ partitions. Raises an error if no partitions are in scope.
+
+ Type: PipesPartitionKeyRange
+
+
+
+
+
+
property partition_time_window
+
+
+ The partition time window for the currently scoped partition
+ or partitions. Returns None if partitions in scope are not temporal. Raises an error if no
+ partitions are in scope.
+
+ Type: Optional[PipesTimeWindow]
+
+
+
+
+
+
property provenance
+
+
+ The provenance for the currently scoped asset. Raises an
+ error if 0 or multiple assets are in scope.
+
+ Type: Optional[PipesDataProvenance]
+
+
+
+
+
+
property provenance_by_asset_key
+
+
+ Mapping of asset key to provenance for the
+ currently scoped assets. Raises an error if no assets are in scope.
+
+ Type: Mapping[str, Optional[PipesDataProvenance]]
+
+
+
+
+
+
property retry_number
+
+
+ The retry number for the currently executing run.
+
+ Type: int
+
+
+
+
+
+
property run_id
+
+
+ The run ID for the currently executing pipeline run.
+
+ Type: str
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Advanced
+
+Most Pipes users won’t need to use the APIs in the following sections unless they are customizing the Pipes protocol.
+
+Refer to the [Dagster Pipes details and customization guide](https://docs.dagster.io/guides/build/external-pipelines/dagster-pipes-details-and-customization) for more information.
+
+
+
+
+### Context loaders
+
+Context loaders load the context payload from the location specified in the bootstrap payload.
+
+
+
class dagster_pipes.PipesContextLoader
+
+
+
+
abstract load_context
+
+
+ A @contextmanager that loads context data injected by the orchestration process.
+
+ This method should read and yield the context data from the location specified by the passed in
+ PipesParams.
+
+ Parameters: params (PipesParams) – The params provided by the context injector in the orchestration
+ process.Yields: PipesContextData – The context data.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesDefaultContextLoader
+
+
+ Context loader that loads context data from either a file or directly from the provided params.
+
+ The location of the context data is configured by the params received by the loader. If the params
+ include a key path, then the context data will be loaded from a file at the specified path. If
+ the params instead include a key data, then the corresponding value should be a dict
+ representing the context data.
+
+
+
load_context
+
+
+ A @contextmanager that loads context data injected by the orchestration process.
+
+ This method should read and yield the context data from the location specified by the passed in
+ PipesParams.
+
+ Parameters: params (PipesParams) – The params provided by the context injector in the orchestration
+ process.Yields: PipesContextData – The context data.
+
+
+
+
+
+
DIRECT_KEY `=` 'data'
+
+
+
+
+
+
+
FILE_PATH_KEY `=` 'path'
+
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesDbfsContextLoader
+
+
+ Context loader that reads context from a JSON file on DBFS.
+
+
+
load_context
+
+
+ A @contextmanager that loads context data injected by the orchestration process.
+
+ This method should read and yield the context data from the location specified by the passed in
+ PipesParams.
+
+ Parameters: params (PipesParams) – The params provided by the context injector in the orchestration
+ process.Yields: PipesContextData – The context data.
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+### Params loaders
+
+Params loaders load the bootstrap payload from some globally accessible key-value store.
+
+
+
class dagster_pipes.PipesParamsLoader
+
+
+ Object that loads params passed from the orchestration process by the context injector and
+ message reader. These params are used to respectively bootstrap the
+ [`PipesContextLoader`](#dagster_pipes.PipesContextLoader) and [`PipesMessageWriter`](#dagster_pipes.PipesMessageWriter).
+
+
+
abstract is_dagster_pipes_process
+
+ Whether or not this process has been provided with provided with information to create
+ a PipesContext or should instead return a mock.
+
+
+
+
+
abstract load_context_params
+
+ PipesParams: Load params passed by the orchestration-side context injector.
+
+
+
+
+
abstract load_messages_params
+
+ PipesParams: Load params passed by the orchestration-side message reader.
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesEnvVarParamsLoader
+
+ Params loader that extracts params from environment variables.
+
+
+
+
+
class dagster_pipes.PipesCliArgsParamsLoader
+
+
+ Params loader that extracts params from known CLI arguments.
+
+
+
is_dagster_pipes_process
+
+ Whether or not this process has been provided with provided with information to create
+ a PipesContext or should instead return a mock.
+
+
+
+
+
load_context_params
+
+ PipesParams: Load params passed by the orchestration-side context injector.
+
+
+
+
+
load_messages_params
+
+ PipesParams: Load params passed by the orchestration-side message reader.
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesMappingParamsLoader
+
+
+ Params loader that extracts params from a Mapping provided at init time.
+
+
+
is_dagster_pipes_process
+
+ Whether or not this process has been provided with provided with information to create
+ a PipesContext or should instead return a mock.
+
+
+
+
+
load_context_params
+
+ PipesParams: Load params passed by the orchestration-side context injector.
+
+
+
+
+
load_messages_params
+
+ PipesParams: Load params passed by the orchestration-side message reader.
+
+
+
+
+
+
+
+
+
+
+
+
+
+### Message writers
+
+Message writers write messages to the location specified in the bootstrap payload.
+
+
+
class dagster_pipes.PipesMessageWriter
+
+
+
+
get_opened_extras
+
+
+ Return arbitary reader-specific information to be passed back to the orchestration
+ process under the extras key of the initialization payload.
+
+ Returns: A dict of arbitrary data to be passed back to the orchestration process.Return type: PipesExtras
+
+
+
+
+
+
final get_opened_payload
+
+
+ Return a payload containing information about the external process to be passed back to
+ the orchestration process. This should contain information that cannot be known before
+ the external process is launched.
+
+ This method should not be overridden by users. Instead, users should
+ override get_opened_extras to inject custom data.
+
+
+
+
+
+
+
abstract open
+
+
+ A @contextmanager that initializes a channel for writing messages back to Dagster.
+
+ This method should takes the params passed by the orchestration-side
+ `PipesMessageReader` and use them to construct and yield a
+ [`PipesMessageWriterChannel`](#dagster_pipes.PipesMessageWriterChannel).
+
+ Parameters: params (PipesParams) – The params provided by the message reader in the orchestration
+ process.Yields: PipesMessageWriterChannel – Channel for writing messagse back to Dagster.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesDefaultMessageWriter
+
+
+ Message writer that writes messages to either a file or the stdout or stderr stream.
+
+ The write location is configured by the params received by the writer. If the params include a
+ key path, then messages will be written to a file at the specified path. If the params instead
+ include a key stdio, then messages then the corresponding value must specify either stderr
+ or stdout, and messages will be written to the selected stream.
+
+
+
open
+
+
+ A @contextmanager that initializes a channel for writing messages back to Dagster.
+
+ This method should takes the params passed by the orchestration-side
+ `PipesMessageReader` and use them to construct and yield a
+ [`PipesMessageWriterChannel`](#dagster_pipes.PipesMessageWriterChannel).
+
+ Parameters: params (PipesParams) – The params provided by the message reader in the orchestration
+ process.Yields: PipesMessageWriterChannel – Channel for writing messagse back to Dagster.
+
+
+
+ Construct and yield a [`PipesBlobStoreMessageWriterChannel`](#dagster_pipes.PipesBlobStoreMessageWriterChannel).
+
+ Parameters: params (PipesParams) – The params provided by the message reader in the orchestration
+ process.Yields: PipesBlobStoreMessageWriterChannel – Channel that periodically uploads message chunks to
+ a blob store.
+
+
+ Message writer channel that periodically uploads message chunks to some blob store endpoint.
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesS3MessageWriter
+
+
+ Message writer that writes messages by periodically writing message chunks to an S3 bucket.
+
+ Parameters:
+ - client (Any) – A boto3.client(“s3”) object.
+ - interval (float) – interval in seconds between upload chunk uploads
+
+
+
+
make_channel
+
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesDbfsMessageWriter
+
+
+ Message writer that writes messages by periodically writing message chunks to a directory on DBFS.
+
+
+
get_opened_extras
+
+
+ Return arbitary reader-specific information to be passed back to the orchestration
+ process under the extras key of the initialization payload.
+
+ Returns: A dict of arbitrary data to be passed back to the orchestration process.Return type: PipesExtras
+
+
+
+
+
+
make_channel
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+### Message writer channels
+
+Message writer channels are objects that write messages back to the Dagster orchestration process.
+
+
+
class dagster_pipes.PipesMessageWriterChannel
+
+
+ Object that writes messages back to the Dagster orchestration process.
+
+
+
abstract write_message
+
+
+ Write a message to the orchestration process.
+
+ Parameters: message (PipesMessage) – The message to write.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesBlobStoreMessageWriterChannel
+
+
+ Message writer channel that periodically uploads message chunks to some blob store endpoint.
+
+
+
buffered_upload_loop
+
+
+
+
+
+
+
flush_messages
+
+
+
+
+
+
+
abstract upload_messages_chunk
+
+
+
+
+
+
+
write_message
+
+
+ Write a message to the orchestration process.
+
+ Parameters: message (PipesMessage) – The message to write.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesBufferedFilesystemMessageWriterChannel
+
+
+ Message writer channel that periodically writes message chunks to an endpoint mounted on the filesystem.
+
+ Parameters: interval (float) – interval in seconds between chunk uploads
+
+
upload_messages_chunk
+
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesFileMessageWriterChannel
+
+
+ Message writer channel that writes one message per line to a file.
+
+
+
write_message
+
+
+ Write a message to the orchestration process.
+
+ Parameters: message (PipesMessage) – The message to write.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesStreamMessageWriterChannel
+
+
+ Message writer channel that writes one message per line to a TextIO stream.
+
+
+
write_message
+
+
+ Write a message to the orchestration process.
+
+ Parameters: message (PipesMessage) – The message to write.
+
+
+
+
+
+
+
+
+
+
class dagster_pipes.PipesS3MessageWriterChannel
+
+
+ Message writer channel for writing messages by periodically writing message chunks to an S3 bucket.
+
+ Parameters:
+ - client (Any) – A boto3.client(“s3”) object.
+ - bucket (str) – The name of the S3 bucket to write to.
+ - key_prefix (Optional[str]) – An optional prefix to use for the keys of written blobs.
+ - interval (float) – interval in seconds between upload chunk uploads
+
+
+
+
upload_messages_chunk
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+### Utilities
+
+
+
dagster_pipes.encode_env_var
+
+
+ Encode value by serializing to JSON, compressing with zlib, and finally encoding with base64.
+ base64_encode(compress(to_json(value))) in function notation.
+
+ Parameters: value (Any) – The value to encode. Must be JSON-serializable.Returns: The encoded value.Return type: str
+
+
+
+
+
+
dagster_pipes.decode_env_var
+
+
+ Decode a value by decoding from base64, decompressing with zlib, and finally deserializing from
+ JSON. from_json(decompress(base64_decode(value))) in function notation.
+
+ Parameters: value (Any) – The value to decode.Returns: The decoded value.Return type: Any
+
+
+
+
+
+
class dagster_pipes.DagsterPipesError
+
+
+
+
+
+
+
class dagster_pipes.DagsterPipesWarning
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-polars.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-polars.mdx
new file mode 100644
index 0000000000000..8fb15b4d00b05
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-polars.mdx
@@ -0,0 +1,411 @@
+---
+title: 'polars (dagster-polars)'
+title_meta: 'polars (dagster-polars) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'polars (dagster-polars) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Polars (dagster-polars)
+
+This library provides Dagster integration with [Polars](https://pola.rs).
+It allows using Polars eager or lazy DataFrames as inputs and outputs with Dagster’s @asset and @op.
+Type annotations are used to control whether to load an eager or lazy DataFrame. Lazy DataFrames can be sinked as output.
+Multiple serialization formats (Parquet, Delta Lake, BigQuery) and filesystems (local, S3, GCS, …) are supported.
+
+Comprehensive list of dagster-polars behavior for supported type annotations can be found in [Type Annotations](#types)`Type Annotations` section.
+
+
+
+
+
+
+
+# Installation
+
+ ```default
+ pip install dagster-polars
+ ```
+Some IOManagers (like [`PolarsDeltaIOManager`](#dagster_polars.PolarsDeltaIOManager)) may require additional dependencies, which are provided with extras like dagster-polars[delta].
+Please check the documentation for each IOManager for more details.
+
+
+
+
+
+
+
+# Quickstart
+
+Common filesystem-based IOManagers features highlights, using [`PolarsParquetIOManager`](#dagster_polars.PolarsParquetIOManager) as an example (see [`BasePolarsUPathIOManager`](#dagster_polars.BasePolarsUPathIOManager) for the full list of features provided by dagster-polars):
+
+Type annotations are not required. By default an eager pl.DataFrame will be loaded.
+
+ ```python
+ from dagster import asset
+ import polars as pl
+
+ @asset(io_manager_key="polars_parquet_io_manager")
+ def upstream():
+ return DataFrame({"foo": [1, 2, 3]})
+
+ @asset(io_manager_key="polars_parquet_io_manager")
+ def downstream(upstream) -> pl.LazyFrame:
+ assert isinstance(upstream, pl.DataFrame)
+ return upstream.lazy() # LazyFrame will be sinked
+ ```
+Lazy pl.LazyFrame can be scanned by annotating the input with pl.LazyFrame, and returning a pl.LazyFrame will sink it:
+
+ ```python
+ @asset(io_manager_key="polars_parquet_io_manager")
+ def downstream(upstream: pl.LazyFrame) -> pl.LazyFrame:
+ assert isinstance(upstream, pl.LazyFrame)
+ return upstream
+ ```
+The same logic applies to partitioned assets:
+
+ ```python
+ @asset
+ def downstream(partitioned_upstream: Dict[str, pl.LazyFrame]):
+ assert isinstance(partitioned_upstream, dict)
+ assert isinstance(partitioned_upstream["my_partition"], pl.LazyFrame)
+ ```
+Optional inputs and outputs are supported:
+
+ ```python
+ @asset
+ def upstream() -> Optional[pl.DataFrame]:
+ if has_data:
+ return DataFrame({"foo": [1, 2, 3]}) # type check will pass
+ else:
+ return None # type check will pass and `dagster_polars` will skip writing the output completely
+
+ @asset
+ def downstream(upstream: Optional[pl.LazyFrame]): # upstream will be None if it doesn't exist in storage
+ ...
+ ```
+By default all the IOManagers store separate partitions as physically separated locations, such as:
+
+ - /my/asset/key/partition_0.extension
+ - /my/asset/key/partition_1.extension
+
+
+This mode is useful for e.g. snapshotting.
+
+Some IOManagers (like [`PolarsDeltaIOManager`](#dagster_polars.PolarsDeltaIOManager)) support reading and writing partitions in storage-native format in the same location.
+This mode can be typically enabled by setting “partition_by” metadata value. For example, [`PolarsDeltaIOManager`](#dagster_polars.PolarsDeltaIOManager) would store different partitions in the same /my/asset/key.delta directory, which will be properly partitioned.
+
+This mode should be preferred for true partitioning.
+
+
+
+
+
+
+
+
+# Type Annotations
+
+Type aliases like DataFrameWithPartitions are provided by `dagster_polars.types` for convenience.
+
+# Supported type annotations and dagster-polars behavior
+
+| Type annotation | Type Alias | Behavior |
+| :------------------------ | :--------- | :-------------------------------------------------------------------------- |
+| DataFrame | | read/write aDataFrame |
+| LazyFrame | | read/sink aLazyFrame |
+| Optional[DataFrame] | | read/write aDataFrame. Do nothing if no data is found in storage or the output isNone |
+| Optional[LazyFrame] | | read aLazyFrame. Do nothing if no data is found in storage |
+| Dict[str, DataFrame] | DataFrameWithPartitions | read multipleDataFrame`s as `Dict[str, DataFrame]. Raises an error for missing partitions, unless“allow_missing_partitions”input metadata is set toTrue |
+| Dict[str, LazyFrame] | LazyFramePartitions | read multipleLazyFrame`s as `Dict[str, LazyFrame]. Raises an error for missing partitions, unless“allow_missing_partitions”input metadata is set toTrue |
+
+
+Generic builtins (like tuple[…] instead of Tuple[…]) are supported for Python >= 3.9.
+
+
+
+ Base class for dagster-polars IOManagers.
+
+ Doesn’t define a specific storage format.
+
+ To implement a specific storage format (parquet, csv, etc), inherit from this class and implement the write_df_to_path, sink_df_to_path and scan_df_from_path methods.
+
+ Features:
+ - All the features of [`UPathIOManager`](../io-managers.mdx#dagster.UPathIOManager) - works with local and remote filesystems (like S3), supports loading multiple partitions with respect to [`PartitionMapping`](../partitions.mdx#dagster.PartitionMapping), and more
+ - loads the correct type - polars.DataFrame, polars.LazyFrame, or other types defined in `dagster_polars.types` - based on the input type annotation (or dagster.DagsterType’s typing_type)
+ - can sink lazy pl.LazyFrame DataFrames
+ - handles Nones with Optional types by skipping loading missing inputs or saving None outputs
+ - logs various metadata about the DataFrame - size, schema, sample, stats, …
+ - the “columns” input metadata value can be used to select a subset of columns to load
+
+
+
+
+
+ Implements reading and writing Polars DataFrames from/to [BigQuery](https://cloud.google.com/bigquery)).
+
+ Features:
+ - All `DBIOManager` features
+ - Supports writing partitioned tables (“partition_expr” input metadata key must be specified).
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster import Definitions, EnvVar
+ from dagster_polars import PolarsBigQueryIOManager
+
+ @asset(
+ key_prefix=["my_dataset"] # will be used as the dataset in BigQuery
+ )
+ def my_table() -> pl.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": PolarsBigQueryIOManager(project=EnvVar("GCP_PROJECT"))
+ }
+ )
+ ```
+ You can tell Dagster in which dataset to create tables by setting the “dataset” configuration value.
+ If you do not provide a dataset as configuration to the I/O manager, Dagster will determine a dataset based
+ on the assets and ops using the I/O Manager. For assets, the dataset will be determined from the asset key,
+ as shown in the above example. The final prefix before the asset name will be used as the dataset. For example,
+ if the asset “my_table” had the key prefix [“gcp”, “bigquery”, “my_dataset”], the dataset “my_dataset” will be
+ used. For ops, the dataset can be specified by including a “schema” entry in output metadata. If “schema” is
+ not provided via config or on the asset/op, “public” will be used for the dataset.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_dataset"})}
+ )
+ def make_my_table() -> pl.DataFrame:
+ # the returned value will be stored at my_dataset.my_table
+ ...
+ ```
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pl.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+ If you cannot upload a file to your Dagster deployment, or otherwise cannot
+ [authenticate with GCP](https://cloud.google.com/docs/authentication/provide-credentials-adc)
+ via a standard method, you can provide a service account key as the “gcp_credentials” configuration.
+ Dagster will store this key in a temporary file and set GOOGLE_APPLICATION_CREDENTIALS to point to the file.
+ After the run completes, the file will be deleted, and GOOGLE_APPLICATION_CREDENTIALS will be
+ unset. The key must be base64 encoded to avoid issues with newlines in the keys. You can retrieve
+ the base64 encoded key with this shell command: cat $GOOGLE_APPLICATION_CREDENTIALS | base64
+
+ The “write_disposition” metadata key can be used to set the write_disposition parameter
+ of bigquery.JobConfig. For example, set it to “WRITE_APPEND” to append to an existing table intead of
+ overwriting it.
+
+ Install dagster-polars[gcp] to use this IOManager.
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-postgres.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-postgres.mdx
new file mode 100644
index 0000000000000..70782ce214e11
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-postgres.mdx
@@ -0,0 +1,169 @@
+---
+title: 'postgresql (dagster-postgres)'
+title_meta: 'postgresql (dagster-postgres) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'postgresql (dagster-postgres) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# PostgreSQL (dagster-postgres)
+
+
+
dagster_postgres.PostgresEventLogStorage `=` \
+
+
+
+
+ Postgres-backed event log storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ To use Postgres for all of the components of your instance storage, you can add the following
+ block to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ storage:
+ postgres:
+ postgres_db:
+ username: my_username
+ password: my_password
+ hostname: my_hostname
+ db_name: my_database
+ port: 5432
+ ```
+ If you are configuring the different storage components separately and are specifically
+ configuring your event log storage to use Postgres, you can add a block such as the following
+ to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ event_log_storage:
+ module: dagster_postgres.event_log
+ class: PostgresEventLogStorage
+ config:
+ postgres_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { db_name }
+ port: { port }
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
+
dagster_postgres.PostgresRunStorage `=` \
+
+
+
+
+ Postgres-backed run storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ To use Postgres for all of the components of your instance storage, you can add the following
+ block to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ storage:
+ postgres:
+ postgres_db:
+ username: my_username
+ password: my_password
+ hostname: my_hostname
+ db_name: my_database
+ port: 5432
+ ```
+ If you are configuring the different storage components separately and are specifically
+ configuring your run storage to use Postgres, you can add a block such as the following
+ to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ run_storage:
+ module: dagster_postgres.run_storage
+ class: PostgresRunStorage
+ config:
+ postgres_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { db_name }
+ port: { port }
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
+
dagster_postgres.PostgresScheduleStorage `=` \
+
+
+
+
+ Postgres-backed run storage.
+
+ Users should not directly instantiate this class; it is instantiated by internal machinery when
+ `dagster-webserver` and `dagster-graphql` load, based on the values in the `dagster.yaml` file in
+ `$DAGSTER_HOME`. Configuration of this class should be done by setting values in that file.
+
+ To use Postgres for all of the components of your instance storage, you can add the following
+ block to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ storage:
+ postgres:
+ postgres_db:
+ username: my_username
+ password: my_password
+ hostname: my_hostname
+ db_name: my_database
+ port: 5432
+ ```
+ If you are configuring the different storage components separately and are specifically
+ configuring your schedule storage to use Postgres, you can add a block such as the following
+ to your `dagster.yaml`:
+
+ dagster.yaml
+
+ ```YAML
+ schedule_storage:
+ module: dagster_postgres.schedule_storage
+ class: PostgresScheduleStorage
+ config:
+ postgres_db:
+ username: { username }
+ password: { password }
+ hostname: { hostname }
+ db_name: { db_name }
+ port: { port }
+ ```
+ Note that the fields in this config are [`StringSource`](../config.mdx#dagster.StringSource) and
+ [`IntSource`](../config.mdx#dagster.IntSource) and can be configured from environment variables.
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-powerbi.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-powerbi.mdx
new file mode 100644
index 0000000000000..5f96cda0423d2
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-powerbi.mdx
@@ -0,0 +1,146 @@
+---
+title: 'powerbi (dagster-powerbi)'
+title_meta: 'powerbi (dagster-powerbi) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'powerbi (dagster-powerbi) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# PowerBI (dagster-powerbi)
+
+Dagster allows you to represent your PowerBI Workspaces as assets, alongside other your other
+technologies like dbt and Sling. This allows you to see how your PowerBI assets are connected to
+your other data assets, and how changes to other data assets might impact your PowerBI Workspaces.
+
+
+
+
+## PowerBI API
+
+Here, we provide interfaces to manage PowerBI Workspaces using the PowerBI API.
+
+
+
+### Assets (PowerBI)
+
+
+
class dagster_powerbi.PowerBIServicePrincipal
+
+ Authenticates with PowerBI using a service principal.
+
+
+
+
+
class dagster_powerbi.PowerBIToken
+
+ Authenticates with PowerBI directly using an API access token.
+
+
+
+
+
class dagster_powerbi.PowerBIWorkspace
+
+
+ Represents a workspace in PowerBI and provides utilities
+ to interact with the PowerBI API.
+
+
+
build_defs
+
+
+ :::danger[deprecated]
+ This API will be removed in version 1.9.0.
+ Use dagster_powerbi.load_powerbi_asset_specs instead.
+
+ :::
+
+ Returns a Definitions object which will load Power BI content from
+ the workspace and translate it into assets, using the provided translator.
+
+ Parameters:
+ - context (Optional[DefinitionsLoadContext]) – The context to use when loading the definitions.
+ - dagster_powerbi_translator (Type[[*DagsterPowerBITranslator*](#dagster_powerbi.DagsterPowerBITranslator)]) – The translator to use
+ - enable_refresh_semantic_models (bool) – Whether to enable refreshing semantic models
+
+
+ Returns: A Definitions object which will build and return the Power BI content.Return type: [Definitions](../definitions.mdx#dagster.Definitions)
+
+
+
+
+
+
poll_refresh
+
+ Polls the refresh status of a PowerBI dataset until it completes or fails.
+
+
+
+
+
trigger_and_poll_refresh
+
+ Triggers a refresh of a PowerBI dataset and polls until it completes or fails.
+
+
+
+
+
trigger_refresh
+
+ Triggers a refresh of a PowerBI dataset.
+
+
+
+
+
+
+
+
+
class dagster_powerbi.DagsterPowerBITranslator
+
+ Translator class which converts raw response data from the PowerBI API into AssetSpecs.
+ Subclass this class to implement custom logic for each type of PowerBI content.
+
+
+
+
+
dagster_powerbi.load_powerbi_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Power BI content in the workspace.
+
+ Parameters:
+ - workspace ([*PowerBIWorkspace*](#dagster_powerbi.PowerBIWorkspace)) – The Power BI workspace to load assets from.
+ - dagster_powerbi_translator (Optional[Union[[*DagsterPowerBITranslator*](#dagster_powerbi.DagsterPowerBITranslator), Type[[*DagsterPowerBITranslator*](#dagster_powerbi.DagsterPowerBITranslator)]]]) – The translator to use to convert Power BI content into [`dagster.AssetSpec`](../assets.mdx#dagster.AssetSpec).
+ - use_workspace_scan (bool) – Whether to scan the entire workspace using admin APIs
+
+
+ Returns: The set of assets representing the Power BI content in the workspace.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Builds an asset definition for refreshing a PowerBI semantic model.
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-prometheus.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-prometheus.mdx
new file mode 100644
index 0000000000000..9e6acf11b3522
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-prometheus.mdx
@@ -0,0 +1,66 @@
+---
+title: 'prometheus (dagster-prometheus)'
+title_meta: 'prometheus (dagster-prometheus) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'prometheus (dagster-prometheus) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pyspark.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pyspark.mdx
new file mode 100644
index 0000000000000..3a8f991c012aa
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-pyspark.mdx
@@ -0,0 +1,79 @@
+---
+title: 'pyspark (dagster-pyspark)'
+title_meta: 'pyspark (dagster-pyspark) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'pyspark (dagster-pyspark) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-shell.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-shell.mdx
new file mode 100644
index 0000000000000..8cc6ed4722244
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-shell.mdx
@@ -0,0 +1,250 @@
+---
+title: 'shell (dagster-shell)'
+title_meta: 'shell (dagster-shell) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'shell (dagster-shell) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Shell (dagster-shell)
+
+The Dagster shell library provides utilities and op factories for executing inline shell scripts or script files.
+
+
+
+
+
+
+# APIs
+
+
+
dagster_shell.create_shell_command_op
+
+
+ :::danger[deprecated]
+ This API will be removed in version 0.25.
+ Use PipesSubprocessClient instead..
+
+ :::
+
+ DEPRECATED: Use PipesSubprocessClient instead.
+
+ This function is a factory that constructs ops to execute a shell command.
+
+ Note that you can only use `shell_command_op` if you know the command you’d like to execute
+ at job construction time. If you’d like to construct shell commands dynamically during
+ job execution and pass them between ops, you should use `shell_op` instead.
+
+ The resulting op can take a single `start` argument that is a
+ [Nothing dependency](https://legacy-docs.dagster.io/concepts/ops-jobs-graphs/graphs#defining-nothing-dependencies)
+ to allow you to run ops before the shell op.
+
+ Examples:
+
+ ```python
+ from dagster import graph
+ from dagster_shell import create_shell_command_op
+
+
+ @graph
+ def my_graph():
+ a = create_shell_command_op('echo "hello, world!"', name="a")
+ a()
+ ```
+ ```python
+ @op
+ def run_before_shell_op():
+ do_some_work()
+
+ @graph
+ def my_graph():
+ my_echo_op = create_shell_command_op("echo hello world!", name="echo_op")
+ my_echo_op(start=run_before_shell_op())
+ ```
+ Parameters:
+ - shell_command (str) – The shell command that the constructed op will execute.
+ - name (str) – The name of the constructed op.
+ - description (Optional[str]) – Human-readable description of this op.
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by this op.
+ - tags (Optional[Dict[str, Any]]) – Arbitrary metadata for the op. Frameworks may
+
+
+ Raises: [Failure](../ops.mdx#dagster.Failure)Failure – Raised when the shell command returns a non-zero exit code.Returns: Returns the constructed op definition.Return type: [OpDefinition](../ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
dagster_shell.create_shell_script_op
+
+
+ :::danger[deprecated]
+ This API will be removed in version 0.25.
+ Use PipesSubprocessClient instead..
+
+ :::
+
+ DEPRECATED: Use PipesSubprocessClient instead.
+
+ This function is a factory which constructs an op that will execute a shell command read
+ from a script file.
+
+ Any kwargs passed to this function will be passed along to the underlying [`@op`](../ops.mdx#dagster.op) decorator. However, note that overriding `config` or `output_defs` is not
+ supported.
+
+ You might consider using [`@graph`](../graphs.mdx#dagster.graph) to wrap this op
+ in the cases where you’d like to configure the shell op with different config fields.
+
+ If no `ins` are passed then the resulting op can take a single `start` argument that is a
+ [Nothing dependency](https://legacy-docs.dagster.io/concepts/ops-jobs-graphs/graphs#defining-nothing-dependencies)
+ to allow you to run ops before the shell op.
+
+ Examples:
+
+ ```python
+ from dagster import file_relative_path, graph
+ from dagster_shell import create_shell_script_op
+
+
+ @graph
+ def my_graph():
+ a = create_shell_script_op(file_relative_path(__file__, "hello_world.sh"), name="a")
+ a()
+ ```
+ ```python
+ @op
+ def run_before_shell_op():
+ do_some_work()
+
+ @graph
+ def my_graph():
+ my_echo_op = create_shell_script_op(file_relative_path(__file__, "hello_world.sh"), name="echo_op")
+ my_echo_op(start=run_before_shell_op())
+ ```
+ Parameters:
+ - shell_script_path (str) – The script file to execute.
+ - name (Optional[str]) – The name of this op. Defaults to “create_shell_script_op”.
+ - ins (Optional[Mapping[str, [*In*](../ops.mdx#dagster.In)]]) – Ins for the op. Defaults to
+
+
+ Raises: [Failure](../ops.mdx#dagster.Failure)Failure – Raised when the shell command returns a non-zero exit code.Returns: Returns the constructed op definition.Return type: [OpDefinition](../ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
dagster_shell.shell_op
+
+
+ This op executes a shell command it receives as input.
+ This op is suitable for uses where the command to execute is generated dynamically by
+ upstream ops. If you know the command to execute at job construction time,
+ consider `shell_command_op` instead.
+
+ Parameters:
+ - shell_command – The shell command to be executed
+ - config (ShellOpConfig) – A ShellOpConfig object specifying configuration options
+
+
+ Examples:
+
+ ```python
+ @op
+ def create_shell_command():
+ return "echo hello world!"
+
+ @graph
+ def echo_graph():
+ shell_op(create_shell_command())
+ ```
+
+
+
+
+
+
dagster_shell.execute_shell_command
+
+
+ This function is a utility for executing shell commands from within a Dagster op (or from Python in general).
+ It can be used to execute shell commands on either op input data, or any data generated within a generic python op.
+
+ Internally, it executes a shell script specified by the argument `shell_command`. The script will be written
+ to a temporary file first and invoked via `subprocess.Popen(['bash', shell_script_path], ...)`.
+
+ In the Popen invocation, `stdout=PIPE, stderr=STDOUT` is used, and the combined stdout/stderr
+ output is retrieved.
+
+ Examples:
+
+ ```python
+ from dagster import OpExecutionContext, op
+ from dagster_shell import execute_shell_command
+
+
+ @op
+ def my_shell_op(context: OpExecutionContext, data: str):
+ temp_file = "/tmp/data.txt"
+ with open(temp_file, "w", encoding="utf-8") as temp_file_writer:
+ temp_file_writer.write(data)
+ execute_shell_command(f"cat {temp_file}", output_logging="STREAM", log=context.log)
+ ```
+ Parameters:
+ - shell_command (str) – The shell command to execute
+ - output_logging (str) – The logging mode to use. Supports STREAM, BUFFER, and NONE.
+ - log (Union[logging.Logger, [*DagsterLogManager*](../loggers.mdx#dagster.DagsterLogManager)]) – Any logger which responds to .info()
+ - cwd (str, optional) – Working directory for the shell command to use. Defaults to the
+ - env (Dict[str, str], optional) – Environment dictionary to pass to `subprocess.Popen`.
+ - log_shell_command (bool, optional) – Whether to log the shell command before executing it.
+
+
+ Returns: A tuple where the first element is the combined stdout/stderr output of running the shell
+ command and the second element is the return code.Return type: Tuple[str, int]
+
+
+
+
+
+
dagster_shell.execute_shell_script
+
+
+ Execute a shell script file specified by the argument `shell_script_path`. The script will be
+ invoked via `subprocess.Popen(['bash', shell_script_path], ...)`.
+
+ In the Popen invocation, `stdout=PIPE, stderr=STDOUT` is used, and the combined stdout/stderr
+ output is retrieved.
+
+ Examples:
+
+ ```python
+ from dagster import OpExecutionContext, op
+ from dagster_shell import execute_shell_script
+
+
+ @op
+ def my_shell_op(context: OpExecutionContext, data: str):
+ temp_file = "/tmp/echo_data.sh"
+ with open(temp_file, "w", encoding="utf-8") as temp_file_writer:
+ temp_file_writer.write(f"echo {data}")
+ execute_shell_script(temp_file, output_logging="STREAM", log=context.log)
+ ```
+ Parameters:
+ - shell_script_path (str) – The shell script to execute.
+ - output_logging (str) – The logging mode to use. Supports STREAM, BUFFER, and NONE.
+ - log (Union[logging.Logger, [*DagsterLogManager*](../loggers.mdx#dagster.DagsterLogManager)]) – Any logger which responds to .info()
+ - cwd (str, optional) – Working directory for the shell command to use. Defaults to the
+ - env (Dict[str, str], optional) – Environment dictionary to pass to `subprocess.Popen`.
+ - log_shell_command (bool, optional) – Whether to log the shell command before executing it.
+
+
+ Raises: Exception – When an invalid output_logging is selected. Unreachable from op-based
+ invocation since the config system will check output_logging against the config
+ enum.Returns: A tuple where the first element is the combined stdout/stderr output of running the shell
+ command and the second element is the return code.Return type: Tuple[str, int]
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-sigma.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-sigma.mdx
new file mode 100644
index 0000000000000..463c6e413ea44
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-sigma.mdx
@@ -0,0 +1,148 @@
+---
+title: 'sigma (dagster-sigma)'
+title_meta: 'sigma (dagster-sigma) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'sigma (dagster-sigma) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Sigma (dagster-sigma)
+
+Dagster allows you to represent the workbooks and datasets in your Sigma project as assets alongside other
+technologies including dbt and Sling. This allows you to visualize relationships between your Sigma assets
+and their dependencies.
+
+Related documentation pages: [Using Dagster with Sigma](https://docs.dagster.io/integrations/libraries/sigma/).
+
+
+
+
+## Sigma API
+
+Here, we provide interfaces to manage Sigma projects using the Sigma API.
+
+
+
+### Assets (Sigma API)
+
+
+
class dagster_sigma.SigmaOrganization
+
+
+ Represents a workspace in Sigma and provides utilities
+ to interact with the Sigma API.
+
+
+
build_defs
+
+
+ :::danger[deprecated]
+ This API will be removed in version 1.9.0.
+ Use dagster_sigma.load_sigma_asset_specs instead.
+
+ :::
+
+ Returns a Definitions object representing the Sigma content in the organization.
+
+ Parameters: dagster_sigma_translator (Type[[*DagsterSigmaTranslator*](#dagster_sigma.DagsterSigmaTranslator)]) – The translator to use
+ to convert Sigma content into AssetSpecs. Defaults to DagsterSigmaTranslator.Returns: The set of assets representing the Sigma content in the organization.Return type: [Definitions](../definitions.mdx#dagster.Definitions)
+
+
+
+
+
+
+
+
+
+
class dagster_sigma.SigmaBaseUrl
+
+
+ Enumeration of Sigma API base URLs for different cloud providers.
+
+ [https://help.sigmacomputing.com/reference/get-started-sigma-api#identify-your-api-request-url](https://help.sigmacomputing.com/reference/get-started-sigma-api#identify-your-api-request-url)
+
+
+
+
+
+
+
class dagster_sigma.DagsterSigmaTranslator
+
+ Translator class which converts raw response data from the Sigma API into AssetSpecs.
+ Subclass this class to provide custom translation logic.
+
+
+
+
+
class dagster_sigma.SigmaDataset
+
+
+ Represents a Sigma dataset, a centralized data definition which can
+ contain aggregations or other manipulations.
+
+ [https://help.sigmacomputing.com/docs/datasets](https://help.sigmacomputing.com/docs/datasets)
+
+
+
+
+
+
+
class dagster_sigma.SigmaWorkbook
+
+
+ Represents a Sigma workbook, a collection of visualizations and queries
+ for data exploration and analysis.
+
+ [https://help.sigmacomputing.com/docs/workbooks](https://help.sigmacomputing.com/docs/workbooks)
+
+
+
+
+
+
+
class dagster_sigma.SigmaFilter
+
+
+ Filters the set of Sigma objects to fetch.
+
+ Parameters:
+ - workbook_folders (Optional[Sequence[Sequence[str]]]) – A list of folder paths to fetch workbooks from.
+ - workbooks (Optional[Sequence[Sequence[str]]]) – A list of fully qualified workbook paths to fetch.
+ - include_unused_datasets (bool) – Whether to include datasets that are not used in any workbooks.
+
+
+
+
+
+
+
+
dagster_sigma.load_sigma_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Sigma content in the organization.
+
+ Parameters:
+ - organization ([*SigmaOrganization*](#dagster_sigma.SigmaOrganization)) – The Sigma organization to fetch assets from.
+ - dagster_sigma_translator (Optional[Union[[*DagsterSigmaTranslator*](#dagster_sigma.DagsterSigmaTranslator), Type[DagsterSigmaTranslatorr]]]) – The translator to use to convert Sigma content into [`dagster.AssetSpec`](../assets.mdx#dagster.AssetSpec).
+ - sigma_filter (Optional[[*SigmaFilter*](#dagster_sigma.SigmaFilter)]) – Filters the set of Sigma objects to fetch.
+ - fetch_column_data (bool) – Whether to fetch column data for datasets, which can be slow.
+ - fetch_lineage_data (bool) – Whether to fetch any lineage data for workbooks and datasets.
+ - snapshot_path (Optional[Union[str, Path]]) – Path to a snapshot file to load Sigma data from,
+
+
+ Returns: The set of assets representing the Sigma content in the organization.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-slack.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-slack.mdx
new file mode 100644
index 0000000000000..92204e44fa2c6
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-slack.mdx
@@ -0,0 +1,234 @@
+---
+title: 'slack (dagster-slack)'
+title_meta: 'slack (dagster-slack) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'slack (dagster-slack) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Slack (dagster-slack)
+
+
+
+This library provides an integration with Slack, to support posting messages in your company’s Slack workspace.
+
+
+
+Presently, it provides a thin wrapper on the Slack client API [chat.postMessage](https://api.slack.com/methods/chat.postMessage).
+
+
+
+To use this integration, you’ll first need to create a Slack App for it.
+
+1. Create App: Go to [https://api.slack.com/apps](https://api.slack.com/apps) and click “Create New App”:
+
+2. Install App: After creating an app, on the left-hand side of the app configuration, click “Bot Users”, and then create a bot user. Then, click “Install App” on the left hand side, and finally “Install App to Workspace”.
+3. Bot Token: Once finished, this will create a new bot token for your bot/workspace:
+
+Copy this bot token and put it somewhere safe; see [Safely Storing Credentials](https://api.slack.com/docs/oauth-safety) for more on this topic.
+
+
+
dagster_slack.SlackResource ResourceDefinition
+
+
+ This resource is for connecting to Slack.
+
+ By configuring this Slack resource, you can post messages to Slack from any Dagster op, asset, schedule or sensor.
+
+ Examples:
+
+ ```python
+ import os
+
+ from dagster import EnvVar, job, op
+ from dagster_slack import SlackResource
+
+
+ @op
+ def slack_op(slack: SlackResource):
+ slack.get_client().chat_postMessage(channel='#noise', text=':wave: hey there!')
+
+ @job
+ def slack_job():
+ slack_op()
+
+ defs = Definitions(
+ jobs=[slack_job],
+ resources={
+ "slack": SlackResource(token=EnvVar("MY_SLACK_TOKEN")),
+ },
+ )
+ ```
+
+
+
+
+
+
dagster_slack.make_slack_on_run_failure_sensor
+
+
+ Create a sensor on job failures that will message the given Slack channel.
+
+ Parameters:
+ - channel (str) – The channel to send the message to (e.g. “#my_channel”)
+ - slack_token (str) – The slack token.
+ - text_fn (Optional(Callable[[[*RunFailureSensorContext*](../schedules-sensors.mdx#dagster.RunFailureSensorContext)], str])) – Function which
+ - blocks_fn (Callable[[[*RunFailureSensorContext*](../schedules-sensors.mdx#dagster.RunFailureSensorContext)], List[Dict]]) – Function which takes in
+ - name – (Optional[str]): The name of the sensor. Defaults to “slack_on_run_failure”.
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your Dagit instance. Specify this to allow
+ - minimum_interval_seconds – (Optional[int]): The minimum number of seconds that will elapse
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](../jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](../graphs.mdx#dagster.GraphDefinition), [*RepositorySelector*](../schedules-sensors.mdx#dagster.RepositorySelector), [*JobSelector*](../schedules-sensors.mdx#dagster.JobSelector), CodeLocationSensor]]]) – The jobs in the
+ - job_selection (Optional[List[Union[[*JobDefinition*](../jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](../graphs.mdx#dagster.GraphDefinition), [*RepositorySelector*](../schedules-sensors.mdx#dagster.RepositorySelector), [*JobSelector*](../schedules-sensors.mdx#dagster.JobSelector), CodeLocationSensor]]]) – deprecatedmonitored_jobs instead.) (deprecated in favor of monitored_jobs)
+ - monitor_all_code_locations (bool) – If set to True, the sensor will monitor all runs in the
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - webserver_base_url – (Optional[str]): The base url of your webserver instance. Specify this to allow
+ - monitor_all_repositories (bool) – deprecatedmonitor_all_code_locations instead.) If set to True, the sensor will monitor all runs in the
+
+
+ Examples:
+
+ ```python
+ slack_on_run_failure = make_slack_on_run_failure_sensor(
+ "#my_channel",
+ os.getenv("MY_SLACK_TOKEN")
+ )
+
+ @repository
+ def my_repo():
+ return [my_job + slack_on_run_failure]
+ ```
+ ```python
+ def my_message_fn(context: RunFailureSensorContext) -> str:
+ return (
+ f"Job {context.dagster_run.job_name} failed!"
+ f"Error: {context.failure_event.message}"
+ )
+
+ slack_on_run_failure = make_slack_on_run_failure_sensor(
+ channel="#my_channel",
+ slack_token=os.getenv("MY_SLACK_TOKEN"),
+ text_fn=my_message_fn,
+ webserver_base_url="http://mycoolsite.com",
+ )
+ ```
+
+
+
+
+
+
dagster_slack.slack_on_failure HookDefinition
+
+
+ Create a hook on step failure events that will message the given Slack channel.
+
+ Parameters:
+ - channel (str) – The channel to send the message to (e.g. “#my_channel”)
+ - message_fn (Optional(Callable[[[*HookContext*](../hooks.mdx#dagster.HookContext)], str])) – Function which takes in the HookContext
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your webserver instance. Specify this to allow
+ - webserver_base_url – (Optional[str]): The base url of your webserver instance. Specify this to allow
+
+
+ Examples:
+
+ ```python
+ @slack_on_failure("#foo", webserver_base_url="http://localhost:3000")
+ @job(...)
+ def my_job():
+ pass
+ ```
+ ```python
+ def my_message_fn(context: HookContext) -> str:
+ return f"Op {context.op} failed!"
+
+ @op
+ def an_op(context):
+ pass
+
+ @job(...)
+ def my_job():
+ an_op.with_hooks(hook_defs={slack_on_failure("#foo", my_message_fn)})
+ ```
+
+
+
+
+
+
dagster_slack.slack_on_success HookDefinition
+
+
+ Create a hook on step success events that will message the given Slack channel.
+
+ Parameters:
+ - channel (str) – The channel to send the message to (e.g. “#my_channel”)
+ - message_fn (Optional(Callable[[[*HookContext*](../hooks.mdx#dagster.HookContext)], str])) – Function which takes in the HookContext
+ - dagit_base_url – deprecatedwebserver_base_url instead.) (Optional[str]): The base url of your webserver instance. Specify this to allow
+ - webserver_base_url – (Optional[str]): The base url of your webserver instance. Specify this to allow
+
+
+ Examples:
+
+ ```python
+ @slack_on_success("#foo", webserver_base_url="http://localhost:3000")
+ @job(...)
+ def my_job():
+ pass
+ ```
+ ```python
+ def my_message_fn(context: HookContext) -> str:
+ return f"Op {context.op} worked!"
+
+ @op
+ def an_op(context):
+ pass
+
+ @job(...)
+ def my_job():
+ an_op.with_hooks(hook_defs={slack_on_success("#foo", my_message_fn)})
+ ```
+
+
+
+
+
+
+## Legacy
+
+
+
dagster_slack.slack_resource ResourceDefinition
+
+
+ This resource is for connecting to Slack.
+
+ The resource object is a slack_sdk.WebClient.
+
+ By configuring this Slack resource, you can post messages to Slack from any Dagster op, asset, schedule or sensor.
+
+ Examples:
+
+ ```python
+ import os
+
+ from dagster import job, op
+ from dagster_slack import slack_resource
+
+
+ @op(required_resource_keys={'slack'})
+ def slack_op(context):
+ context.resources.slack.chat_postMessage(channel='#noise', text=':wave: hey there!')
+
+ @job(resource_defs={'slack': slack_resource})
+ def slack_job():
+ slack_op()
+
+ slack_job.execute_in_process(
+ run_config={'resources': {'slack': {'config': {'token': os.getenv('SLACK_TOKEN')}}}}
+ )
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-sling.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-sling.mdx
new file mode 100644
index 0000000000000..0b3043d75d243
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-sling.mdx
@@ -0,0 +1,454 @@
+---
+title: 'dlt (dagster-dlt)'
+title_meta: 'dlt (dagster-dlt) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dlt (dagster-dlt) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# dlt (dagster-dlt)
+
+This library provides a Dagster integration with [Sling](https://slingdata.io).
+
+For more information on getting started, see the [Sling & Dagster](https://docs.dagster.io/integrations/libraries/sling) documentation.
+
+
+
+
+## Assets
+
+
+
@dagster_sling.sling_assets
+
+
+ Create a definition for how to materialize a set of Sling replication streams as Dagster assets, as
+ described by a Sling replication config. This will create on Asset for every Sling target stream.
+
+ A Sling Replication config is a configuration that maps sources to destinations. For the full
+ spec and descriptions, see [Sling’s Documentation](https://docs.slingdata.io/sling-cli/run/configuration).
+
+ Parameters:
+ - replication_config (Union[Mapping[str, Any], str, Path]) – A path to a Sling replication config, or a dictionary
+ - dagster_sling_translator – (DagsterSlingTranslator): Allows customization of how to map a Sling stream to a Dagster
+ - (Optional[str] (name) – The name of the op.
+ - partitions_def (Optional[[*PartitionsDefinition*](../partitions.mdx#dagster.PartitionsDefinition)]) – The partitions definition for this asset.
+ - backfill_policy (Optional[[*BackfillPolicy*](../partitions.mdx#dagster.BackfillPolicy)]) – The backfill policy for this asset.
+ - op_tags (Optional[Mapping[str, Any]]) – The tags for the underlying op.
+
+
+ Examples:
+
+ Running a sync by providing a path to a Sling Replication config:
+
+ ```python
+ from dagster_sling import sling_assets, SlingResource, SlingConnectionResource
+
+ sling_resource = SlingResource(
+ connections=[
+ SlingConnectionResource(
+ name="MY_POSTGRES", type="postgres", connection_string=EnvVar("POSTGRES_URL")
+ ),
+ SlingConnectionResource(
+ name="MY_DUCKDB",
+ type="duckdb",
+ connection_string="duckdb:///var/tmp/duckdb.db",
+ ),
+ ]
+ )
+
+ config_path = "/path/to/replication.yaml"
+ @sling_assets(replication_config=config_path)
+ def my_assets(context, sling: SlingResource):
+ yield from sling.replicate(context=context)
+ ```
+
+
+
+
+
+
class dagster_sling.DagsterSlingTranslator
+
+
+
+
get_asset_key
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).key` instead..
+
+ :::
+
+ A function that takes a stream definition from a Sling replication config and returns a
+ Dagster AssetKey.
+
+ The stream definition is a dictionary key/value pair where the key is the stream name and
+ the value is a dictionary representing the Sling Replication Stream Config.
+
+ For example:
+
+ ```python
+ stream_definition = {"public.users":
+ {'sql': 'select all_user_id, name from public."all_Users"',
+ 'object': 'public.all_users'}
+ }
+ ```
+ By default, this returns the class’s target_prefix parameter concatenated with the stream name.
+ A stream named “public.accounts” will create an AssetKey named “target_public_accounts”.
+
+ Override this function to customize how to map a Sling stream to a Dagster AssetKey.
+
+ Alternatively, you can provide metadata in your Sling replication config to specify the
+ Dagster AssetKey for a stream as follows:
+
+ ```yaml
+ public.users:
+ meta:
+ dagster:
+ asset_key: "mydb_users"
+ ```
+ Parameters: stream_definition (Mapping[str, Any]) – A dictionary representing the stream definitionReturns: The Dagster AssetKey for the replication stream.Return type: [AssetKey](../assets.mdx#dagster.AssetKey)
+ Examples:
+
+ Using a custom mapping for streams:
+
+ ```python
+ class CustomSlingTranslator(DagsterSlingTranslator):
+ def get_asset_spec(self, stream_definition: Mapping[str, Any]) -> AssetKey:
+ default_spec = super().get_asset_spec(stream_definition)
+ map = {"stream1": "asset1", "stream2": "asset2"}
+ return default_spec.replace_attributes(key=AssetKey(map[stream_definition["name"]]))
+ ```
+
+
+
+
+
+
get_asset_spec
+
+
+ A function that takes a stream definition from a Sling replication config and returns a
+ Dagster AssetSpec.
+
+ The stream definition is a dictionary key/value pair where the key is the stream name and
+ the value is a dictionary representing the Sling Replication Stream Config.
+
+
+
+
+
+
+
get_auto_materialize_policy
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).auto_materialize_policy` instead..
+
+ :::
+
+ Defines the auto-materialize policy for a given stream definition.
+
+ This method checks the provided stream definition for a specific configuration
+ indicating an auto-materialize policy. If the configuration is found, it returns
+ an eager auto-materialize policy. Otherwise, it returns None.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: An eager auto-materialize policy if the configuration
+ is found, otherwise None.Return type: Optional[AutoMaterializePolicy]
+
+
+
+
+
+
get_deps_asset_key
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Iterate over `DagsterSlingTranslator.get_asset_spec(...).deps` to access `AssetDep.asset_key` instead..
+
+ :::
+
+ A function that takes a stream definition from a Sling replication config and returns a
+ Dagster AssetKey for each dependency of the replication stream.
+
+ By default, this returns the stream name. For example, a stream named “public.accounts”
+ will create an AssetKey named “target_public_accounts” and a dependency named “public_accounts”.
+
+ Override this function to customize how to map a Sling stream to a Dagster dependency.
+ Alternatively, you can provide metadata in your Sling replication config to specify the
+ Dagster AssetKey for a stream as follows:
+
+ ```yaml
+ public.users:
+ meta:
+ dagster:
+ deps: "sourcedb_users"
+ ```
+ Parameters: stream_definition (Mapping[str, Any]) – A dictionary representing the stream definitionReturns: A list of Dagster AssetKey for each dependency of the replication stream.Return type: Iterable[[AssetKey](../assets.mdx#dagster.AssetKey)]
+
+
+
+
+
+
get_description
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).description` instead..
+
+ :::
+
+ Retrieves the description for a given stream definition.
+
+ This method checks the provided stream definition for a description. It first looks
+ for an “sql” key in the configuration and returns its value if found. If not, it looks
+ for a description in the metadata under the “dagster” key.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: The description of the stream if found, otherwise None.Return type: Optional[str]
+
+
+
+
+
+
get_freshness_policy
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).freshness_policy` instead..
+
+ :::
+
+ Retrieves the freshness policy for a given stream definition.
+
+ This method checks the provided stream definition for a specific configuration
+ indicating a freshness policy. If the configuration is found, it constructs and
+ returns a FreshnessPolicy object based on the provided parameters. Otherwise,
+ it returns None.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: A FreshnessPolicy object if the configuration is found,
+ otherwise None.Return type: Optional[FreshnessPolicy]
+
+
+
+
+
+
get_group_name
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).group_name` instead..
+
+ :::
+
+ Retrieves the group name for a given stream definition.
+
+ This method checks the provided stream definition for a group name in the metadata
+ under the “dagster” key.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: The group name if found, otherwise None.Return type: Optional[str]
+
+
+
+
+
+
get_kinds
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).kinds` instead..
+
+ :::
+
+ Retrieves the kinds for a given stream definition.
+
+ This method returns “sling” by default. This method can be overridden to provide custom kinds.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: A set containing kinds for the stream’s assets.Return type: Set[str]
+
+
+
+
+
+
get_metadata
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).metadata` instead..
+
+ :::
+
+ Retrieves the metadata for a given stream definition.
+
+ This method extracts the configuration from the provided stream definition and returns
+ it as a JSON metadata value.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: A dictionary containing the stream configuration as JSON metadata.Return type: Mapping[str, Any]
+
+
+
+
+
+
get_tags
+
+
+ :::warning[superseded]
+ This API has been superseded.
+ Use `DagsterSlingTranslator.get_asset_spec(...).tags` instead..
+
+ :::
+
+ Retrieves the tags for a given stream definition.
+
+ This method returns an empty dictionary, indicating that no tags are associated with
+ the stream definition by default. This method can be overridden to provide custom tags.
+
+ Parameters:
+ - stream_definition (Mapping[str, Any]) – A dictionary representing the stream definition,
+ - details. (which includes configuration)
+
+
+ Returns: An empty dictionary.Return type: Mapping[str, Any]
+
+
+
+
+
+
sanitize_stream_name
+
+
+ A function that takes a stream name from a Sling replication config and returns a
+ sanitized name for the stream.
+
+ By default, this removes any non-alphanumeric characters from the stream name and replaces
+ them with underscores, while removing any double quotes.
+
+ Parameters: stream_name (str) – The name of the stream.
+ Examples:
+
+ Using a custom stream name sanitizer:
+
+ ```python
+ class CustomSlingTranslator(DagsterSlingTranslator):
+ def sanitize_stream_name(self, stream_name: str) -> str:
+ return stream_name.replace(".", "")
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Resources
+
+
+
class dagster_sling.SlingResource
+
+
+ Resource for interacting with the Sling package. This resource can be used to run Sling replications.
+
+ Parameters: connections (List[[*SlingConnectionResource*](#dagster_sling.SlingConnectionResource)]) – A list of connections to use for the replication.
+ Examples:
+
+ ```python
+ from dagster_etl.sling import SlingResource, SlingConnectionResource
+
+ sling_resource = SlingResource(
+ connections=[
+ SlingConnectionResource(
+ name="MY_POSTGRES",
+ type="postgres",
+ connection_string=EnvVar("POSTGRES_CONNECTION_STRING"),
+ ),
+ SlingConnectionResource(
+ name="MY_SNOWFLAKE",
+ type="snowflake",
+ host=EnvVar("SNOWFLAKE_HOST"),
+ user=EnvVar("SNOWFLAKE_USER"),
+ database=EnvVar("SNOWFLAKE_DATABASE"),
+ password=EnvVar("SNOWFLAKE_PASSWORD"),
+ role=EnvVar("SNOWFLAKE_ROLE"),
+ ),
+ ]
+ )
+ ```
+
+
+
+
+
+
class dagster_sling.SlingConnectionResource
+
+
+ A representation of a connection to a database or file to be used by Sling. This resource can be used as a source or a target for a Sling syncs.
+
+ Reference the Sling docs for more information on possible connection types and parameters: [https://docs.slingdata.io/connections](https://docs.slingdata.io/connections)
+
+ The name of the connection is passed to Sling and must match the name of the connection provided in the replication configuration: [https://docs.slingdata.io/sling-cli/run/configuration/replication](https://docs.slingdata.io/sling-cli/run/configuration/replication)
+ You may provide either a connection string or keyword arguments for the connection.
+
+ Examples:
+
+ Creating a Sling Connection for a file, such as CSV or JSON:
+
+ ```python
+ source = SlingConnectionResource(name="MY_FILE", type="file")
+ ```
+ Create a Sling Connection for a Postgres database, using a connection string:
+
+ ```python
+ postgres_conn = SlingConnectionResource(name="MY_POSTGRES", type="postgres", connection_string=EnvVar("POSTGRES_CONNECTION_STRING"))
+ mysql_conn = SlingConnectionResource(name="MY_MYSQL", type="mysql", connection_string="mysql://user:password@host:port/schema")
+ ```
+ Create a Sling Connection for a Postgres or Snowflake database, using keyword arguments:
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-snowflake-pandas.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-snowflake-pandas.mdx
new file mode 100644
index 0000000000000..af8c2faca8d0f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-snowflake-pandas.mdx
@@ -0,0 +1,238 @@
+---
+title: 'snowflake with pandas (dagster-snowflake-pandas)'
+title_meta: 'snowflake with pandas (dagster-snowflake-pandas) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'snowflake with pandas (dagster-snowflake-pandas) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Snowflake with Pandas (dagster-snowflake-pandas)
+
+This library provides an integration with the [Snowflake](https://www.snowflake.com/) data
+warehouse and Pandas data processing library.
+
+To use this library, you should first ensure that you have an appropriate [Snowflake user](https://docs.snowflake.net/manuals/user-guide/admin-user-management.html) configured to access
+your data warehouse.
+
+Related Guides:
+
+ - [Using Dagster with Snowflake guide](https://docs.dagster.io/integrations/snowflake/)
+ - [Snowflake I/O manager reference](https://docs.dagster.io/integrations/libraries/snowflake/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes Pandas DataFrames to Snowflake. When
+ using the SnowflakePandasIOManager, any inputs and outputs without type annotations will be loaded
+ as Pandas DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_snowflake_pandas import SnowflakePandasIOManager
+ from dagster import asset, Definitions, EnvVar
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": SnowflakePandasIOManager(database="MY_DATABASE", account=EnvVar("SNOWFLAKE_ACCOUNT"), ...)
+ }
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={
+ "io_manager" SnowflakePandasIOManager(database="my_database", schema="my_schema", ...)
+ }
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_snowflake_pandas.SnowflakePandasTypeHandler
+
+
+ Plugin for the Snowflake I/O Manager that can store and load Pandas DataFrames as Snowflake tables.
+
+ Examples:
+
+ ```python
+ from dagster_snowflake import SnowflakeIOManager
+ from dagster_snowflake_pandas import SnowflakePandasTypeHandler
+ from dagster_snowflake_pyspark import SnowflakePySparkTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MySnowflakeIOManager(SnowflakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [SnowflakePandasTypeHandler(), SnowflakePySparkTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MySnowflakeIOManager(database="MY_DATABASE", account=EnvVar("SNOWFLAKE_ACCOUNT"), ...)
+ }
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes Pandas DataFrames to Snowflake. When
+ using the snowflake_pandas_io_manager, any inputs and outputs without type annotations will be loaded
+ as Pandas DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_snowflake_pandas import snowflake_pandas_io_manager
+ from dagster import asset, Definitions
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": snowflake_pandas_io_manager.configured({
+ "database": "my_database",
+ "account" : {"env": "SNOWFLAKE_ACCOUNT"}
+ ...
+ })
+ }
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={"io_manager" snowflake_pandas_io_manager.configured(
+ {"database": "my_database", "schema": "my_schema", ...} # will be used as the schema
+ )}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-snowflake-pyspark.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-snowflake-pyspark.mdx
new file mode 100644
index 0000000000000..66199ad40a73a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-snowflake-pyspark.mdx
@@ -0,0 +1,251 @@
+---
+title: 'snowflake with pyspark (dagster-snowflake-pyspark)'
+title_meta: 'snowflake with pyspark (dagster-snowflake-pyspark) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'snowflake with pyspark (dagster-snowflake-pyspark) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Snowflake with PySpark (dagster-snowflake-pyspark)
+
+This library provides an integration with the [Snowflake](https://www.snowflake.com/) data
+warehouse and PySpark data processing library.
+
+To use this library, you should first ensure that you have an appropriate [Snowflake user](https://docs.snowflake.net/manuals/user-guide/admin-user-management.html) configured to access
+your data warehouse.
+
+Related Guides:
+
+ - [Using Dagster with Snowflake guide](https://docs.dagster.io/integrations/libraries/snowflake/)
+ - [Snowflake I/O manager reference](https://docs.dagster.io/integrations/libraries/snowflake/reference)
+
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to Snowflake. When
+ using the SnowflakePySparkIOManager, any inputs and outputs without type annotations will be loaded
+ as PySpark DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_snowflake_pyspark import SnowflakePySparkIOManager
+ from pyspark.sql import DataFrame
+ from dagster import Definitions, EnvVar
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": SnowflakePySparkIOManager(
+ database="my_database",
+ warehouse="my_warehouse", # required for SnowflakePySparkIOManager
+ account=EnvVar("SNOWFLAKE_ACCOUNT"),
+ password=EnvVar("SNOWFLAKE_PASSWORD"),
+ ...
+ )
+ }
+ )
+ ```
+ Note that the warehouse configuration value is required when using the SnowflakePySparkIOManager
+
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={
+ "io_manager" SnowflakePySparkIOManager(database="my_database", schema="my_schema", ...)
+ }
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: DataFrame) -> DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
+
class dagster_snowflake_pyspark.SnowflakePySparkTypeHandler
+
+
+ Plugin for the Snowflake I/O Manager that can store and load PySpark DataFrames as Snowflake tables.
+
+ Examples:
+
+ ```python
+ from dagster_snowflake import SnowflakeIOManager
+ from dagster_snowflake_pandas import SnowflakePandasTypeHandler
+ from dagster_snowflake_pyspark import SnowflakePySparkTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MySnowflakeIOManager(SnowflakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [SnowflakePandasTypeHandler(), SnowflakePySparkTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MySnowflakeIOManager(database="MY_DATABASE", account=EnvVar("SNOWFLAKE_ACCOUNT"), warehouse="my_warehouse", ...)
+ }
+ )
+ ```
+
+
+
+ An I/O manager definition that reads inputs from and writes PySpark DataFrames to Snowflake. When
+ using the snowflake_pyspark_io_manager, any inputs and outputs without type annotations will be loaded
+ as PySpark DataFrames.
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_snowflake_pyspark import snowflake_pyspark_io_manager
+ from pyspark.sql import DataFrame
+ from dagster import Definitions
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": snowflake_pyspark_io_manager.configured({
+ "database": "my_database",
+ "warehouse": "my_warehouse", # required for snowflake_pyspark_io_manager
+ "account" : {"env": "SNOWFLAKE_ACCOUNT"},
+ "password": {"env": "SNOWFLAKE_PASSWORD"},
+ ...
+ })
+ }
+ )
+ ```
+ Note that the warehouse configuration value is required when using the snowflake_pyspark_io_manager
+
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={"io_manager" snowflake_pyspark_io_manager.configured(
+ {"database": "my_database", "schema": "my_schema", ...} # will be used as the schema
+ )}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata “columns” to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: DataFrame) -> DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-snowflake.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-snowflake.mdx
new file mode 100644
index 0000000000000..146e5e22c4aab
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-snowflake.mdx
@@ -0,0 +1,522 @@
+---
+title: 'snowflake (dagster-snowflake)'
+title_meta: 'snowflake (dagster-snowflake) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'snowflake (dagster-snowflake) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Snowflake (dagster-snowflake)
+
+This library provides an integration with the [Snowflake](https://www.snowflake.com/) data
+warehouse.
+
+To use this library, you should first ensure that you have an appropriate [Snowflake user](https://docs.snowflake.net/manuals/user-guide/admin-user-management.html) configured to access
+your data warehouse.
+
+Related Guides:
+
+ - [Using Dagster with Snowflake](https://docs.dagster.io/integrations/libraries/snowflake/)
+ - [Snowflake I/O manager reference](https://docs.dagster.io/integrations/libraries/snowflake/reference)
+ - [Transitioning data pipelines from development to production](https://docs.dagster.io/guides/deploy/dev-to-prod)
+ - [Testing against production with Dagster+ Branch Deployments](https://docs.dagster.io/dagster-plus/features/ci-cd/branch-deployments/)
+
+
+
+
+ Base class for an IO manager definition that reads inputs from and writes outputs to Snowflake.
+
+ Examples:
+
+ ```python
+ from dagster_snowflake import SnowflakeIOManager
+ from dagster_snowflake_pandas import SnowflakePandasTypeHandler
+ from dagster_snowflake_pyspark import SnowflakePySparkTypeHandler
+ from dagster import Definitions, EnvVar
+
+ class MySnowflakeIOManager(SnowflakeIOManager):
+ @staticmethod
+ def type_handlers() -> Sequence[DbTypeHandler]:
+ return [SnowflakePandasTypeHandler(), SnowflakePySparkTypeHandler()]
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ defs = Definitions(
+ assets=[my_table],
+ resources={
+ "io_manager": MySnowflakeIOManager(database="my_database", account=EnvVar("SNOWFLAKE_ACCOUNT"), ...)
+ }
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={
+ "io_manager" MySnowflakeIOManager(database="my_database", schema="my_schema", ...)
+ }
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata `columns` to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+ A resource for connecting to the Snowflake data warehouse.
+
+ If connector configuration is not set, SnowflakeResource.get_connection() will return a
+ [snowflake.connector.Connection](https://docs.snowflake.com/en/developer-guide/python-connector/python-connector-api#object-connection)
+ object. If connector=”sqlalchemy” configuration is set, then SnowflakeResource.get_connection() will
+ return a [SQLAlchemy Connection](https://docs.sqlalchemy.org/en/20/core/connections.html#sqlalchemy.engine.Connection)
+ or a [SQLAlchemy raw connection](https://docs.sqlalchemy.org/en/20/core/connections.html#sqlalchemy.engine.Engine.raw_connection).
+
+ A simple example of loading data into Snowflake and subsequently querying that data is shown below:
+
+ Examples:
+
+ ```python
+ from dagster import job, op
+ from dagster_snowflake import SnowflakeResource
+
+ @op
+ def get_one(snowflake_resource: SnowflakeResource):
+ with snowflake_resource.get_connection() as conn:
+ # conn is a snowflake.connector.Connection object
+ conn.cursor().execute("SELECT 1")
+
+ @job
+ def my_snowflake_job():
+ get_one()
+
+ my_snowflake_job.execute_in_process(
+ resources={
+ 'snowflake_resource': SnowflakeResource(
+ account=EnvVar("SNOWFLAKE_ACCOUNT"),
+ user=EnvVar("SNOWFLAKE_USER"),
+ password=EnvVar("SNOWFLAKE_PASSWORD")
+ database="MY_DATABASE",
+ schema="MY_SCHEMA",
+ warehouse="MY_WAREHOUSE"
+ )
+ }
+ )
+ ```
+
+
+
+
+
+
class dagster_snowflake.SnowflakeConnection
+
+
+ A connection to Snowflake that can execute queries. In general this class should not be
+ directly instantiated, but rather used as a resource in an op or asset via the
+ [`snowflake_resource()`](#dagster_snowflake.snowflake_resource).
+
+ Note that the SnowflakeConnection is only used by the snowflake_resource. The Pythonic SnowflakeResource does
+ not use this SnowflakeConnection class.
+
+
+
execute_queries
+
+
+ Execute multiple queries in Snowflake.
+
+ Parameters:
+ - sql_queries (str) – List of queries to be executed in series
+ - parameters (Optional[Union[Sequence[Any], Mapping[Any, Any]]]) – Parameters to be passed to every query. See the
+ - fetch_results (bool) – If True, will return the results of the queries as a list. Defaults to False. If True
+ - use_pandas_result (bool) – If True, will return the results of the queries as a list of a Pandas DataFrames.
+
+
+ Returns: The results of the queries as a list if fetch_results or use_pandas_result is True,
+ otherwise returns None
+ Examples:
+
+ ```python
+ @op
+ def create_fresh_database(snowflake: SnowflakeResource):
+ queries = ["DROP DATABASE IF EXISTS MY_DATABASE", "CREATE DATABASE MY_DATABASE"]
+ snowflake.execute_queries(
+ sql_queries=queries
+ )
+ ```
+
+
+
+
+
+
execute_query
+
+
+ Execute a query in Snowflake.
+
+ Parameters:
+ - sql (str) – the query to be executed
+ - parameters (Optional[Union[Sequence[Any], Mapping[Any, Any]]]) – Parameters to be passed to the query. See the
+ - fetch_results (bool) – If True, will return the result of the query. Defaults to False. If True
+ - use_pandas_result (bool) – If True, will return the result of the query as a Pandas DataFrame.
+
+
+ Returns: The result of the query if fetch_results or use_pandas_result is True, otherwise returns None
+ Examples:
+
+ ```python
+ @op
+ def drop_database(snowflake: SnowflakeResource):
+ snowflake.execute_query(
+ "DROP DATABASE IF EXISTS MY_DATABASE"
+ )
+ ```
+
+
+
+
+
+
get_connection
+
+
+ Gets a connection to Snowflake as a context manager.
+
+ If using the execute_query, execute_queries, or load_table_from_local_parquet methods,
+ you do not need to create a connection using this context manager.
+
+ Parameters: raw_conn (bool) – If using the sqlalchemy connector, you can set raw_conn to True to create a raw
+ connection. Defaults to True.
+ Examples:
+
+ ```python
+ @op(
+ required_resource_keys={"snowflake"}
+ )
+ def get_query_status(query_id):
+ with context.resources.snowflake.get_connection() as conn:
+ # conn is a Snowflake Connection object or a SQLAlchemy Connection if
+ # sqlalchemy is specified as the connector in the Snowflake Resource config
+
+ return conn.get_query_status(query_id)
+ ```
+
+
+
+
+
+
load_table_from_local_parquet
+
+
+ Stores the content of a parquet file to a Snowflake table.
+
+ Parameters:
+ - src (str) – the name of the file to store in Snowflake
+ - table (str) – the name of the table to store the data. If the table does not exist, it will
+
+
+ Examples:
+
+ ```python
+ import pandas as pd
+ import pyarrow as pa
+ import pyarrow.parquet as pq
+
+ @op
+ def write_parquet_file(snowflake: SnowflakeResource):
+ df = pd.DataFrame({"one": [1, 2, 3], "ten": [11, 12, 13]})
+ table = pa.Table.from_pandas(df)
+ pq.write_table(table, "example.parquet')
+ snowflake.load_table_from_local_parquet(
+ src="example.parquet",
+ table="MY_TABLE"
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Data Freshness
+
+
+
dagster_snowflake.fetch_last_updated_timestamps
+
+
+ Fetch the last updated times of a list of tables in Snowflake.
+
+ If the underlying query to fetch the last updated time returns no results, a ValueError will be raised.
+
+ Parameters:
+ - snowflake_connection (Union[SqlDbConnection, [*SnowflakeConnection*](#dagster_snowflake.SnowflakeConnection)]) – A connection to Snowflake.
+ - schema (str) – The schema of the tables to fetch the last updated time for.
+ - tables (Sequence[str]) – A list of table names to fetch the last updated time for.
+ - database (Optional[str]) – The database of the table. Only required if the connection
+ - ignore_missing_tables (Optional[bool]) – If True, tables not found in Snowflake
+
+
+ Returns: A dictionary of table names to their last updated time in UTC.Return type: Mapping[str, datetime]
+
+
+
+
+
+
+
+
+
+
+## Ops
+
+
+
dagster_snowflake.snowflake_op_for_query
+
+
+ This function is an op factory that constructs an op to execute a snowflake query.
+
+ Note that you can only use snowflake_op_for_query if you know the query you’d like to
+ execute at graph construction time. If you’d like to execute queries dynamically during
+ job execution, you should manually execute those queries in your custom op using the
+ snowflake resource.
+
+ Parameters:
+ - sql (str) – The sql query that will execute against the provided snowflake resource.
+ - parameters (dict) – The parameters for the sql query.
+
+
+ Returns: Returns the constructed op definition.Return type: [OpDefinition](../ops.mdx#dagster.OpDefinition)
+
+
+
+ Builds an IO manager definition that reads inputs from and writes outputs to Snowflake.
+
+ Parameters:
+ - type_handlers (Sequence[DbTypeHandler]) – Each handler defines how to translate between
+ - default_load_type (Type) – When an input has no type annotation, load it as this type.
+
+
+ Returns: IOManagerDefinition
+ Examples:
+
+ ```python
+ from dagster_snowflake import build_snowflake_io_manager
+ from dagster_snowflake_pandas import SnowflakePandasTypeHandler
+ from dagster_snowflake_pyspark import SnowflakePySparkTypeHandler
+ from dagster import Definitions
+
+ @asset(
+ key_prefix=["my_prefix"]
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_second_table() -> pd.DataFrame: # the name of the asset will be the table name
+ ...
+
+ snowflake_io_manager = build_snowflake_io_manager([SnowflakePandasTypeHandler(), SnowflakePySparkTypeHandler()])
+
+ defs = Definitions(
+ assets=[my_table, my_second_table],
+ resources={
+ "io_manager": snowflake_io_manager.configured({
+ "database": "my_database",
+ "account" : {"env": "SNOWFLAKE_ACCOUNT"}
+ ...
+ })
+ }
+ )
+ ```
+ You can set a default schema to store the assets using the `schema` configuration value of the Snowflake I/O
+ Manager. This schema will be used if no other schema is specified directly on an asset or op.
+
+ ```python
+ defs = Definitions(
+ assets=[my_table]
+ resources={"io_manager" snowflake_io_manager.configured(
+ {"database": "my_database", "schema": "my_schema", ...} # will be used as the schema
+ )}
+ )
+ ```
+ On individual assets, you an also specify the schema where they should be stored using metadata or
+ by adding a `key_prefix` to the asset key. If both `key_prefix` and metadata are defined, the metadata will
+ take precedence.
+
+ ```python
+ @asset(
+ key_prefix=["my_schema"] # will be used as the schema in snowflake
+ )
+ def my_table() -> pd.DataFrame:
+ ...
+
+ @asset(
+ metadata={"schema": "my_schema"} # will be used as the schema in snowflake
+ )
+ def my_other_table() -> pd.DataFrame:
+ ...
+ ```
+ For ops, the schema can be specified by including a “schema” entry in output metadata.
+
+ ```python
+ @op(
+ out={"my_table": Out(metadata={"schema": "my_schema"})}
+ )
+ def make_my_table() -> pd.DataFrame:
+ ...
+ ```
+ If none of these is provided, the schema will default to “public”.
+
+ To only use specific columns of a table as input to a downstream op or asset, add the metadata `columns` to the
+ In or AssetIn.
+
+ ```python
+ @asset(
+ ins={"my_table": AssetIn("my_table", metadata={"columns": ["a"]})}
+ )
+ def my_table_a(my_table: pd.DataFrame) -> pd.DataFrame:
+ # my_table will just contain the data from column "a"
+ ...
+ ```
+
+
+
+ A resource for connecting to the Snowflake data warehouse. The returned resource object is an
+ instance of [`SnowflakeConnection`](#dagster_snowflake.SnowflakeConnection).
+
+ A simple example of loading data into Snowflake and subsequently querying that data is shown below:
+
+ Examples:
+
+ ```python
+ from dagster import job, op
+ from dagster_snowflake import snowflake_resource
+
+ @op(required_resource_keys={'snowflake'})
+ def get_one(context):
+ context.resources.snowflake.execute_query('SELECT 1')
+
+ @job(resource_defs={'snowflake': snowflake_resource})
+ def my_snowflake_job():
+ get_one()
+
+ my_snowflake_job.execute_in_process(
+ run_config={
+ 'resources': {
+ 'snowflake': {
+ 'config': {
+ 'account': {'env': 'SNOWFLAKE_ACCOUNT'},
+ 'user': {'env': 'SNOWFLAKE_USER'},
+ 'password': {'env': 'SNOWFLAKE_PASSWORD'},
+ 'database': {'env': 'SNOWFLAKE_DATABASE'},
+ 'schema': {'env': 'SNOWFLAKE_SCHEMA'},
+ 'warehouse': {'env': 'SNOWFLAKE_WAREHOUSE'},
+ }
+ }
+ }
+ }
+ )
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-spark.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-spark.mdx
new file mode 100644
index 0000000000000..f27607b42761b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-spark.mdx
@@ -0,0 +1,62 @@
+---
+title: 'spark (dagster-spark)'
+title_meta: 'spark (dagster-spark) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'spark (dagster-spark) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Spark (dagster-spark)
+
+
+
class dagster_spark.SparkOpError
+
+
+
+
+
+
+
dagster_spark.define_spark_config
+
+
+ Spark configuration.
+
+ See the Spark documentation for reference:
+ [https://spark.apache.org/docs/latest/submitting-applications.html](https://spark.apache.org/docs/latest/submitting-applications.html)
+
+
+
+
+
+
+
dagster_spark.create_spark_op
+
+
+
+
+
+
+
dagster_spark.construct_spark_shell_command
+
+ Constructs the spark-submit command for a Spark job.
+
+
+
+
+
+## Legacy
+
+
+
dagster_spark.spark_resource ResourceDefinition
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-ssh.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-ssh.mdx
new file mode 100644
index 0000000000000..84ef25f121454
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-ssh.mdx
@@ -0,0 +1,25 @@
+---
+title: 'ssh / sftp (dagster-ssh)'
+title_meta: 'ssh / sftp (dagster-ssh) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'ssh / sftp (dagster-ssh) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# SSH / SFTP (dagster-ssh)
+
+This library provides an integration with SSH and SFTP.
+
+
+
dagster_ssh.ssh_resource ResourceDefinition
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-tableau.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-tableau.mdx
new file mode 100644
index 0000000000000..b4e0a6afaa658
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-tableau.mdx
@@ -0,0 +1,137 @@
+---
+title: 'tableau (dagster-tableau)'
+title_meta: 'tableau (dagster-tableau) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'tableau (dagster-tableau) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Tableau (dagster-tableau)
+
+Dagster allows you to represent your Tableau workspace as assets, alongside other your other
+technologies like dbt and Sling. This allows you to see how your Tableau assets are connected to
+your other data assets, and how changes to other data assets might impact your Tableau workspace.
+
+
+
+
+## Tableau API
+
+Here, we provide interfaces to manage Tableau projects using the Tableau API.
+
+
+
+### Assets (Tableau API)
+
+
+
class dagster_tableau.TableauCloudWorkspace
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Represents a workspace in Tableau Cloud and provides utilities
+ to interact with Tableau APIs.
+
+
+
+
+
+
+
class dagster_tableau.TableauServerWorkspace
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Represents a workspace in Tableau Server and provides utilities
+ to interact with Tableau APIs.
+
+
+
+
+
+
+
class dagster_tableau.DagsterTableauTranslator
+
+ Translator class which converts raw response data from the Tableau API into AssetSpecs.
+ Subclass this class to implement custom logic for each type of Tableau content.
+
+
+
+
+
dagster_tableau.load_tableau_asset_specs
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns a list of AssetSpecs representing the Tableau content in the workspace.
+
+ Parameters:
+ - workspace (Union[[*TableauCloudWorkspace*](#dagster_tableau.TableauCloudWorkspace), [*TableauServerWorkspace*](#dagster_tableau.TableauServerWorkspace)]) – The Tableau workspace to fetch assets from.
+ - dagster_tableau_translator (Optional[Union[[*DagsterTableauTranslator*](#dagster_tableau.DagsterTableauTranslator), Type[[*DagsterTableauTranslator*](#dagster_tableau.DagsterTableauTranslator)]]]) – The translator to use to convert Tableau content into [`dagster.AssetSpec`](../assets.mdx#dagster.AssetSpec).
+
+
+ Returns: The set of assets representing the Tableau content in the workspace.Return type: List[[AssetSpec](../assets.mdx#dagster.AssetSpec)]
+
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Returns the AssetsDefinition of the materializable assets in the Tableau workspace.
+
+ Parameters:
+ - resource_key (str) – The resource key to use for the Tableau resource.
+ - specs (Sequence[[*AssetSpec*](../assets.mdx#dagster.AssetSpec)]) – The asset specs of the executable assets in the Tableau workspace.
+ - refreshable_workbook_ids (Optional[Sequence[str]]) –
+
+ A list of workbook IDs. The workbooks provided must
+ have extracts as data sources and be refreshable in Tableau.
+
+ When materializing your Tableau assets, the workbooks provided are refreshed,
+ refreshing their sheets and dashboards before pulling their data in Dagster.
+
+
+
+ Returns: The AssetsDefinition of the executable assets in the Tableau workspace.Return type: [AssetsDefinition](../assets.mdx#dagster.AssetsDefinition)
+
+
+
+ Parses a list of Tableau AssetSpecs provided as input and return two lists of AssetSpecs,
+ one for the Tableau external assets and another one for the Tableau materializable assets.
+
+ In Tableau, data sources are considered external assets,
+ while sheets and dashboards are considered materializable assets.
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-twilio.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-twilio.mdx
new file mode 100644
index 0000000000000..62f3759256415
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-twilio.mdx
@@ -0,0 +1,42 @@
+---
+title: 'twilio (dagster-twilio)'
+title_meta: 'twilio (dagster-twilio) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'twilio (dagster-twilio) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Twilio (dagster-twilio)
+
+This library provides an integration with Twilio.
+
+
+
dagster_twilio.TwilioResource ResourceDefinition
+
+
+
+
+ This resource is for connecting to Twilio.
+
+
+
+
+
+
+
+## Legacy
+
+
+
dagster_twilio.twilio_resource ResourceDefinition
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-wandb.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-wandb.mdx
new file mode 100644
index 0000000000000..2b8e9422c6338
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagster-wandb.mdx
@@ -0,0 +1,289 @@
+---
+title: 'weights & biases (dagster-wandb)'
+title_meta: 'weights & biases (dagster-wandb) API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'weights & biases (dagster-wandb) Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Weights & Biases (dagster-wandb)
+
+This library provides a Dagster integration with [Weights & Biases](https://wandb.ai/).
+
+Use Dagster and Weights & Biases (W&B) to orchestrate your MLOps pipelines and maintain ML assets.
+
+The integration with W&B makes it easy within Dagster to:
+
+ - use and create [W&B Artifacts](https://docs.wandb.ai/guides/artifacts).
+ - use and create Registered Models in the [W&B Model Registry](https://docs.wandb.ai/guides/models).
+ - run training jobs on dedicated compute using [W&B Launch](https://docs.wandb.ai/guides/launch).
+ - use the [wandb](https://github.com/wandb/wandb) client in ops and assets.
+
+
+
+
+
+## Useful links
+
+For a complete set of documentation, see [Dagster integration](https://docs.wandb.ai/guides/integrations/dagster) on the W&B website.
+
+For full-code examples, see [examples/with_wandb](https://github.com/dagster-io/dagster/tree/master/examples/with_wandb) in the Dagster’s Github repo.
+
+
+
+
+
+
+
+## Resource
+
+
+
dagster_wandb.wandb_resource ResourceDefinition
+
+
+
+
+ Dagster resource used to communicate with the W&B API. It’s useful when you want to use the
+ wandb client within your ops and assets. It’s a required resources if you are using the W&B IO
+ Manager.
+
+ It automatically authenticates using the provided API key.
+
+ For a complete set of documentation, see [Dagster integration](https://docs.wandb.ai/guides/integrations/dagster).
+
+ To configure this resource, we recommend using the [configured](https://legacy-docs.dagster.io/concepts/configuration/configured) method.
+
+ Example:
+
+ ```python
+ from dagster import job
+ from dagster_wandb import wandb_resource
+
+ my_wandb_resource = wandb_resource.configured({"api_key": {"env": "WANDB_API_KEY"}})
+
+ @job(resource_defs={"wandb_resource": my_wandb_resource})
+ def my_wandb_job():
+ ...
+ ```
+
+
+ Represents an execution error of the W&B Artifacts IO Manager.
+
+
+
+
+
+
+
+
+## Ops
+
+
+
dagster_wandb.run_launch_agent
+
+
+ It starts a Launch Agent and runs it as a long running process until stopped manually.
+
+ Agents are processes that poll launch queues and execute the jobs (or dispatch them to external
+ services to be executed) in order.
+
+ Example:
+
+ ```YAML
+ # config.yaml
+
+ resources:
+ wandb_config:
+ config:
+ entity: my_entity
+ project: my_project
+ ops:
+ run_launch_agent:
+ config:
+ max_jobs: -1
+ queues:
+ - my_dagster_queue
+ ```
+ ```python
+ from dagster_wandb.launch.ops import run_launch_agent
+ from dagster_wandb.resources import wandb_resource
+
+ from dagster import job, make_values_resource
+
+
+ @job(
+ resource_defs={
+ "wandb_config": make_values_resource(
+ entity=str,
+ project=str,
+ ),
+ "wandb_resource": wandb_resource.configured(
+ {"api_key": {"env": "WANDB_API_KEY"}}
+ ),
+ },
+ )
+ def run_launch_agent_example():
+ run_launch_agent()
+ ```
+
+
+
+
+
+
dagster_wandb.run_launch_job
+
+
+ Executes a Launch job.
+
+ A Launch job is assigned to a queue in order to be executed. You can create a queue or use the
+ default one. Make sure you have an active agent listening to that queue. You can run an agent
+ inside your Dagster instance but can also consider using a deployable agent in Kubernetes.
+
+ Example:
+
+ ```YAML
+ # config.yaml
+
+ resources:
+ wandb_config:
+ config:
+ entity: my_entity
+ project: my_project
+ ops:
+ my_launched_job:
+ config:
+ entry_point:
+ - python
+ - train.py
+ queue: my_dagster_queue
+ uri: https://github.com/wandb/example-dagster-integration-with-launch
+ ```
+ ```python
+ from dagster_wandb.launch.ops import run_launch_job
+ from dagster_wandb.resources import wandb_resource
+
+ from dagster import job, make_values_resource
+
+
+ @job(
+ resource_defs={
+ "wandb_config": make_values_resource(
+ entity=str,
+ project=str,
+ ),
+ "wandb_resource": wandb_resource.configured(
+ {"api_key": {"env": "WANDB_API_KEY"}}
+ ),
+ },
+ )
+ def run_launch_job_example():
+ run_launch_job.alias("my_launched_job")() # we rename the job with an alias
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagstermill.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagstermill.mdx
new file mode 100644
index 0000000000000..e65c0cdbdbd27
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/libraries/dagstermill.mdx
@@ -0,0 +1,269 @@
+---
+title: 'dagstermill'
+title_meta: 'dagstermill API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dagstermill Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Dagstermill
+
+This library provides an integration with papermill to allow you to run Jupyter notebooks with Dagster.
+
+Related Guides:
+
+ - [Using Jupyter notebooks with Papermill and Dagster](https://docs.dagster.io/integrations/libraries/jupyter/)
+
+
+
+
dagstermill.define_dagstermill_asset
+
+
+ Creates a Dagster asset for a Jupyter notebook.
+
+ Parameters:
+ - name (str) – The name for the asset
+ - notebook_path (str) – Path to the backing notebook
+ - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the
+ - ins (Optional[Mapping[str, [*AssetIn*](../assets.mdx#dagster.AssetIn)]]) – A dictionary that maps input names to information
+ - deps (Optional[Sequence[Union[[*AssetsDefinition*](../assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](../assets.mdx#dagster.SourceAsset), [*AssetKey*](../assets.mdx#dagster.AssetKey), str]]]) – The assets
+ - config_schema (Optional[[*ConfigSchema*](../config.mdx#dagster.ConfigSchema)) – The configuration schema for the asset’s underlying
+ - metadata (Optional[Dict[str, Any]]) – A dict of metadata entries for the asset.
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the notebook.
+ - description (Optional[str]) – Description of the asset to display in the Dagster UI.
+ - partitions_def (Optional[[*PartitionsDefinition*](../partitions.mdx#dagster.PartitionsDefinition)]) – Defines the set of partition keys that
+ - op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset.
+ - group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided,
+ - resource_defs (Optional[Mapping[str, [*ResourceDefinition*](../resources.mdx#dagster.ResourceDefinition)]]) – (Experimental) A mapping of resource keys to resource definitions. These resources
+ - io_manager_key (Optional[str]) – A string key for the IO manager used to store the output notebook.
+ - retry_policy (Optional[[*RetryPolicy*](../ops.mdx#dagster.RetryPolicy)]) – The retry policy for the op that computes the asset.
+ - save_notebook_on_failure (bool) – If True and the notebook fails during execution, the failed notebook will be
+ - asset_tags (Optional[Dict[str, Any]]) – A dictionary of tags to apply to the asset.
+ - non_argument_deps (Optional[Union[Set[[*AssetKey*](../assets.mdx#dagster.AssetKey)], Set[str]]]) – Deprecated, use deps instead. Set of asset keys that are
+
+
+ Examples:
+
+ ```python
+ from dagstermill import define_dagstermill_asset
+ from dagster import asset, AssetIn, AssetKey
+ from sklearn import datasets
+ import pandas as pd
+ import numpy as np
+
+ @asset
+ def iris_dataset():
+ sk_iris = datasets.load_iris()
+ return pd.DataFrame(
+ data=np.c_[sk_iris["data"], sk_iris["target"]],
+ columns=sk_iris["feature_names"] + ["target"],
+ )
+
+ iris_kmeans_notebook = define_dagstermill_asset(
+ name="iris_kmeans_notebook",
+ notebook_path="/path/to/iris_kmeans.ipynb",
+ ins={
+ "iris": AssetIn(key=AssetKey("iris_dataset"))
+ }
+ )
+ ```
+
+
+
+
+
+
dagstermill.define_dagstermill_op
+
+
+ Wrap a Jupyter notebook in a op.
+
+ Parameters:
+ - name (str) – The name of the op.
+ - notebook_path (str) – Path to the backing notebook.
+ - ins (Optional[Mapping[str, [*In*](../ops.mdx#dagster.In)]]) – The op’s inputs.
+ - outs (Optional[Mapping[str, [*Out*](../ops.mdx#dagster.Out)]]) – The op’s outputs. Your notebook should
+ - required_resource_keys (Optional[Set[str]]) – The string names of any required resources.
+ - output_notebook_name – (Optional[str]): If set, will be used as the name of an injected output
+ - asset_key_prefix (Optional[Union[List[str], str]]) – If set, will be used to prefix the
+ - description (Optional[str]) – If set, description used for op.
+ - tags (Optional[Dict[str, str]]) – If set, additional tags used to annotate op.
+ - io_manager_key (Optional[str]) – If using output_notebook_name, you can additionally provide
+ - save_notebook_on_failure (bool) – If True and the notebook fails during execution, the failed notebook will be
+
+
+ Returns: [`OpDefinition`](../ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
class dagstermill.ConfigurableLocalOutputNotebookIOManager
+
+ Built-in IO Manager for handling output notebook.
+
+
+
+
+
dagstermill.get_context
+
+
+ Get a dagstermill execution context for interactive exploration and development.
+
+ Parameters:
+ - op_config (Optional[Any]) – If specified, this value will be made available on the
+ - resource_defs (Optional[Mapping[str, [*ResourceDefinition*](../resources.mdx#dagster.ResourceDefinition)]]) – Specifies resources to provide to context.
+ - logger_defs (Optional[Mapping[str, [*LoggerDefinition*](../loggers.mdx#dagster.LoggerDefinition)]]) – Specifies loggers to provide to context.
+ - run_config (Optional[dict]) – The config dict with which to construct
+
+
+ Returns: [`DagstermillExecutionContext`](#dagstermill.DagstermillExecutionContext)
+
+
+
+
+
+
dagstermill.yield_event
+
+
+ Yield a dagster event directly from notebook code.
+
+ When called interactively or in development, returns its input.
+
+ Parameters: dagster_event (Union[[`dagster.AssetMaterialization`](../ops.mdx#dagster.AssetMaterialization), [`dagster.ExpectationResult`](../ops.mdx#dagster.ExpectationResult), [`dagster.TypeCheck`](../ops.mdx#dagster.TypeCheck), [`dagster.Failure`](../ops.mdx#dagster.Failure), [`dagster.RetryRequested`](../ops.mdx#dagster.RetryRequested)]) – An event to yield back to Dagster.
+
+
+
+
+
+
dagstermill.yield_result
+
+
+ Yield a result directly from notebook code.
+
+ When called interactively or in development, returns its input.
+
+ Parameters:
+ - value (Any) – The value to yield.
+ - output_name (Optional[str]) – The name of the result to yield (default: `'result'`).
+
+
+
+
+
+
+
+
class dagstermill.DagstermillExecutionContext
+
+
+ Dagstermill-specific execution context.
+
+ Do not initialize directly: use [`dagstermill.get_context()`](#dagstermill.get_context).
+
+
+
property job_def
+
+
+ The job definition for the context.
+
+ This will be a dagstermill-specific shim.
+
+ Type: [`dagster.JobDefinition`](../jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
property job_name
+
+
+ The name of the executing job.
+
+ Type: str
+
+
+
+
+
+
property logging_tags
+
+
+ The logging tags for the context.
+
+ Type: dict
+
+
+
+
+
+
property op_config
+
+
+ A dynamically-created type whose properties allow access to
+ op-specific config.
+
+ Type: collections.namedtuple
+
+
+
+
+
+
property op_def
+
+
+ The op definition for the context.
+
+ In interactive contexts, this may be a dagstermill-specific shim, depending whether an
+ op definition was passed to `dagstermill.get_context`.
+
+ Type: [`dagster.OpDefinition`](../ops.mdx#dagster.OpDefinition)
+
+
+
+
+
+
property run
+
+
+ The job run for the context.
+
+ Type: [`dagster.DagsterRun`](../internals.mdx#dagster.DagsterRun)
+
+
+
+
+
+
property run_config
+
+
+ The run_config for the context.
+
+ Type: dict
+
+
+
+
+
+
property run_id
+
+
+ The run_id for the context.
+
+ Type: str
+
+
+
+
+
+
+
+
+
+
class dagstermill.DagstermillError
+
+ Base class for errors raised by dagstermill.
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/loggers.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/loggers.mdx
new file mode 100644
index 0000000000000..6f3e79040bf68
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/loggers.mdx
@@ -0,0 +1,244 @@
+---
+title: 'loggers'
+title_meta: 'loggers API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'loggers Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Loggers
+
+
+
+
+## Built-in loggers
+
+
+
dagster._loggers.colored_console_logger
+
+
+ Core class for defining loggers.
+
+ Loggers are job-scoped logging handlers, which will be automatically invoked whenever
+ dagster messages are logged from within a job.
+
+ Parameters:
+ - logger_fn (Callable[[[*InitLoggerContext*](#dagster.InitLoggerContext)], logging.Logger]) – User-provided function to
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of this logger.
+
+
+
+
+
+
+
+
dagster._loggers.json_console_logger
+
+
+ Core class for defining loggers.
+
+ Loggers are job-scoped logging handlers, which will be automatically invoked whenever
+ dagster messages are logged from within a job.
+
+ Parameters:
+ - logger_fn (Callable[[[*InitLoggerContext*](#dagster.InitLoggerContext)], logging.Logger]) – User-provided function to
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of this logger.
+
+
+
+
+
+
+
+
+
+
+
+
+## Logging from an @op
+
+
+
class dagster.DagsterLogManager
+
+
+ Centralized dispatch for logging from user code.
+
+ Handles the construction of uniform structured log messages and passes them through to the
+ underlying loggers/handlers.
+
+ An instance of the log manager is made available to ops as `context.log`. Users should not
+ initialize instances of the log manager directly. To configure custom loggers, set the
+ `logger_defs` argument in an @job decorator or when calling the to_job() method on a
+ [`GraphDefinition`](graphs.mdx#dagster.GraphDefinition).
+
+ The log manager inherits standard convenience methods like those exposed by the Python standard
+ library `python:logging` module (i.e., within the body of an op,
+ `context.log.\{debug, info, warning, warn, error, critical, fatal}`).
+
+ The underlying integer API can also be called directly using, e.g.
+ `context.log.log(5, msg)`, and the log manager will delegate to the `log` method
+ defined on each of the loggers it manages.
+
+ User-defined custom log levels are not supported, and calls to, e.g.,
+ `context.log.trace` or `context.log.notice` will result in hard exceptions at runtime.
+
+
+
+
+
+
+
+
+
+
+## Defining custom loggers
+
+
+
@dagster.logger
+
+
+ Define a logger.
+
+ The decorated function should accept an [`InitLoggerContext`](#dagster.InitLoggerContext) and return an instance of
+ `python:logging.Logger`. This function will become the `logger_fn` of an underlying
+ [`LoggerDefinition`](#dagster.LoggerDefinition).
+
+ Parameters:
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of the logger.
+
+
+
+
+
+
+
+
class dagster.LoggerDefinition
+
+
+ Core class for defining loggers.
+
+ Loggers are job-scoped logging handlers, which will be automatically invoked whenever
+ dagster messages are logged from within a job.
+
+ Parameters:
+ - logger_fn (Callable[[[*InitLoggerContext*](#dagster.InitLoggerContext)], logging.Logger]) – User-provided function to
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of this logger.
+
+
+
+
property config_schema
+
+
+ The schema for the logger’s config. Configuration data available in init_context.logger_config.
+
+ Type: Any
+
+
+
+
+
+
property description
+
+
+ A human-readable description of the logger.
+
+ Type: Optional[str]
+
+
+
+
+
+
property logger_fn
+
+
+ The function that will be invoked to
+ instantiate the logger.
+
+ Type: Callable[[[InitLoggerContext](#dagster.InitLoggerContext)], logging.Logger]
+
+
+
+
+
+
+
+
+
+
class dagster.InitLoggerContext
+
+
+ The context object available as the argument to the initialization function of a [`dagster.LoggerDefinition`](#dagster.LoggerDefinition).
+
+ Users should not instantiate this object directly. To construct an
+ InitLoggerContext for testing purposes, use `dagster.
+ build_init_logger_context()`.
+
+ Example:
+
+ ```python
+ from dagster import logger, InitLoggerContext
+
+ @logger
+ def hello_world(init_context: InitLoggerContext):
+ ...
+ ```
+
+
property logger_config
+
+ The configuration data provided by the run config. The
+ schema for this data is defined by `config_schema` on the [`LoggerDefinition`](#dagster.LoggerDefinition).
+
+
+
+
+
property logger_def
+
+ The logger definition for the logger being constructed.
+
+
+
+
+
property run_id
+
+ The ID for this run of the job.
+
+
+
+
+
+
+
+
+
dagster.build_init_logger_context
+
+
+ Builds logger initialization context from provided parameters.
+
+ This function can be used to provide the context argument to the invocation of a logger
+ definition.
+
+ Note that you may only specify one of pipeline_def and job_def.
+
+ Parameters:
+ - logger_config (Any) – The config to provide during initialization of logger.
+ - job_def (Optional[[*JobDefinition*](jobs.mdx#dagster.JobDefinition)]) – The job definition that the logger will be used with.
+
+
+ Examples:
+
+ ```python
+ context = build_init_logger_context()
+ logger_to_init(context)
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/metadata.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/metadata.mdx
new file mode 100644
index 0000000000000..e358a3442f4d6
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/metadata.mdx
@@ -0,0 +1,1239 @@
+---
+title: 'metadata'
+title_meta: 'metadata API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'metadata Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Metadata
+
+Dagster uses metadata to communicate arbitrary user-specified metadata about structured
+events.
+
+Refer to the [Metadata](https://docs.dagster.io/guides/build/assets/metadata-and-tags/) documentation for more information.
+
+
+
class dagster.MetadataValue
+
+
+ Utility class to wrap metadata values passed into Dagster events so that they can be
+ displayed in the Dagster UI and other tooling.
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "my_text_label": "hello",
+ "dashboard_url": MetadataValue.url("http://mycoolsite.com/my_dashboard"),
+ "num_rows": 0,
+ },
+ )
+ ```
+
+
static asset
+
+
+ Static constructor for a metadata value referencing a Dagster asset, by key.
+
+ For example:
+
+ ```python
+ @op
+ def validate_table(context, df):
+ yield AssetMaterialization(
+ asset_key=AssetKey("my_table"),
+ metadata={
+ "Related asset": MetadataValue.asset(AssetKey('my_other_table')),
+ },
+ )
+ ```
+ Parameters: asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – The asset key referencing the asset.
+
+
+
+
+
+
static bool
+
+
+ Static constructor for a metadata value wrapping a bool as
+ `BoolMetadataValuye`. Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "num rows > 1000": MetadataValue.bool(len(df) > 1000),
+ },
+ )
+ ```
+ Parameters: value (bool) – The bool value for a metadata entry.
+
+
+
+
+
+
static column_lineage
+
+
+ Static constructor for a metadata value wrapping a column lineage as
+ [`TableColumnLineageMetadataValue`](#dagster.TableColumnLineageMetadataValue). Can be used as the value type
+ for the metadata parameter for supported events.
+
+ Parameters: lineage ([*TableColumnLineage*](#dagster.TableColumnLineage)) – The column lineage for a metadata entry.
+
+
+
+
+
+
static dagster_run
+
+
+ Static constructor for a metadata value wrapping a reference to a Dagster run.
+
+ Parameters: run_id (str) – The ID of the run.
+
+
+
+
+
+
static float
+
+
+ Static constructor for a metadata value wrapping a float as
+ [`FloatMetadataValue`](#dagster.FloatMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "size (bytes)": MetadataValue.float(calculate_bytes(df)),
+ }
+ )
+ ```
+ Parameters: value (float) – The float value for a metadata entry.
+
+
+
+
+
+
static int
+
+
+ Static constructor for a metadata value wrapping an int as
+ [`IntMetadataValue`](#dagster.IntMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "number of rows": MetadataValue.int(len(df)),
+ },
+ )
+ ```
+ Parameters: value (int) – The int value for a metadata entry.
+
+
+
+
+
+
static job
+
+
+ Static constructor for a metadata value referencing a Dagster job, by name.
+
+ For example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset"
+ metadata={
+ "Producing job": MetadataValue.job('my_other_job'),
+ },
+ )
+ ```
+ Parameters:
+ - job_name (str) – The name of the job.
+ - location_name (Optional[str]) – The code location name for the job.
+ - repository_name (Optional[str]) – The repository name of the job, if different from the
+
+
+
+
+
+
+
+
static json
+
+
+ Static constructor for a metadata value wrapping a json-serializable list or dict
+ as [`JsonMetadataValue`](#dagster.JsonMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context):
+ yield ExpectationResult(
+ success=not missing_things,
+ label="is_present",
+ metadata={
+ "about my dataset": MetadataValue.json({"missing_columns": missing_things})
+ },
+ )
+ ```
+ Parameters: data (Union[Sequence[Any], Mapping[str, Any]]) – The JSON data for a metadata entry.
+
+
+
+
+
+
static md
+
+
+ Static constructor for a metadata value wrapping markdown data as
+ [`MarkdownMetadataValue`](#dagster.MarkdownMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, md_str):
+ yield AssetMaterialization(
+ asset_key="info",
+ metadata={
+ 'Details': MetadataValue.md(md_str)
+ },
+ )
+ ```
+ Parameters: md_str (str) – The markdown for a metadata entry.
+
+
+
+
+
+
static notebook
+
+
+ Static constructor for a metadata value wrapping a notebook path as
+ [`NotebookMetadataValue`](#dagster.NotebookMetadataValue).
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "notebook_path": MetadataValue.notebook("path/to/notebook.ipynb"),
+ }
+ )
+ ```
+ Parameters: path (str) – The path to a notebook for a metadata entry.
+
+
+
+
+
+
static null
+
+ Static constructor for a metadata value representing null. Can be used as the value type
+ for the metadata parameter for supported events.
+
+
+
+
+
static path
+
+
+ Static constructor for a metadata value wrapping a path as
+ [`PathMetadataValue`](#dagster.PathMetadataValue).
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "filepath": MetadataValue.path("path/to/file"),
+ }
+ )
+ ```
+ Parameters: path (str) – The path for a metadata entry.
+
+
+
+
+
+
static python_artifact
+
+
+ Static constructor for a metadata value wrapping a python artifact as
+ [`PythonArtifactMetadataValue`](#dagster.PythonArtifactMetadataValue). Can be used as the value type for the
+ metadata parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "class": MetadataValue.python_artifact(MyClass),
+ "function": MetadataValue.python_artifact(my_function),
+ }
+ )
+ ```
+ Parameters: value (Callable) – The python class or function for a metadata entry.
+
+
+
+
+
+
static table
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Static constructor for a metadata value wrapping arbitrary tabular data as
+ [`TableMetadataValue`](#dagster.TableMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context):
+ yield ExpectationResult(
+ success=not has_errors,
+ label="is_valid",
+ metadata={
+ "errors": MetadataValue.table(
+ records=[
+ TableRecord(code="invalid-data-type", row=2, col="name"),
+ ],
+ schema=TableSchema(
+ columns=[
+ TableColumn(name="code", type="string"),
+ TableColumn(name="row", type="int"),
+ TableColumn(name="col", type="string"),
+ ]
+ )
+ ),
+ },
+ )
+ ```
+
+
+
+
+
+
static table_schema
+
+
+ Static constructor for a metadata value wrapping a table schema as
+ [`TableSchemaMetadataValue`](#dagster.TableSchemaMetadataValue). Can be used as the value type
+ for the metadata parameter for supported events.
+
+ Example:
+
+ ```python
+ schema = TableSchema(
+ columns = [
+ TableColumn(name="id", type="int"),
+ TableColumn(name="status", type="bool"),
+ ]
+ )
+
+ DagsterType(
+ type_check_fn=some_validation_fn,
+ name='MyTable',
+ metadata={
+ 'my_table_schema': MetadataValue.table_schema(schema),
+ }
+ )
+ ```
+ Parameters: schema ([*TableSchema*](#dagster.TableSchema)) – The table schema for a metadata entry.
+
+
+
+
+
+
static text
+
+
+ Static constructor for a metadata value wrapping text as
+ [`TextMetadataValue`](#dagster.TextMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context, df):
+ yield AssetMaterialization(
+ asset_key="my_dataset",
+ metadata={
+ "my_text_label": MetadataValue.text("hello")
+ },
+ )
+ ```
+ Parameters: text (str) – The text string for a metadata entry.
+
+
+
+
+
+
static timestamp
+
+
+ Static constructor for a metadata value wrapping a UNIX timestamp as a
+ [`TimestampMetadataValue`](#dagster.TimestampMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Parameters: value (Union[float, datetime]) – The unix timestamp value for a metadata entry. If a
+ datetime is provided, the timestamp will be extracted. datetimes without timezones
+ are not accepted, because their timestamps can be ambiguous.
+
+
+
+
+
+
static url
+
+
+ Static constructor for a metadata value wrapping a URL as
+ [`UrlMetadataValue`](#dagster.UrlMetadataValue). Can be used as the value type for the metadata
+ parameter for supported events.
+
+ Example:
+
+ ```python
+ @op
+ def emit_metadata(context):
+ yield AssetMaterialization(
+ asset_key="my_dashboard",
+ metadata={
+ "dashboard_url": MetadataValue.url("http://mycoolsite.com/my_dashboard"),
+ }
+ )
+ ```
+ Parameters: url (str) – The URL for a metadata entry.
+
+
+
+
+
+
abstract property value
+
+ The wrapped value.
+
+
+
+
+
+
+
+
+
class dagster.MetadataEntry
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Please use a dict with `MetadataValue` values instead..
+
+ :::
+
+ A structure for describing metadata for Dagster events.
+
+ Note: This class is no longer usable in any Dagster API, and will be completely removed in 2.0.
+
+ Lists of objects of this type can be passed as arguments to Dagster events and will be displayed
+ in the Dagster UI and other tooling.
+
+ Should be yielded from within an IO manager to append metadata for a given input/output event.
+ For other event types, passing a dict with MetadataValue values to the metadata argument
+ is preferred.
+
+ Parameters:
+ - label (str) – Short display label for this metadata entry.
+ - description (Optional[str]) – A human-readable description of this metadata entry.
+ - value ([*MetadataValue*](#dagster.MetadataValue)) – Typed metadata entry data. The different types allow
+
+
+
+
+
+
+
+
+
+## Metadata types
+
+All metadata types inherit from MetadataValue. The following types are defined:
+
+
+
class dagster.DagsterAssetMetadataValue
+
+
+ Representation of a dagster asset.
+
+ Parameters: asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – The dagster asset key
+
+
+ Container class for markdown metadata entry data.
+
+ Parameters: md_str (Optional[str]) – The markdown as a string.
+
+
property value
+
+
+ The wrapped markdown as a string.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.PathMetadataValue
+
+
+ Container class for path metadata entry data.
+
+ Parameters: path (Optional[str]) – The path as a string or conforming to os.PathLike.
+
+
property value
+
+
+ The wrapped path.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.NotebookMetadataValue
+
+
+ Container class for notebook metadata entry data.
+
+ Parameters: path (Optional[str]) – The path to the notebook as a string or conforming to os.PathLike.
+
+
property value
+
+
+ The wrapped path to the notebook as a string.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.PythonArtifactMetadataValue
+
+
+ Container class for python artifact metadata entry data.
+
+ Parameters:
+ - module (str) – The module where the python artifact can be found
+ - name (str) – The name of the python artifact
+
+
+
+
+ Representation of the lineage of column inputs to column outputs of arbitrary tabular data.
+
+ Parameters: column_lineage ([*TableColumnLineage*](#dagster.TableColumnLineage)) – The lineage of column inputs to column outputs
+ for the table.
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Container class for table metadata entry data.
+
+ Parameters:
+ - records ([*TableRecord*](#dagster.TableRecord)) – The data as a list of records (i.e. rows).
+ - schema (Optional[[*TableSchema*](#dagster.TableSchema)]) – A schema for the table.
+
+
+ Example:
+
+ ```python
+ from dagster import TableMetadataValue, TableRecord
+
+ TableMetadataValue(
+ schema=None,
+ records=[
+ TableRecord({"column1": 5, "column2": "x"}),
+ TableRecord({"column1": 7, "column2": "y"}),
+ ]
+ )
+ ```
+
+
static infer_column_type
+
+ str: Infer the [`TableSchema`](#dagster.TableSchema) column type that will be used for a value.
+
+
+ Representation of a schema for arbitrary tabular data.
+
+ Parameters: schema ([*TableSchema*](#dagster.TableSchema)) – The dictionary containing the schema representation.
+
+
+ Container class for text metadata entry data.
+
+ Parameters: text (Optional[str]) – The text data.
+
+
property value
+
+
+ The wrapped text data.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.TimestampMetadataValue
+
+
+ Container class for metadata value that’s a unix timestamp.
+
+ Parameters: value (float) – Seconds since the unix epoch.
+
+
+
+
+
+
class dagster.UrlMetadataValue
+
+
+ Container class for URL metadata entry data.
+
+ Parameters: url (Optional[str]) – The URL as a string.
+
+
property value
+
+
+ The wrapped URL.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
class dagster.CodeReferencesMetadataValue
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Metadata value type which represents source locations (locally or otherwise)
+ of the asset in question. For example, the file path and line number where the
+ asset is defined.
+
+
+
sources
+
+
+ A list of code references for the asset, such as file locations or
+ references to source control.
+
+ Type: List[Union[LocalFileCodeReference, SourceControlCodeReference]]
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Tables
+
+These APIs provide the ability to express column schemas (TableSchema), rows/records (TableRecord), and column lineage (TableColumnLineage) in Dagster as metadata.
+
+
+
class dagster.TableRecord
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Represents one record in a table. Field keys are arbitrary strings– field values must be
+ strings, integers, floats, or bools.
+
+
+
+
+
+
+
class dagster.TableSchema
+
+
+ Representation of a schema for tabular data.
+
+ Schema is composed of two parts:
+
+ - A required list of columns (TableColumn). Each column specifies a
+ - An optional list of table-level constraints (TableConstraints). A
+
+
+ ```python
+ # example schema
+ TableSchema(
+ constraints = TableConstraints(
+ other = [
+ "foo > bar",
+ ],
+ ),
+ columns = [
+ TableColumn(
+ name = "foo",
+ type = "string",
+ description = "Foo description",
+ constraints = TableColumnConstraints(
+ nullable = False,
+ other = [
+ "starts with the letter 'a'",
+ ],
+ ),
+ ),
+ TableColumn(
+ name = "bar",
+ type = "string",
+ ),
+ TableColumn(
+ name = "baz",
+ type = "custom_type",
+ constraints = TableColumnConstraints(
+ unique = True,
+ )
+ ),
+ ],
+ )
+ ```
+ Parameters:
+ - columns (List[[*TableColumn*](#dagster.TableColumn)]) – The columns of the table.
+ - constraints (Optional[[*TableConstraints*](#dagster.TableConstraints)]) – The constraints of the table.
+
+
+
+
static from_name_type_dict
+
+ Constructs a TableSchema from a dictionary whose keys are column names and values are the
+ names of data types of those columns.
+
+
+
+
+
+
+
+
+
class dagster.TableConstraints
+
+
+ Descriptor for “table-level” constraints. Presently only one property,
+ other is supported. This contains strings describing arbitrary
+ table-level constraints. A table-level constraint is a constraint defined
+ in terms of multiple columns (e.g. col_A > col_B) or in terms of rows.
+
+ Parameters: other (List[str]) – Descriptions of arbitrary table-level constraints.
+
+
+
+
+
+
class dagster.TableColumn
+
+
+ Descriptor for a table column. The only property that must be specified
+ by the user is name. If no type is specified, string is assumed. If
+ no constraints are specified, the column is assumed to be nullable
+ (i.e. required = False) and have no other constraints beyond the data type.
+
+ Parameters:
+ - name (List[str]) – Descriptions of arbitrary table-level constraints.
+ - type (Optional[str]) – The type of the column. Can be an arbitrary
+ - description (Optional[str]) – Description of this column. Defaults to None.
+ - constraints (Optional[[*TableColumnConstraints*](#dagster.TableColumnConstraints)]) – Column-level constraints.
+ - tags (Optional[Mapping[str, str]]) – Tags for filtering or organizing columns.
+
+
+
+
+
+
+
+
class dagster.TableColumnConstraints
+
+
+ Descriptor for a table column’s constraints. Nullability and uniqueness are specified with
+ boolean properties. All other constraints are described using arbitrary strings under the
+ other property.
+
+ Parameters:
+ - nullable (Optional[bool]) – If true, this column can hold null values.
+ - unique (Optional[bool]) – If true, all values in this column must be unique.
+ - other (List[str]) – Descriptions of arbitrary column-level constraints
+
+
+
+
+
+
+
+
class dagster.TableColumnLineage
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Represents the lineage of column outputs to column inputs for a tabular asset.
+
+ Parameters: deps_by_column (Mapping[str, Sequence[[*TableColumnDep*](#dagster.TableColumnDep)]]) – A mapping from column names to
+ the columns that the column depends on.
+ Examples:
+
+ Defining column lineage at materialization time, where the resulting asset has two columns,
+ `new_column_foo` and `new_column_qux`. The first column, `new_column_foo`, depends on
+ `column_bar` in `source_bar` and `column_baz` in `source_baz`. The second column,
+ `new_column_qux`, depends on `column_quuz` in `source_bar`.
+
+ ```python
+ from dagster import (
+ AssetKey,
+ MaterializeResult,
+ TableColumnDep,
+ TableColumnLineage,
+ asset,
+ )
+
+
+ @asset(deps=[AssetKey("source_bar"), AssetKey("source_baz")])
+ def my_asset():
+ yield MaterializeResult(
+ metadata={
+ "dagster/column_lineage": TableColumnLineage(
+ deps_by_column={
+ "new_column_foo": [
+ TableColumnDep(
+ asset_key=AssetKey("source_bar"),
+ column_name="column_bar",
+ ),
+ TableColumnDep(
+ asset_key=AssetKey("source_baz"),
+ column_name="column_baz",
+ ),
+ ],
+ "new_column_qux": [
+ TableColumnDep(
+ asset_key=AssetKey("source_bar"),
+ column_name="column_quuz",
+ ),
+ ],
+ }
+ )
+ }
+ )
+ ```
+
+
+
+
+
+
class dagster.TableColumnDep
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Object representing an identifier for a column in an asset.
+
+
+
+
+
+
+
+
+
+
+## Code references
+
+The following functions are used to attach source code references to your assets.
+For more information, refer to the [Linking to asset definition code with code references](https://docs.dagster.io/guides/dagster/code-references) guide.
+
+
+
dagster.with_source_code_references
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Wrapper function which attaches local code reference metadata to the provided asset definitions.
+ This points to the filepath and line number where the asset body is defined.
+
+ Parameters: assets_defs (Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset), CacheableAssetsDefinition]]) – The asset definitions to which source code metadata should be attached.Returns: The asset definitions with source code metadata attached.Return type: Sequence[[AssetsDefinition](assets.mdx#dagster.AssetsDefinition)]
+
+
+
+
+
+
dagster.link_code_references_to_git
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Wrapper function which converts local file path code references to source control URLs
+ based on the provided source control URL and branch.
+
+ Parameters:
+ - assets_defs (Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset), CacheableAssetsDefinition]]) – The asset definitions to which source control metadata should be attached.
+ - git_url (str) – The base URL for the source control system. For example,
+ - git_branch (str) – The branch in the source control system, such as “master”.
+ - file_path_mapping ([*FilePathMapping*](#dagster.FilePathMapping)) – Specifies the mapping between local file paths and their corresponding paths in a source control repository.
+
+
+ Example:
+
+ ```python
+ defs = Definitions(
+ assets=link_code_references_to_git(
+ with_source_code_references([my_dbt_assets]),
+ git_url="https://github.com/dagster-io/dagster",
+ git_branch="master",
+ file_path_mapping=AnchorBasedFilePathMapping(
+ local_file_anchor=Path(__file__),
+ file_anchor_path_in_repository="python_modules/my_module/my-module/__init__.py",
+ ),
+ )
+ )
+ ```
+
+
+
+
+
+
class dagster.FilePathMapping
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Base class which defines a file path mapping function. These functions are used to map local file paths
+ to their corresponding paths in a source control repository.
+
+ In many cases where a source control repository is reproduced exactly on a local machine, the included
+ AnchorBasedFilePathMapping class can be used to specify a direct mapping between the local file paths and the
+ repository paths. However, in cases where the repository structure differs from the local structure, a custom
+ mapping function can be provided to handle these cases.
+
+
+
abstract convert_to_source_control_path
+
+
+ Maps a local file path to the corresponding path in a source control repository.
+
+ Parameters: local_path (Path) – The local file path to map.Returns: The corresponding path in the hosted source control repository, relative to the repository root.Return type: str
+
+
+
+
+
+
+
+
+
+
class dagster.AnchorBasedFilePathMapping
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Specifies the mapping between local file paths and their corresponding paths in a source control repository,
+ using a specific file “anchor” as a reference point. All other paths are calculated relative to this anchor file.
+
+ For example, if the chosen anchor file is /Users/dagster/Documents/python_modules/my_module/my-module/__init__.py
+ locally, and python_modules/my_module/my-module/__init__.py in a source control repository, in order to map a
+ different file /Users/dagster/Documents/python_modules/my_module/my-module/my_asset.py to the repository path,
+ the mapping function will position the file in the repository relative to the anchor file’s position in the repository,
+ resulting in python_modules/my_module/my-module/my_asset.py.
+
+ Parameters:
+ - local_file_anchor (Path) – The path to a local file that is present in the repository.
+ - file_anchor_path_in_repository (str) – The path to the anchor file in the repository.
+
+
+ Example:
+
+ ```python
+ mapping_fn = AnchorBasedFilePathMapping(
+ local_file_anchor=Path(__file__),
+ file_anchor_path_in_repository="python_modules/my_module/my-module/__init__.py",
+ )
+ ```
+
+
convert_to_source_control_path
+
+
+ Maps a local file path to the corresponding path in a source control repository
+ based on the anchor file and its corresponding path in the repository.
+
+ Parameters: local_path (Path) – The local file path to map.Returns: The corresponding path in the hosted source control repository, relative to the repository root.Return type: str
+
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/ops.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/ops.mdx
new file mode 100644
index 0000000000000..9997cc38eb341
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/ops.mdx
@@ -0,0 +1,575 @@
+---
+title: 'ops'
+title_meta: 'ops API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'ops Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Ops
+
+The foundational unit of computation in Dagster.
+
+
+
+
+## Defining ops
+
+
+
@dagster.op
+
+
+ Create an op with the specified parameters from the decorated function.
+
+ Ins and outs will be inferred from the type signature of the decorated function
+ if not explicitly provided.
+
+ The decorated function will be used as the op’s compute function. The signature of the
+ decorated function is more flexible than that of the `compute_fn` in the core API; it may:
+
+ 1. Return a value. This value will be wrapped in an [`Output`](#dagster.Output) and yielded by the compute function.
+ 2. Return an [`Output`](#dagster.Output). This output will be yielded by the compute function.
+ 3. Yield [`Output`](#dagster.Output) or other [event objects](#events)`event objects`. Same as default compute behavior.
+ Note that options 1) and 2) are incompatible with yielding other events – if you would like
+ to decorate a function that yields events, it must also wrap its eventual output in an
+ [`Output`](#dagster.Output) and yield it.
+
+ @op supports `async def` functions as well, including async generators when yielding multiple
+ events or outputs. Note that async ops will generally be run on their own unless using a custom
+ [`Executor`](internals.mdx#dagster.Executor) implementation that supports running them together.
+
+ Parameters:
+ - name (Optional[str]) – Name of op. Must be unique within any [`GraphDefinition`](graphs.mdx#dagster.GraphDefinition)
+ - description (Optional[str]) – Human-readable description of this op. If not provided, and
+ - ins (Optional[Dict[str, [*In*](#dagster.In)]]) – Information about the inputs to the op. Information provided here will be combined
+ - out (Optional[Union[[*Out*](#dagster.Out), Dict[str, [*Out*](#dagster.Out)]]]) – Information about the op outputs. Information provided here will be combined with
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The schema for the config. If set, Dagster will check
+ - required_resource_keys (Optional[Set[str]]) – Set of resource handles required by this op.
+ - tags (Optional[Dict[str, Any]]) – Arbitrary metadata for the op. Frameworks may
+ - code_version (Optional[str]) – (Experimental) Version of the logic encapsulated by the op. If set,
+ - retry_policy (Optional[[*RetryPolicy*](#dagster.RetryPolicy)]) – The retry policy for this op.
+
+
+ Examples:
+
+ ```python
+ @op
+ def hello_world():
+ print('hello')
+
+ @op
+ def echo(msg: str) -> str:
+ return msg
+
+ @op(
+ ins={'msg': In(str)},
+ out=Out(str)
+ )
+ def echo_2(msg): # same as above
+ return msg
+
+ @op(
+ out={'word': Out(), 'num': Out()}
+ )
+ def multi_out() -> Tuple[str, int]:
+ return 'cool', 4
+ ```
+
+
+
+
+
+
class dagster.OpDefinition
+
+
+ Defines an op, the functional unit of user-defined computation.
+
+ End users should prefer the [`@op`](#dagster.op) decorator. OpDefinition is generally intended to be
+ used by framework authors or for programatically generated ops.
+
+ Parameters:
+ - name (str) – Name of the op. Must be unique within any [`GraphDefinition`](graphs.mdx#dagster.GraphDefinition) or
+ - input_defs (List[InputDefinition]) – Inputs of the op.
+ - compute_fn (Callable) –
+
+ The core of the op, the function that performs the actual
+ computation. The signature of this function is determined by `input_defs`, and
+ optionally, an injected first argument, `context`, a collection of information
+ provided by the system.
+
+ - output_defs (List[OutputDefinition]) – Outputs of the op.
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The schema for the config. If set, Dagster will check
+ - description (Optional[str]) – Human-readable description of the op.
+ - tags (Optional[Dict[str, Any]]) – Arbitrary metadata for the op. Frameworks may
+ - required_resource_keys (Optional[Set[str]]) – Set of resources handles required by this op.
+ - code_version (Optional[str]) – (Experimental) Version of the code encapsulated by the op. If set,
+ - retry_policy (Optional[[*RetryPolicy*](#dagster.RetryPolicy)]) – The retry policy for this op.
+ - pool (Optional[str]) – A string that identifies the pool that governs this op’s execution.
+
+
+ Examples:
+
+ ```python
+ def _add_one(_context, inputs):
+ yield Output(inputs["num"] + 1)
+
+ OpDefinition(
+ name="add_one",
+ ins={"num": In(int)},
+ outs={"result": Out(int)},
+ compute_fn=_add_one,
+ )
+ ```
+
+
alias
+
+ Creates a copy of this op with the given name.
+
+
+
+
+
tag
+
+ Creates a copy of this op with the given tags.
+
+
+
+
+
with_hooks
+
+ Creates a copy of this op with the given hook definitions.
+
+
+
+
+
with_retry_policy
+
+ Creates a copy of this op with the given retry policy.
+
+
+
+
+
property config_schema
+
+
+ The config schema for this op.
+
+ Type: IDefinitionConfigSchema
+
+
+
+
+
+
property ins
+
+
+ A mapping from input name to the In object that represents that input.
+
+ Type: Mapping[str, [In](#dagster.In)]
+
+
+
+
+
+
property name
+
+
+ The name of this op.
+
+ Type: str
+
+
+
+
+
+
property outs
+
+
+ A mapping from output name to the Out object that represents that output.
+
+ Type: Mapping[str, [Out](#dagster.Out)]
+
+
+
+
+
+
property required_resource_keys
+
+
+ A set of keys for resources that must be provided to this OpDefinition.
+
+ Type: AbstractSet[str]
+
+
+
+
+
+
property retry_policy
+
+
+ The RetryPolicy for this op.
+
+ Type: Optional[[RetryPolicy](#dagster.RetryPolicy)]
+
+
+
+
+
+
property tags
+
+
+ The tags for this op.
+
+ Type: Mapping[str, str]
+
+
+
+
+
+
property version
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `code_version` instead..
+
+ :::
+
+ Version of the code encapsulated by the op. If set, this is used as a
+ default code version for all outputs.
+
+ Type: str
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Ins & outs
+
+
+
class dagster.In
+
+
+ Defines an argument to an op’s compute function.
+
+ Inputs may flow from previous op’s outputs, or be stubbed using config. They may optionally
+ be typed using the Dagster type system.
+
+ Parameters:
+ - dagster_type (Optional[Union[Type, [*DagsterType*](types.mdx#dagster.DagsterType)]]]) – The type of this input. Should only be set if the correct type can not
+ - description (Optional[str]) – Human-readable description of the input.
+ - default_value (Optional[Any]) – The default value to use if no input is provided.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – A dict of metadata for the input.
+ - asset_key (Optional[Union[[*AssetKey*](assets.mdx#dagster.AssetKey), InputContext -> AssetKey]]) – (Experimental) An AssetKey
+ - asset_partitions (Optional[Union[Set[str], InputContext -> Set[str]]]) – (Experimental) A
+ - input_manager_key (Optional[str]) – (Experimental) The resource key for the
+
+
+
+
+
+
+
+
class dagster.Out
+
+
+ Defines an output from an op’s compute function.
+
+ Ops can have multiple outputs, in which case outputs cannot be anonymous.
+
+ Many ops have only one output, in which case the user can provide a single output definition
+ that will be given the default name, “result”.
+
+ Outs may be typed using the Dagster type system.
+
+ Parameters:
+ - dagster_type (Optional[Union[Type, [*DagsterType*](types.mdx#dagster.DagsterType)]]]) – The type of this output. Should only be set if the correct type can not
+ - description (Optional[str]) – Human-readable description of the output.
+ - is_required (bool) – Whether the presence of this field is required. (default: True)
+ - io_manager_key (Optional[str]) – The resource key of the output manager used for this output.
+ - metadata (Optional[Dict[str, Any]]) – A dict of the metadata for the output.
+ - code_version (Optional[str]) – (Experimental) Version of the code that generates this output. In
+
+
+
+
+
+
+
+
+
+
+
+
+## Execution
+
+
+
class dagster.RetryPolicy
+
+
+ A declarative policy for when to request retries when an exception occurs during op execution.
+
+ Parameters:
+ - max_retries (int) – The maximum number of retries to attempt. Defaults to 1.
+ - delay (Optional[Union[int,float]]) – The time in seconds to wait between the retry being requested and the next attempt
+ - backoff (Optional[[*Backoff*](#dagster.Backoff)]) – A modifier for delay as a function of retry attempt number.
+ - jitter (Optional[[*Jitter*](#dagster.Jitter)]) – A randomizing modifier for delay, applied after backoff calculation.
+
+
+
+
+
+
+
+
class dagster.Backoff
+
+
+ A modifier for delay as a function of attempt number.
+
+ LINEAR: attempt_num * delay
+ EXPONENTIAL: ((2 ^ attempt_num) - 1) * delay
+
+
+
+
+
+
+
class dagster.Jitter
+
+
+ A randomizing modifier for delay, applied after backoff calculation.
+
+ FULL: between 0 and the calculated delay based on backoff: random() * backoff_delay
+ PLUS_MINUS: +/- the delay: backoff_delay + ((2 * (random() * delay)) - delay)
+
+
+
+
+
+
+
+
+
+
+
+
+## Events
+
+The objects that can be yielded by the body of ops’ compute functions to communicate with the
+Dagster framework.
+
+(Note that [`Failure`](#dagster.Failure) and [`RetryRequested`](#dagster.RetryRequested) are intended to be raised from ops rather than yielded.)
+
+
+
+### Event types
+
+
+
class dagster.Output
+
+
+ Event corresponding to one of an op’s outputs.
+
+ Op compute functions must explicitly yield events of this type when they have more than
+ one output, or when they also yield events of other types, or when defining a op using the
+ [`OpDefinition`](#dagster.OpDefinition) API directly.
+
+ Outputs are values produced by ops that will be consumed by downstream ops in a job.
+ They are type-checked at op boundaries when their corresponding [`Out`](#dagster.Out)
+ or the downstream [`In`](#dagster.In) is typed.
+
+ Parameters:
+ - value (Any) – The value returned by the compute function.
+ - output_name (str) – Name of the corresponding Out. (default: “result”)
+ - metadata (Optional[Dict[str, Union[str, float, int, [*MetadataValue*](metadata.mdx#dagster.MetadataValue)]]]) – Arbitrary metadata about the output. Keys are displayed string labels, and values are
+ - data_version (Optional[DataVersion]) – experimental
+ - tags (Optional[Mapping[str, str]]) – (Experimental) Tags that will be attached to the asset
+
+
+
+
property data_version
+
+
+ A data version that was manually set on the Output.
+
+ Type: Optional[DataVersion]
+
+
+
+
+
+
property output_name
+
+
+ Name of the corresponding [`Out`](#dagster.Out).
+
+ Type: str
+
+
+
+
+
+
property value
+
+
+ The value returned by the compute function.
+
+ Type: Any
+
+
+
+
+
+
+
+
+
+
class dagster.AssetMaterialization
+
+
+ Event indicating that an op has materialized an asset.
+
+ Op compute functions may yield events of this type whenever they wish to indicate to the
+ Dagster framework (and the end user) that they have produced a materialized value as a
+ side effect of computation. Unlike outputs, asset materializations can not be passed to other
+ ops, and their persistence is controlled by op logic, rather than by the Dagster
+ framework.
+
+ Op authors should use these events to organize metadata about the side effects of their
+ computations, enabling tooling like the Assets dashboard in the Dagster UI.
+
+ Parameters:
+ - asset_key (Union[str, List[str], [*AssetKey*](assets.mdx#dagster.AssetKey)]) – A key to identify the materialized asset across
+ - description (Optional[str]) – A longer human-readable description of the materialized value.
+ - partition (Optional[str]) – The name of the partition
+ - tags (Optional[Mapping[str, str]]) – A mapping containing tags for the materialization.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the asset. Keys are displayed string labels, and values are
+
+
+
+
static file
+
+
+ Static constructor for standard materializations corresponding to files on disk.
+
+ Parameters:
+ - path (str) – The path to the file.
+ - description (Optional[str]) – A human-readable description of the materialization.
+
+
+
+
+
+
+
+
+
+
+
+
class dagster.ExpectationResult
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ If using assets, use AssetCheckResult and @asset_check instead..
+
+ :::
+
+ Event corresponding to a data quality test.
+
+ Op compute functions may yield events of this type whenever they wish to indicate to the
+ Dagster framework (and the end user) that a data quality test has produced a (positive or
+ negative) result.
+
+ Parameters:
+ - success (bool) – Whether the expectation passed or not.
+ - label (Optional[str]) – Short display name for expectation. Defaults to “result”.
+ - description (Optional[str]) – A longer human-readable description of the expectation.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
+
+
+
+
+
+
+
+
class dagster.TypeCheck
+
+
+ Event corresponding to a successful typecheck.
+
+ Events of this type should be returned by user-defined type checks when they need to encapsulate
+ additional metadata about a type check’s success or failure. (i.e., when using
+ `as_dagster_type()`, `@usable_as_dagster_type`, or the underlying
+ [`PythonObjectDagsterType()`](types.mdx#dagster.PythonObjectDagsterType) API.)
+
+ Op compute functions should generally avoid yielding events of this type to avoid confusion.
+
+ Parameters:
+ - success (bool) – `True` if the type check succeeded, `False` otherwise.
+ - description (Optional[str]) – A human-readable description of the type check.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
+
+
+
+
+
+
+
+
class dagster.Failure
+
+
+ Event indicating op failure.
+
+ Raise events of this type from within op compute functions or custom type checks in order to
+ indicate an unrecoverable failure in user code to the Dagster machinery and return
+ structured metadata about the failure.
+
+ Parameters:
+ - description (Optional[str]) – A human-readable description of the failure.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
+ - allow_retries (Optional[bool]) – Whether this Failure should respect the retry policy or bypass it and immediately fail.
+
+
+
+
+
+
+
+
class dagster.RetryRequested
+
+
+ An exception to raise from an op to indicate that it should be retried.
+
+ Parameters:
+ - max_retries (Optional[int]) – The max number of retries this step should attempt before failing
+ - seconds_to_wait (Optional[Union[float,int]]) – Seconds to wait before restarting the step after putting the step in
+
+
+ Example:
+
+ ```python
+ @op
+ def flakes():
+ try:
+ flakey_operation()
+ except Exception as e:
+ raise RetryRequested(max_retries=3) from e
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/partitions.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/partitions.mdx
new file mode 100644
index 0000000000000..f502f38b85b56
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/partitions.mdx
@@ -0,0 +1,1659 @@
+---
+title: 'partitions definitions'
+title_meta: 'partitions definitions API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'partitions definitions Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Partitions Definitions
+
+
+
class dagster.PartitionsDefinition
+
+
+ Defines a set of partitions, which can be attached to a software-defined asset or job.
+
+ Abstract class with implementations for different kinds of partitions.
+
+
+
abstract get_partition_keys
+
+
+ Returns a list of strings representing the partition keys of the PartitionsDefinition.
+
+ Parameters:
+ - current_time (Optional[datetime]) – A datetime object representing the current time, only
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: Sequence[str]
+
+
+
+
+
+
+
+
+
+
class dagster.HourlyPartitionsDefinition
+
+
+ A set of hourly partitions.
+
+ The first partition in the set will start on the start_date at midnight. The last partition
+ in the set will end before the current time, unless the end_offset argument is set to a
+ positive number. If minute_offset is provided, the start and end times of each partition
+ will be minute_offset past the hour.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions. Can
+ - end_date (Union[datetime.datetime, str, None]) – The last date(excluding) in the set of partitions.
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d. Note that if a non-UTC
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+
+
+ ```python
+ HourlyPartitionsDefinition(start_date=datetime(2022, 03, 12))
+ # creates partitions (2022-03-12-00:00, 2022-03-12-01:00), (2022-03-12-01:00, 2022-03-12-02:00), ...
+
+ HourlyPartitionsDefinition(start_date=datetime(2022, 03, 12), minute_offset=15)
+ # creates partitions (2022-03-12-00:15, 2022-03-12-01:15), (2022-03-12-01:15, 2022-03-12-02:15), ...
+ ```
+
+
get_cron_schedule
+
+
+ The schedule executes at the cadence specified by the partitioning, but may overwrite
+ the minute/hour/day offset of the partitioning.
+
+ This is useful e.g. if you have partitions that span midnight to midnight but you want to
+ schedule a job that runs at 2 am.
+
+
+
+
+
+
+
property day_offset
+
+
+ For a weekly or monthly partitions definition, returns the day to “split” partitions
+ by. Each partition will start on this day, and end before this day in the following
+ week/month. Returns 0 if the day_offset parameter is unset in the
+ WeeklyPartitionsDefinition, MonthlyPartitionsDefinition, or the provided cron schedule.
+
+ For weekly partitions, returns a value between 0 (representing Sunday) and 6 (representing
+ Saturday). Providing a value of 1 means that a partition will exist weekly from Monday to
+ the following Sunday.
+
+ For monthly partitions, returns a value between 0 (the first day of the month) and 31 (the
+ last possible day of the month).
+
+ Type: int
+
+
+
+
+
+
property end
+
+
+
+
+
+
+
property hour_offset
+
+
+ Number of hours past 00:00 to “split” partitions. Defaults to 0.
+
+ For example, returns 1 if each partition starts at 01:00.
+
+ Type: int
+
+
+
+
+
+
property minute_offset
+
+
+ Number of minutes past the hour to “split” partitions. Defaults to 0.
+
+ For example, returns 15 if each partition starts at 15 minutes past the hour.
+
+ Type: int
+
+
+
+
+
+
property schedule_type
+
+
+ An enum representing the partition cadence (hourly, daily,
+ weekly, or monthly).
+
+ Type: Optional[ScheduleType]
+
+
+
+
+
+
property start
+
+
+
+
+
+
+
+
+
+
+
class dagster.DailyPartitionsDefinition
+
+
+ A set of daily partitions.
+
+ The first partition in the set will start at the start_date at midnight. The last partition
+ in the set will end before the current time, unless the end_offset argument is set to a
+ positive number. If minute_offset and/or hour_offset are used, the start and end times of
+ each partition will be hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions. Can
+ - end_date (Union[datetime.datetime, str, None]) – The last date(excluding) in the set of partitions.
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+
+
+ ```python
+ DailyPartitionsDefinition(start_date="2022-03-12")
+ # creates partitions (2022-03-12-00:00, 2022-03-13-00:00), (2022-03-13-00:00, 2022-03-14-00:00), ...
+
+ DailyPartitionsDefinition(start_date="2022-03-12", minute_offset=15, hour_offset=16)
+ # creates partitions (2022-03-12-16:15, 2022-03-13-16:15), (2022-03-13-16:15, 2022-03-14-16:15), ...
+ ```
+
+
get_cron_schedule
+
+
+ The schedule executes at the cadence specified by the partitioning, but may overwrite
+ the minute/hour/day offset of the partitioning.
+
+ This is useful e.g. if you have partitions that span midnight to midnight but you want to
+ schedule a job that runs at 2 am.
+
+
+
+
+
+
+
property day_offset
+
+
+ For a weekly or monthly partitions definition, returns the day to “split” partitions
+ by. Each partition will start on this day, and end before this day in the following
+ week/month. Returns 0 if the day_offset parameter is unset in the
+ WeeklyPartitionsDefinition, MonthlyPartitionsDefinition, or the provided cron schedule.
+
+ For weekly partitions, returns a value between 0 (representing Sunday) and 6 (representing
+ Saturday). Providing a value of 1 means that a partition will exist weekly from Monday to
+ the following Sunday.
+
+ For monthly partitions, returns a value between 0 (the first day of the month) and 31 (the
+ last possible day of the month).
+
+ Type: int
+
+
+
+
+
+
property end
+
+
+
+
+
+
+
property hour_offset
+
+
+ Number of hours past 00:00 to “split” partitions. Defaults to 0.
+
+ For example, returns 1 if each partition starts at 01:00.
+
+ Type: int
+
+
+
+
+
+
property minute_offset
+
+
+ Number of minutes past the hour to “split” partitions. Defaults to 0.
+
+ For example, returns 15 if each partition starts at 15 minutes past the hour.
+
+ Type: int
+
+
+
+
+
+
property schedule_type
+
+
+ An enum representing the partition cadence (hourly, daily,
+ weekly, or monthly).
+
+ Type: Optional[ScheduleType]
+
+
+
+
+
+
property start
+
+
+
+
+
+
+
+
+
+
+
class dagster.WeeklyPartitionsDefinition
+
+
+ Defines a set of weekly partitions.
+
+ The first partition in the set will start at the start_date. The last partition in the set will
+ end before the current time, unless the end_offset argument is set to a positive number. If
+ day_offset is provided, the start and end date of each partition will be day of the week
+ corresponding to day_offset (0 indexed with Sunday as the start of the week). If
+ minute_offset and/or hour_offset are used, the start and end times of each partition will be
+ hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions will
+ - end_date (Union[datetime.datetime, str, None]) – The last date(excluding) in the set of partitions.
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - day_offset (int) – Day of the week to “split” the partition. Defaults to 0 (Sunday).
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+
+
+ ```python
+ WeeklyPartitionsDefinition(start_date="2022-03-12")
+ # creates partitions (2022-03-13-00:00, 2022-03-20-00:00), (2022-03-20-00:00, 2022-03-27-00:00), ...
+
+ WeeklyPartitionsDefinition(start_date="2022-03-12", minute_offset=15, hour_offset=3, day_offset=6)
+ # creates partitions (2022-03-12-03:15, 2022-03-19-03:15), (2022-03-19-03:15, 2022-03-26-03:15), ...
+ ```
+
+
get_cron_schedule
+
+
+ The schedule executes at the cadence specified by the partitioning, but may overwrite
+ the minute/hour/day offset of the partitioning.
+
+ This is useful e.g. if you have partitions that span midnight to midnight but you want to
+ schedule a job that runs at 2 am.
+
+
+
+
+
+
+
property day_offset
+
+
+ For a weekly or monthly partitions definition, returns the day to “split” partitions
+ by. Each partition will start on this day, and end before this day in the following
+ week/month. Returns 0 if the day_offset parameter is unset in the
+ WeeklyPartitionsDefinition, MonthlyPartitionsDefinition, or the provided cron schedule.
+
+ For weekly partitions, returns a value between 0 (representing Sunday) and 6 (representing
+ Saturday). Providing a value of 1 means that a partition will exist weekly from Monday to
+ the following Sunday.
+
+ For monthly partitions, returns a value between 0 (the first day of the month) and 31 (the
+ last possible day of the month).
+
+ Type: int
+
+
+
+
+
+
property end
+
+
+
+
+
+
+
property hour_offset
+
+
+ Number of hours past 00:00 to “split” partitions. Defaults to 0.
+
+ For example, returns 1 if each partition starts at 01:00.
+
+ Type: int
+
+
+
+
+
+
property minute_offset
+
+
+ Number of minutes past the hour to “split” partitions. Defaults to 0.
+
+ For example, returns 15 if each partition starts at 15 minutes past the hour.
+
+ Type: int
+
+
+
+
+
+
property schedule_type
+
+
+ An enum representing the partition cadence (hourly, daily,
+ weekly, or monthly).
+
+ Type: Optional[ScheduleType]
+
+
+
+
+
+
property start
+
+
+
+
+
+
+
+
+
+
+
class dagster.MonthlyPartitionsDefinition
+
+
+ A set of monthly partitions.
+
+ The first partition in the set will start at the soonest first of the month after start_date
+ at midnight. The last partition in the set will end before the current time, unless the
+ end_offset argument is set to a positive number. If day_offset is provided, the start and
+ end date of each partition will be day_offset. If minute_offset and/or hour_offset are used,
+ the start and end times of each partition will be hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions will be
+ - end_date (Union[datetime.datetime, str, None]) – The last date(excluding) in the set of partitions.
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - day_offset (int) – Day of the month to “split” the partition. Defaults to 1.
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+
+
+ ```python
+ MonthlyPartitionsDefinition(start_date="2022-03-12")
+ # creates partitions (2022-04-01-00:00, 2022-05-01-00:00), (2022-05-01-00:00, 2022-06-01-00:00), ...
+
+ MonthlyPartitionsDefinition(start_date="2022-03-12", minute_offset=15, hour_offset=3, day_offset=5)
+ # creates partitions (2022-04-05-03:15, 2022-05-05-03:15), (2022-05-05-03:15, 2022-06-05-03:15), ...
+ ```
+
+
get_cron_schedule
+
+
+ The schedule executes at the cadence specified by the partitioning, but may overwrite
+ the minute/hour/day offset of the partitioning.
+
+ This is useful e.g. if you have partitions that span midnight to midnight but you want to
+ schedule a job that runs at 2 am.
+
+
+
+
+
+
+
property day_offset
+
+
+ For a weekly or monthly partitions definition, returns the day to “split” partitions
+ by. Each partition will start on this day, and end before this day in the following
+ week/month. Returns 0 if the day_offset parameter is unset in the
+ WeeklyPartitionsDefinition, MonthlyPartitionsDefinition, or the provided cron schedule.
+
+ For weekly partitions, returns a value between 0 (representing Sunday) and 6 (representing
+ Saturday). Providing a value of 1 means that a partition will exist weekly from Monday to
+ the following Sunday.
+
+ For monthly partitions, returns a value between 0 (the first day of the month) and 31 (the
+ last possible day of the month).
+
+ Type: int
+
+
+
+
+
+
property end
+
+
+
+
+
+
+
property hour_offset
+
+
+ Number of hours past 00:00 to “split” partitions. Defaults to 0.
+
+ For example, returns 1 if each partition starts at 01:00.
+
+ Type: int
+
+
+
+
+
+
property minute_offset
+
+
+ Number of minutes past the hour to “split” partitions. Defaults to 0.
+
+ For example, returns 15 if each partition starts at 15 minutes past the hour.
+
+ Type: int
+
+
+
+
+
+
property schedule_type
+
+
+ An enum representing the partition cadence (hourly, daily,
+ weekly, or monthly).
+
+ Type: Optional[ScheduleType]
+
+
+
+
+
+
property start
+
+
+
+
+
+
+
+
+
+
+
class dagster.TimeWindowPartitionsDefinition
+
+
+ A set of partitions where each partition corresponds to a time window.
+
+ The provided cron_schedule determines the bounds of the time windows. E.g. a cron_schedule of
+ “0 0 \* \* \*” will result in daily partitions that start at midnight and end at midnight of the
+ following day.
+
+ The string partition_key associated with each partition corresponds to the start of the
+ partition’s time window.
+
+ The first partition in the set will start on at the first cron_schedule tick that is equal to
+ or after the given start datetime. The last partition in the set will end before the current
+ time, unless the end_offset argument is set to a positive number.
+
+ We recommended limiting partition counts for each asset to 25,000 partitions or fewer.
+
+ Parameters:
+ - cron_schedule (str) – Determines the bounds of the time windows.
+ - start (datetime) – The first partition in the set will start on at the first cron_schedule
+ - timezone (Optional[str]) – The timezone in which each time should exist.
+ - end (datetime) – The last partition (excluding) in the set.
+ - fmt (str) – The date format to use for partition_keys. Note that if a non-UTC timezone is
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+
+
+
+
get_cron_schedule
+
+
+ The schedule executes at the cadence specified by the partitioning, but may overwrite
+ the minute/hour/day offset of the partitioning.
+
+ This is useful e.g. if you have partitions that span midnight to midnight but you want to
+ schedule a job that runs at 2 am.
+
+
+
+
+
+
+
property day_offset
+
+
+ For a weekly or monthly partitions definition, returns the day to “split” partitions
+ by. Each partition will start on this day, and end before this day in the following
+ week/month. Returns 0 if the day_offset parameter is unset in the
+ WeeklyPartitionsDefinition, MonthlyPartitionsDefinition, or the provided cron schedule.
+
+ For weekly partitions, returns a value between 0 (representing Sunday) and 6 (representing
+ Saturday). Providing a value of 1 means that a partition will exist weekly from Monday to
+ the following Sunday.
+
+ For monthly partitions, returns a value between 0 (the first day of the month) and 31 (the
+ last possible day of the month).
+
+ Type: int
+
+
+
+
+
+
property end
+
+
+
+
+
+
+
property hour_offset
+
+
+ Number of hours past 00:00 to “split” partitions. Defaults to 0.
+
+ For example, returns 1 if each partition starts at 01:00.
+
+ Type: int
+
+
+
+
+
+
property minute_offset
+
+
+ Number of minutes past the hour to “split” partitions. Defaults to 0.
+
+ For example, returns 15 if each partition starts at 15 minutes past the hour.
+
+ Type: int
+
+
+
+
+
+
property schedule_type
+
+
+ An enum representing the partition cadence (hourly, daily,
+ weekly, or monthly).
+
+ Type: Optional[ScheduleType]
+
+
+
+
+
+
property start
+
+
+
+
+
+
+
+
+
+
+
class dagster.TimeWindow
+
+
+ An interval that is closed at the start and open at the end.
+
+
+
start
+
+
+ A datetime that marks the start of the window.
+
+ Type: datetime
+
+
+
+
+
+
end
+
+
+ A datetime that marks the end of the window.
+
+ Type: datetime
+
+
+
+
+
+
+
+
+
+
class dagster.StaticPartitionsDefinition
+
+
+ A statically-defined set of partitions.
+
+ We recommended limiting partition counts for each asset to 25,000 partitions or fewer.
+
+ Example:
+
+ ```python
+ from dagster import StaticPartitionsDefinition, asset
+
+ oceans_partitions_def = StaticPartitionsDefinition(
+ ["arctic", "atlantic", "indian", "pacific", "southern"]
+ )
+
+ @asset(partitions_def=oceans_partitions_defs)
+ def ml_model_for_each_ocean():
+ ...
+ ```
+
+
get_partition_keys
+
+
+ Returns a list of strings representing the partition keys of the PartitionsDefinition.
+
+ Parameters:
+ - current_time (Optional[datetime]) – A datetime object representing the current time, only
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: Sequence[str]
+
+
+
+
+
+
+
+
+
+
class dagster.MultiPartitionsDefinition
+
+
+ Takes the cross-product of partitions from two partitions definitions.
+
+ For example, with a static partitions definition where the partitions are [“a”, “b”, “c”]
+ and a daily partitions definition, this partitions definition will have the following
+ partitions:
+
+ 2020-01-01|a
+ 2020-01-01|b
+ 2020-01-01|c
+ 2020-01-02|a
+ 2020-01-02|b
+ …
+
+ We recommended limiting partition counts for each asset to 25,000 partitions or fewer.
+
+ Parameters: partitions_defs (Mapping[str, [*PartitionsDefinition*](#dagster.PartitionsDefinition)]) – A mapping of dimension name to partitions definition. The total set of partitions will
+ be the cross-product of the partitions from each PartitionsDefinition.
+
+
partitions_defs
+
+
+ A sequence of PartitionDimensionDefinition objects, each of which contains a dimension
+ name and a PartitionsDefinition. The total set of partitions will be the cross-product
+ of the partitions from each PartitionsDefinition. This sequence is ordered by
+ dimension name, to ensure consistent ordering of the partitions.
+
+ Type: Sequence[PartitionDimensionDefinition]
+
+
+
+
+
+
get_partition_keys
+
+
+ Returns a list of MultiPartitionKeys representing the partition keys of the
+ PartitionsDefinition.
+
+ Parameters:
+ - current_time (Optional[datetime]) – A datetime object representing the current time, only
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: Sequence[MultiPartitionKey]
+
+
+
+
+
+
+
+
+
+
class dagster.MultiPartitionKey
+
+
+ A multi-dimensional partition key stores the partition key for each dimension.
+ Subclasses the string class to keep partition key type as a string.
+
+ Contains additional methods to access the partition key for each dimension.
+ Creates a string representation of the partition key for each dimension, separated by a pipe (|).
+ Orders the dimensions by name, to ensure consistent string representation.
+
+
+
+
+
+
+
class dagster.DynamicPartitionsDefinition
+
+
+ A partitions definition whose partition keys can be dynamically added and removed.
+
+ This is useful for cases where the set of partitions is not known at definition time,
+ but is instead determined at runtime.
+
+ Partitions can be added and removed using instance.add_dynamic_partitions and
+ instance.delete_dynamic_partition methods.
+
+ We recommended limiting partition counts for each asset to 25,000 partitions or fewer.
+
+ Parameters:
+ - name (Optional[str]) – The name of the partitions definition.
+ - partition_fn (Optional[Callable[[Optional[datetime]], Union[Sequence[Partition], Sequence[str]]]]) – deprecated
+
+
+ Examples:
+
+ ```python
+ fruits = DynamicPartitionsDefinition(name="fruits")
+
+ @sensor(job=my_job)
+ def my_sensor(context):
+ return SensorResult(
+ run_requests=[RunRequest(partition_key="apple")],
+ dynamic_partitions_requests=[fruits.build_add_request(["apple"])]
+ )
+ ```
+
+
get_partition_keys
+
+
+ Returns a list of strings representing the partition keys of the
+ PartitionsDefinition.
+
+ Parameters:
+ - current_time (Optional[datetime]) – A datetime object representing the current time, only
+ - dynamic_partitions_store (Optional[DynamicPartitionsStore]) – The DynamicPartitionsStore
+
+
+ Returns: Sequence[str]
+
+
+
+
+
+
+
+
+
+
class dagster.PartitionKeyRange
+
+
+ Defines a range of partitions.
+
+
+
start
+
+
+ The starting partition key in the range (inclusive).
+
+ Type: str
+
+
+
+
+
+
end
+
+
+ The ending partition key in the range (inclusive).
+
+ Type: str
+
+
+
+ Creates a schedule from a job that targets
+ time window-partitioned or statically-partitioned assets. The job can also be
+ multi-partitioned, as long as one of the partition dimensions is time-partitioned.
+
+ The schedule executes at the cadence specified by the time partitioning of the job or assets.
+
+ Example:
+ ```python
+ ######################################
+ # Job that targets partitioned assets
+ ######################################
+
+ from dagster import (
+ DailyPartitionsDefinition,
+ asset,
+ build_schedule_from_partitioned_job,
+ define_asset_job,
+ Definitions,
+ )
+
+ @asset(partitions_def=DailyPartitionsDefinition(start_date="2020-01-01"))
+ def asset1():
+ ...
+
+ asset1_job = define_asset_job("asset1_job", selection=[asset1])
+
+ # The created schedule will fire daily
+ asset1_job_schedule = build_schedule_from_partitioned_job(asset1_job)
+
+ defs = Definitions(assets=[asset1], schedules=[asset1_job_schedule])
+
+ ################
+ # Non-asset job
+ ################
+
+ from dagster import DailyPartitionsDefinition, build_schedule_from_partitioned_job, jog
+
+
+ @job(partitions_def=DailyPartitionsDefinition(start_date="2020-01-01"))
+ def do_stuff_partitioned():
+ ...
+
+ # The created schedule will fire daily
+ do_stuff_partitioned_schedule = build_schedule_from_partitioned_job(
+ do_stuff_partitioned,
+ )
+
+ defs = Definitions(schedules=[do_stuff_partitioned_schedule])
+ ```
+
+
+
+
+
+
+
+
+
+
+# Partition Mapping
+
+
+
class dagster.PartitionMapping
+
+
+ Defines a correspondence between the partitions in an asset and the partitions in an asset
+ that it depends on.
+
+ Overriding PartitionMapping outside of Dagster is not supported. The abstract methods of this
+ class may change at any time.
+
+
+
abstract get_downstream_partitions_for_partitions
+
+
+ Returns the subset of partition keys in the downstream asset that use the data in the given
+ partition key subset of the upstream asset.
+
+ Parameters:
+ - upstream_partitions_subset (Union[[*PartitionKeyRange*](#dagster.PartitionKeyRange), PartitionsSubset]) – The
+ - downstream_partitions_def ([*PartitionsDefinition*](#dagster.PartitionsDefinition)) – The partitions definition for the
+
+
+
+
+
+ Returns a UpstreamPartitionsResult object containing the partition keys the downstream
+ partitions subset was mapped to in the upstream partitions definition.
+
+ Valid upstream partitions will be included in UpstreamPartitionsResult.partitions_subset.
+ Invalid upstream partitions will be included in UpstreamPartitionsResult.required_but_nonexistent_subset.
+
+ For example, if an upstream asset is time-partitioned and starts in June 2023, and the
+ downstream asset is time-partitioned and starts in May 2023, this function would return a
+ UpstreamPartitionsResult(PartitionsSubset(“2023-06-01”), required_but_nonexistent_subset=PartitionsSubset(“2023-05-01”))
+ when downstream_partitions_subset contains 2023-05-01 and 2023-06-01.
+
+
+
+
+
+
+
+
+
+
+
class dagster.TimeWindowPartitionMapping
+
+
+ The default mapping between two TimeWindowPartitionsDefinitions.
+
+ A partition in the downstream partitions definition is mapped to all partitions in the upstream
+ asset whose time windows overlap it.
+
+ This means that, if the upstream and downstream partitions definitions share the same time
+ period, then this mapping is essentially the identity partition mapping - plus conversion of
+ datetime formats.
+
+ If the upstream time period is coarser than the downstream time period, then each partition in
+ the downstream asset will map to a single (larger) upstream partition. E.g. if the downstream is
+ hourly and the upstream is daily, then each hourly partition in the downstream will map to the
+ daily partition in the upstream that contains that hour.
+
+ If the upstream time period is finer than the downstream time period, then each partition in the
+ downstream asset will map to multiple upstream partitions. E.g. if the downstream is daily and
+ the upstream is hourly, then each daily partition in the downstream asset will map to the 24
+ hourly partitions in the upstream that occur on that day.
+
+
+
start_offset
+
+
+ If not 0, then the starts of the upstream windows are shifted by this
+ offset relative to the starts of the downstream windows. For example, if start_offset=-1
+ and end_offset=0, then the downstream partition “2022-07-04” would map to the upstream
+ partitions “2022-07-03” and “2022-07-04”. If the upstream and downstream
+ PartitionsDefinitions are different, then the offset is in the units of the downstream.
+ Defaults to 0.
+
+ Type: int
+
+
+
+
+
+
end_offset
+
+
+ If not 0, then the ends of the upstream windows are shifted by this
+ offset relative to the ends of the downstream windows. For example, if start_offset=0
+ and end_offset=1, then the downstream partition “2022-07-04” would map to the upstream
+ partitions “2022-07-04” and “2022-07-05”. If the upstream and downstream
+ PartitionsDefinitions are different, then the offset is in the units of the downstream.
+ Defaults to 0.
+
+ Type: int
+
+
+
+
+
+
allow_nonexistent_upstream_partitions
+
+
+ Defaults to false. If true, does not
+ raise an error when mapped upstream partitions fall outside the start-end time window of the
+ partitions def. For example, if the upstream partitions def starts on “2023-01-01” but
+ the downstream starts on “2022-01-01”, setting this bool to true would return no
+ partition keys when get_upstream_partitions_for_partitions is called with “2022-06-01”.
+ When set to false, would raise an error.
+
+ Type: bool
+
+
+ Expects that the upstream and downstream assets are partitioned in the same way, and maps
+ partitions in the downstream asset to the same partition in the upstream asset.
+
+
+
+
+
class dagster.AllPartitionMapping
+
+
+ Maps every partition in the downstream asset to every partition in the upstream asset.
+
+ Commonly used in the case when the downstream asset is not partitioned, in which the entire
+ downstream asset depends on all partitions of the upstream asset.
+
+
+
+
+
+
+
class dagster.LastPartitionMapping
+
+
+ Maps all dependencies to the last partition in the upstream asset.
+
+ Commonly used in the case when the downstream asset is not partitioned, in which the entire
+ downstream asset depends on the last partition of the upstream asset.
+
+
+
+
+
+
+
class dagster.StaticPartitionMapping
+
+
+ Define an explicit correspondence between two StaticPartitionsDefinitions.
+
+ Parameters: downstream_partition_keys_by_upstream_partition_key (Dict[str, str | Collection[str]]) – The single or multi-valued correspondence from upstream keys to downstream keys.
+
+
+
+
+
+
class dagster.SpecificPartitionsPartitionMapping
+
+
+ Maps to a specific subset of partitions in the upstream asset.
+
+ Example:
+
+ ```python
+ from dagster import SpecificPartitionsPartitionMapping, StaticPartitionsDefinition, asset
+
+ @asset(partitions_def=StaticPartitionsDefinition(["a", "b", "c"]))
+ def upstream():
+ ...
+
+ @asset(
+ ins={
+ "upstream": AssetIn(partition_mapping=SpecificPartitionsPartitionMapping(["a"]))
+ }
+ )
+ def a_downstream(upstream):
+ ...
+ ```
+
+
+
+
+
+
class dagster.MultiToSingleDimensionPartitionMapping
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Defines a correspondence between an single-dimensional partitions definition
+ and a MultiPartitionsDefinition. The single-dimensional partitions definition must be
+ a dimension of the MultiPartitionsDefinition.
+
+ This class handles the case where the upstream asset is multipartitioned and the
+ downstream asset is single dimensional, and vice versa.
+
+ For a partition key X, this partition mapping assumes that any multi-partition key with
+ X in the selected dimension is a dependency.
+
+ Parameters: partition_dimension_name (Optional[str]) – The name of the partition dimension in the
+ MultiPartitionsDefinition that matches the single-dimension partitions definition.
+
+
+
+
+
+
class dagster.MultiPartitionMapping
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ Defines a correspondence between two MultiPartitionsDefinitions.
+
+ Accepts a mapping of upstream dimension name to downstream DimensionPartitionMapping, representing
+ the explicit correspondence between the upstream and downstream MultiPartitions dimensions
+ and the partition mapping used to calculate the downstream partitions.
+
+ Examples:
+
+ ```python
+ weekly_abc = MultiPartitionsDefinition(
+ {
+ "abc": StaticPartitionsDefinition(["a", "b", "c"]),
+ "weekly": WeeklyPartitionsDefinition("2023-01-01"),
+ }
+ )
+ daily_123 = MultiPartitionsDefinition(
+ {
+ "123": StaticPartitionsDefinition(["1", "2", "3"]),
+ "daily": DailyPartitionsDefinition("2023-01-01"),
+ }
+ )
+
+ MultiPartitionMapping(
+ {
+ "abc": DimensionPartitionMapping(
+ dimension_name="123",
+ partition_mapping=StaticPartitionMapping({"a": "1", "b": "2", "c": "3"}),
+ ),
+ "weekly": DimensionPartitionMapping(
+ dimension_name="daily",
+ partition_mapping=TimeWindowPartitionMapping(),
+ )
+ }
+ )
+ ```
+ For upstream or downstream dimensions not explicitly defined in the mapping, Dagster will
+ assume an AllPartitionsMapping, meaning that all upstream partitions in those dimensions
+ will be mapped to all downstream partitions in those dimensions.
+
+ Examples:
+
+ ```python
+ weekly_abc = MultiPartitionsDefinition(
+ {
+ "abc": StaticPartitionsDefinition(["a", "b", "c"]),
+ "daily": DailyPartitionsDefinition("2023-01-01"),
+ }
+ )
+ daily_123 = MultiPartitionsDefinition(
+ {
+ "123": StaticPartitionsDefinition(["1", "2", "3"]),
+ "daily": DailyPartitionsDefinition("2023-01-01"),
+ }
+ )
+
+ MultiPartitionMapping(
+ {
+ "daily": DimensionPartitionMapping(
+ dimension_name="daily",
+ partition_mapping=IdentityPartitionMapping(),
+ )
+ }
+ )
+
+ # Will map `daily_123` partition key {"123": "1", "daily": "2023-01-01"} to the upstream:
+ # {"abc": "a", "daily": "2023-01-01"}
+ # {"abc": "b", "daily": "2023-01-01"}
+ # {"abc": "c", "daily": "2023-01-01"}
+ ```
+ Parameters: downstream_mappings_by_upstream_dimension (Mapping[str, DimensionPartitionMapping]) – A
+ mapping that defines an explicit correspondence between one dimension of the upstream
+ MultiPartitionsDefinition and one dimension of the downstream MultiPartitionsDefinition.
+ Maps a string representing upstream dimension name to downstream DimensionPartitionMapping,
+ containing the downstream dimension name and partition mapping.
+
+
+
+
+
+
+
+
+
+
+# Backfill Policy (Experimental)
+
+
+
class dagster.BackfillPolicy
+
+
+ :::warning[experimental]
+ This API may break in future versions, even between dot releases.
+
+
+ :::
+
+ A BackfillPolicy specifies how Dagster should attempt to backfill a partitioned asset.
+
+ There are two main kinds of backfill policies: single-run and multi-run.
+
+ An asset with a single-run backfill policy will take a single run to backfill all of its
+ partitions at once.
+
+ An asset with a multi-run backfill policy will take multiple runs to backfill all of its
+ partitions. Each run will backfill a subset of the partitions. The number of partitions to
+ backfill in each run is controlled by the max_partitions_per_run parameter.
+
+ For example:
+
+ - If an asset has 100 partitions, and the max_partitions_per_run is set to 10, then it will
+ - If an asset has 100 partitions, and the max_partitions_per_run is set to 11, then it will
+
+
+ Warning:
+
+ Constructing an BackfillPolicy directly is not recommended as the API is subject to change.
+ BackfillPolicy.single_run() and BackfillPolicy.multi_run(max_partitions_per_run=x) are the
+ recommended APIs.
+
+
+
static multi_run
+
+
+ Creates a BackfillPolicy that executes the entire backfill in multiple runs.
+ Each run will backfill [max_partitions_per_run] number of partitions.
+
+ Parameters: max_partitions_per_run (Optional[int]) – The maximum number of partitions in each run of
+ the multiple runs. Defaults to 1.
+
+
+
+
+
+
static single_run
+
+ Creates a BackfillPolicy that executes the entire backfill in a single run.
+
+
+
+
+
+
+
+
+
+
+
+
+# Partitioned Config
+
+
+
class dagster.PartitionedConfig
+
+
+ Defines a way of configuring a job where the job can be run on one of a discrete set of
+ partitions, and each partition corresponds to run configuration for the job.
+
+ Setting PartitionedConfig as the config for a job allows you to launch backfills for that job
+ and view the run history across partitions.
+
+
+
get_partition_keys
+
+
+ Returns a list of partition keys, representing the full set of partitions that
+ config can be applied to.
+
+ Parameters: current_time (Optional[datetime]) – A datetime object representing the current time. Only
+ applicable to time-based partitions definitions.Returns: Sequence[str]
+
+
+
+
+
+
property partitions_def
+
+
+ The partitions definition associated with this PartitionedConfig.
+
+ Type: T_PartitionsDefinition
+
+
+
+
+
+
property run_config_for_partition_fn
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `run_config_for_partition_key_fn` instead..
+
+ :::
+
+ A function that accepts a partition
+ and returns a dictionary representing the config to attach to runs for that partition.
+ Deprecated as of 1.3.3.
+
+ Type: Optional[Callable[[Partition], Mapping[str, Any]]]
+
+
+
+
+
+
property run_config_for_partition_key_fn
+
+
+ A function that accepts a partition key
+ and returns a dictionary representing the config to attach to runs for that partition.
+
+ Type: Optional[Callable[[str], Union[[RunConfig](config.mdx#dagster.RunConfig), Mapping[str, Any]]]]
+
+
+
+
+
+
property tags_for_partition_fn
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Use `tags_for_partition_key_fn` instead..
+
+ :::
+
+ A function that
+ accepts a partition and returns a dictionary of tags to attach to runs for
+ that partition. Deprecated as of 1.3.3.
+
+ Type: Optional[Callable[[Partition], Mapping[str, str]]]
+
+
+
+
+
+
property tags_for_partition_key_fn
+
+
+ A function that
+ accepts a partition key and returns a dictionary of tags to attach to runs for
+ that partition.
+
+ Type: Optional[Callable[[str], Mapping[str, str]]]
+
+
+
+
+
+
+
+
+
+
dagster.static_partitioned_config
+
+
+ Creates a static partitioned config for a job.
+
+ The provided partition_keys is a static list of strings identifying the set of partitions. The
+ list of partitions is static, so while the run config returned by the decorated function may
+ change over time, the list of valid partition keys does not.
+
+ This has performance advantages over dynamic_partitioned_config in terms of loading different
+ partition views in the Dagster UI.
+
+ The decorated function takes in a partition key and returns a valid run config for a particular
+ target job.
+
+ Parameters:
+ - partition_keys (Sequence[str]) – A list of valid partition keys, which serve as the range of
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – deprecated
+ - tags_for_partition_key_fn (Optional[Callable[[str], Mapping[str, str]]]) – A function that
+
+
+ Returns: PartitionedConfig
+
+
+
+
+
+
dagster.dynamic_partitioned_config
+
+
+ Creates a dynamic partitioned config for a job.
+
+ The provided partition_fn returns a list of strings identifying the set of partitions, given
+ an optional datetime argument (representing the current time). The list of partitions returned
+ may change over time.
+
+ The decorated function takes in a partition key and returns a valid run config for a particular
+ target job.
+
+ Parameters:
+ - partition_fn (Callable[[datetime.datetime], Sequence[str]]) – A function that generates a
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – deprecated
+
+
+ Returns: PartitionedConfig
+
+
+
+
+
+
dagster.hourly_partitioned_config
+
+
+ Defines run config over a set of hourly partitions.
+
+ The decorated function should accept a start datetime and end datetime, which represent the date
+ partition the config should delineate.
+
+ The decorated function should return a run config dictionary.
+
+ The resulting object created by this decorator can be provided to the config argument of a Job.
+ The first partition in the set will start at the start_date at midnight. The last partition in
+ the set will end before the current time, unless the end_offset argument is set to a positive
+ number. If minute_offset is provided, the start and end times of each partition will be
+ minute_offset past the hour.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions. Can
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – A function that
+
+
+ ```python
+ @hourly_partitioned_config(start_date=datetime(2022, 03, 12))
+ # creates partitions (2022-03-12-00:00, 2022-03-12-01:00), (2022-03-12-01:00, 2022-03-12-02:00), ...
+
+ @hourly_partitioned_config(start_date=datetime(2022, 03, 12), minute_offset=15)
+ # creates partitions (2022-03-12-00:15, 2022-03-12-01:15), (2022-03-12-01:15, 2022-03-12-02:15), ...
+ ```
+
+
+
+
+
+
dagster.daily_partitioned_config
+
+
+ Defines run config over a set of daily partitions.
+
+ The decorated function should accept a start datetime and end datetime, which represent the bounds
+ of the date partition the config should delineate.
+
+ The decorated function should return a run config dictionary.
+
+ The resulting object created by this decorator can be provided to the config argument of a Job.
+ The first partition in the set will start at the start_date at midnight. The last partition in
+ the set will end before the current time, unless the end_offset argument is set to a positive
+ number. If minute_offset and/or hour_offset are used, the start and end times of each partition
+ will be hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions. Can
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – A function that
+
+
+ ```python
+ @daily_partitioned_config(start_date="2022-03-12")
+ # creates partitions (2022-03-12-00:00, 2022-03-13-00:00), (2022-03-13-00:00, 2022-03-14-00:00), ...
+
+ @daily_partitioned_config(start_date="2022-03-12", minute_offset=15, hour_offset=16)
+ # creates partitions (2022-03-12-16:15, 2022-03-13-16:15), (2022-03-13-16:15, 2022-03-14-16:15), ...
+ ```
+
+
+
+
+
+
dagster.weekly_partitioned_config
+
+
+ Defines run config over a set of weekly partitions.
+
+ The decorated function should accept a start datetime and end datetime, which represent the date
+ partition the config should delineate.
+
+ The decorated function should return a run config dictionary.
+
+ The resulting object created by this decorator can be provided to the config argument of a Job.
+ The first partition in the set will start at the start_date. The last partition in the set will
+ end before the current time, unless the end_offset argument is set to a positive number. If
+ day_offset is provided, the start and end date of each partition will be day of the week
+ corresponding to day_offset (0 indexed with Sunday as the start of the week). If
+ minute_offset and/or hour_offset are used, the start and end times of each partition will be
+ hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions will
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - day_offset (int) – Day of the week to “split” the partition. Defaults to 0 (Sunday).
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – A function that
+
+
+ ```python
+ @weekly_partitioned_config(start_date="2022-03-12")
+ # creates partitions (2022-03-13-00:00, 2022-03-20-00:00), (2022-03-20-00:00, 2022-03-27-00:00), ...
+
+ @weekly_partitioned_config(start_date="2022-03-12", minute_offset=15, hour_offset=3, day_offset=6)
+ # creates partitions (2022-03-12-03:15, 2022-03-19-03:15), (2022-03-19-03:15, 2022-03-26-03:15), ...
+ ```
+
+
+
+
+
+
dagster.monthly_partitioned_config
+
+
+ Defines run config over a set of monthly partitions.
+
+ The decorated function should accept a start datetime and end datetime, which represent the date
+ partition the config should delineate.
+
+ The decorated function should return a run config dictionary.
+
+ The resulting object created by this decorator can be provided to the config argument of a Job.
+ The first partition in the set will start at midnight on the soonest first of the month after
+ start_date. The last partition in the set will end before the current time, unless the
+ end_offset argument is set to a positive number. If day_offset is provided, the start and end
+ date of each partition will be day_offset. If minute_offset and/or hour_offset are used, the
+ start and end times of each partition will be hour_offset:minute_offset of each day.
+
+ Parameters:
+ - start_date (Union[datetime.datetime, str]) – The first date in the set of partitions will be
+ - minute_offset (int) – Number of minutes past the hour to “split” the partition. Defaults
+ - hour_offset (int) – Number of hours past 00:00 to “split” the partition. Defaults to 0.
+ - day_offset (int) – Day of the month to “split” the partition. Defaults to 1.
+ - timezone (Optional[str]) – The timezone in which each date should exist.
+ - fmt (Optional[str]) – The date format to use. Defaults to %Y-%m-%d.
+ - end_offset (int) – Extends the partition set by a number of partitions equal to the value
+ - tags_for_partition_fn (Optional[Callable[[str], Mapping[str, str]]]) – A function that
+
+
+ ```python
+ @monthly_partitioned_config(start_date="2022-03-12")
+ # creates partitions (2022-04-01-00:00, 2022-05-01-00:00), (2022-05-01-00:00, 2022-06-01-00:00), ...
+
+ @monthly_partitioned_config(start_date="2022-03-12", minute_offset=15, hour_offset=3, day_offset=5)
+ # creates partitions (2022-04-05-03:15, 2022-05-05-03:15), (2022-05-05-03:15, 2022-06-05-03:15), ...
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/pipes.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/pipes.mdx
new file mode 100644
index 0000000000000..60f8019af296c
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/pipes.mdx
@@ -0,0 +1,416 @@
+---
+title: 'dagster pipes'
+title_meta: 'dagster pipes API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'dagster pipes Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Dagster Pipes
+
+[Dagster Pipes](https://docs.dagster.io/guides/build/external-pipelines/) is a toolkit for building integrations between Dagster and external execution environments. This reference outlines the APIs included with the `dagster` library, which should be used in the orchestration environment.
+
+For a detailed look at the Pipes process, including how to customize it, refer to the [Dagster Pipes details and customization guide](https://docs.dagster.io/guides/build/external-pipelines/dagster-pipes-details-and-customization).
+
+Looking to write code in an external process? Refer to the API reference for the separately-installed [dagster-pipes](https://docs.dagster.io/api/python-api/libraries/dagster-pipes) library.
+
+
+
+
+## Sessions
+
+
+
class dagster.PipesSession
+
+
+ Object representing a pipes session.
+
+ A pipes session is defined by a pair of [`PipesContextInjector`](#dagster.PipesContextInjector) and
+ [`PipesMessageReader`](#dagster.PipesMessageReader) objects. At the opening of the session, the context injector
+ writes context data to an externally accessible location, and the message reader starts
+ monitoring an externally accessible location. These locations are encoded in parameters stored
+ on a PipesSession object.
+
+ During the session, an external process should be started and the parameters injected into its
+ environment. The typical way to do this is to call [`PipesSession.get_bootstrap_env_vars()`](#dagster.PipesSession.get_bootstrap_env_vars)
+ and pass the result as environment variables.
+
+ During execution, results (e.g. asset materializations) are reported by the external process and
+ buffered on the PipesSession object. The buffer can periodically be cleared and yielded to
+ Dagster machinery by calling yield from PipesSession.get_results().
+
+ When the external process exits, the session can be closed. Closing consists of handling any
+ unprocessed messages written by the external process and cleaning up any resources used for
+ context injection and message reading.
+
+ Parameters:
+ - context_data (PipesContextData) – The context for the executing op/asset.
+ - message_handler ([*PipesMessageHandler*](#dagster.PipesMessageHandler)) – The message handler to use for processing messages
+ - context_injector_params (PipesParams) – Parameters yielded by the context injector,
+ - message_reader_params (PipesParams) – Parameters yielded by the message reader, indicating
+ - created_at (datetime) – The time at which the session was created. Useful as cutoff for
+
+
+
+
get_bootstrap_cli_arguments
+
+
+ Encode context injector and message reader params as CLI arguments.
+
+ Passing CLI arguments is an alternative way to expose the pipes I/O parameters to a pipes process.
+ Using environment variables should be preferred when possible.
+
+ Returns: CLI arguments pass to the external process. The values are
+ serialized as json, compressed with zlib, and then base64-encoded.Return type: Mapping[str, str]
+
+
+
+
+
+
get_bootstrap_env_vars
+
+
+ Encode context injector and message reader params as environment variables.
+
+ Passing environment variables is the typical way to expose the pipes I/O parameters
+ to a pipes process.
+
+ Returns: Environment variables to pass to the external process. The values are
+ serialized as json, compressed with gzip, and then base-64-encoded.Return type: Mapping[str, str]
+
+
+
+
+
+
get_bootstrap_params
+
+
+ Get the params necessary to bootstrap a launched pipes process. These parameters are typically
+ are as environment variable. See get_bootstrap_env_vars. It is the context injector’s
+ responsibility to decide how to pass these parameters to the external environment.
+
+ Returns: Parameters to pass to the external process and their corresponding
+ values that must be passed by the context injector.Return type: Mapping[str, str]
+
+
+
+
+
+
get_custom_messages
+
+
+ Get the sequence of deserialized JSON data that was reported from the external process using
+ report_custom_message.
+
+ Returns: Sequence[Any]
+
+
+
+
+
+
+
get_reported_results
+
+
+ `PipesExecutionResult` objects only explicitly received from the external process.
+
+ Returns: Result reported by external process.Return type: Sequence[PipesExecutionResult]
+
+
+
+
+
+
get_results
+
+
+ `PipesExecutionResult` objects reported from the external process,
+ potentially modified by Pipes.
+
+ Parameters:
+ - implicit_materializations (bool) – Create MaterializeResults for expected assets
+ - metadata (Optional[Mapping[str, [*MetadataValue*](metadata.mdx#dagster.MetadataValue)]]) – Arbitrary metadata that will be attached to all
+
+
+ Returns: Result reported by external process.Return type: Sequence[PipesExecutionResult]
+
+
+
+
+
+
+
+
+
+
dagster.open_pipes_session
+
+
+ Context manager that opens and closes a pipes session.
+
+ This context manager should be used to wrap the launch of an external process using the pipe
+ protocol to report results back to Dagster. The yielded [`PipesSession`](#dagster.PipesSession) should be used
+ to (a) obtain the environment variables that need to be provided to the external process; (b)
+ access results streamed back from the external process.
+
+ This method is an alternative to [`PipesClient`](#dagster.PipesClient) subclasses for users who want more
+ control over how pipes processes are launched. When using open_pipes_session, it is the user’s
+ responsibility to inject the message reader and context injector parameters available on the
+ yielded PipesSession and pass them to the appropriate API when launching the external process.
+ Typically these parameters should be set as environment variables.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](execution.mdx#dagster.AssetExecutionContext)]) – The context for the current op/asset execution.
+ - context_injector ([*PipesContextInjector*](#dagster.PipesContextInjector)) – The context injector to use to inject context into the external process.
+ - message_reader ([*PipesMessageReader*](#dagster.PipesMessageReader)) – The message reader to use to read messages from the external process.
+ - extras (Optional[PipesExtras]) – Optional extras to pass to the external process via the injected context.
+
+
+ Yields: PipesSession – Interface for interacting with the external process.
+ ```python
+ import subprocess
+ from dagster import open_pipes_session
+
+ extras = {"foo": "bar"}
+
+ @asset
+ def ext_asset(context: AssetExecutionContext):
+ with open_pipes_session(
+ context=context,
+ extras={"foo": "bar"},
+ context_injector=PipesTempFileContextInjector(),
+ message_reader=PipesTempFileMessageReader(),
+ ) as pipes_session:
+ subprocess.Popen(
+ ["/bin/python", "/path/to/script.py"],
+ env={**pipes_session.get_bootstrap_env_vars()}
+ )
+ while process.poll() is None:
+ yield from pipes_session.get_results()
+
+ yield from pipes_session.get_results()
+ ```
+
+
+
+
+
+
+
+
+
+
+## Clients
+
+
+
class dagster.PipesClient
+
+
+ Pipes client base class.
+
+ Pipes clients for specific external environments should subclass this.
+
+
+
abstract run
+
+
+ Synchronously execute an external process with the pipes protocol. Derived
+ clients must have context and extras arguments, but also can add arbitrary
+ arguments that are appropriate for their own implementation.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](execution.mdx#dagster.AssetExecutionContext)]) – The context from the executing op/asset.
+ - extras (Optional[PipesExtras]) – Arbitrary data to pass to the external environment.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
class dagster.PipesSubprocessClient
+
+
+ A pipes client that runs a subprocess with the given command and environment.
+
+ By default parameters are injected via environment variables. Context is passed via
+ a temp file, and structured messages are read from from a temp file.
+
+ Parameters:
+ - env (Optional[Mapping[str, str]]) – An optional dict of environment variables to pass to the
+ - cwd (Optional[str]) – Working directory in which to launch the subprocess command.
+ - context_injector (Optional[[*PipesContextInjector*](#dagster.PipesContextInjector)]) – A context injector to use to inject
+ - message_reader (Optional[[*PipesMessageReader*](#dagster.PipesMessageReader)]) – A message reader to use to read messages from
+ - forward_termination (bool) – Whether to send a SIGINT signal to the subprocess
+ - forward_stdio (bool) – Whether to forward stdout and stderr from the subprocess to the
+ - termination_timeout_seconds (float) – How long to wait after forwarding termination
+
+
+
+
run
+
+
+ Synchronously execute a subprocess with in a pipes session.
+
+ Parameters:
+ - command (Union[str, Sequence[str]]) – The command to run. Will be passed to subprocess.Popen().
+ - context (Union[[*OpExecutionContext*](execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](execution.mdx#dagster.AssetExecutionContext)]) – The context from the executing op or asset.
+ - extras (Optional[PipesExtras]) – An optional dict of extra parameters to pass to the subprocess.
+ - env (Optional[Mapping[str, str]]) – An optional dict of environment variables to pass to the subprocess.
+ - cwd (Optional[str]) – Working directory in which to launch the subprocess command.
+
+
+ Returns: Wrapper containing results reported by the external
+ process.Return type: PipesClientCompletedInvocation
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Advanced
+
+Most Pipes users won’t need to use the APIs in the following sections unless they are customizing the Pipes protocol.
+
+Refer to the [Dagster Pipes details and customization guide](https://docs.dagster.io/guides/build/external-pipelines/dagster-pipes-details-and-customization) for more information.
+
+
+
+
+### Context injectors
+
+Context injectors write context payloads to an externally accessible location and yield a set of parameters encoding the location for inclusion in the bootstrap payload.
+
+
+
class dagster.PipesContextInjector
+
+
+
+
+
+
+
class dagster.PipesEnvContextInjector
+
+ Context injector that injects context data into the external process by injecting it directly into the external process environment.
+
+
+
+
+
class dagster.PipesFileContextInjector
+
+
+ Context injector that injects context data into the external process by writing it to a
+ specified file.
+
+ Parameters: path (str) – The path of a file to which to write context data. The file will be deleted on
+ close of the pipes session.
+
+
+
+
+
+
class dagster.PipesTempFileContextInjector
+
+ Context injector that injects context data into the external process by writing it to an
+ automatically-generated temporary file.
+
+
+
+
+
+
+
+
+### Message readers
+
+Message readers read messages (and optionally log files) from an externally accessible location and yield a set of parameters encoding the location in the bootstrap payload.
+
+
+
class dagster.PipesMessageReader
+
+
+
+
+
+
+
class dagster.PipesBlobStoreMessageReader
+
+
+ Message reader that reads a sequence of message chunks written by an external process into a
+ blob store such as S3, Azure blob storage, or GCS.
+
+ The reader maintains a counter, starting at 1, that is synchronized with a message writer in
+ some pipes process. The reader starts a thread that periodically attempts to read a chunk
+ indexed by the counter at some location expected to be written by the pipes process. The chunk
+ should be a file with each line corresponding to a JSON-encoded pipes message. When a chunk is
+ successfully read, the messages are processed and the counter is incremented. The
+ `PipesBlobStoreMessageWriter` on the other end is expected to similarly increment a
+ counter (starting from 1) on successful write, keeping counters on the read and write end in
+ sync.
+
+ If log_readers is passed, the message reader will start the passed log readers when the
+ opened message is received from the external process.
+
+ Parameters:
+ - interval (float) – interval in seconds between attempts to download a chunk
+ - log_readers (Optional[Sequence[PipesLogReader]]) – A set of log readers to use to read logs.
+
+
+
+
+
+
+
+
class dagster.PipesFileMessageReader
+
+
+ Message reader that reads messages by tailing a specified file.
+
+ Parameters:
+ - path (str) – The path of the file to which messages will be written. The file will be deleted
+ - include_stdio_in_messages (bool) – Whether to include stdout/stderr logs in the messages produced by the message writer in the external process.
+ - cleanup_file (bool) – Whether to delete the file on close of the pipes session.
+
+
+
+
+
+
+
+
class dagster.PipesTempFileMessageReader
+
+ Message reader that reads messages by tailing an automatically-generated temporary file.
+
+
+
+
+
class dagster.PipesMessageHandler
+
+
+ Class to process `PipesMessage` objects received from a pipes process.
+
+ Parameters:
+ - context (Union[[*OpExecutionContext*](execution.mdx#dagster.OpExecutionContext), [*AssetExecutionContext*](execution.mdx#dagster.AssetExecutionContext)]) – The context for the executing op/asset.
+ - message_reader ([*PipesMessageReader*](#dagster.PipesMessageReader)) – The message reader used to read messages from the
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/repositories.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/repositories.mdx
new file mode 100644
index 0000000000000..b72ac8c88e3f1
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/repositories.mdx
@@ -0,0 +1,552 @@
+---
+title: 'repositories'
+title_meta: 'repositories API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'repositories Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Repositories
+
+
+
dagster.repository RepositoryDefinition
+
+
+ Create a repository from the decorated function.
+
+ In most cases, [`Definitions`](definitions.mdx#dagster.Definitions) should be used instead.
+
+ The decorated function should take no arguments and its return value should one of:
+
+ 1. `List[Union[JobDefinition, ScheduleDefinition, SensorDefinition]]`.
+ Use this form when you have no need to lazy load jobs or other definitions. This is the
+ typical use case.
+
+ 2. A dict of the form:
+ ```python
+ {
+ 'jobs': Dict[str, Callable[[], JobDefinition]],
+ 'schedules': Dict[str, Callable[[], ScheduleDefinition]]
+ 'sensors': Dict[str, Callable[[], SensorDefinition]]
+ }
+ ```
+ This form is intended to allow definitions to be created lazily when accessed by name,
+ which can be helpful for performance when there are many definitions in a repository, or
+ when constructing the definitions is costly.
+
+ 3. A [`RepositoryData`](#dagster.RepositoryData). Return this object if you need fine-grained
+ control over the construction and indexing of definitions within the repository, e.g., to
+ create definitions dynamically from .yaml files in a directory.
+
+ Parameters:
+ - name (Optional[str]) – The name of the repository. Defaults to the name of the decorated
+ - description (Optional[str]) – A string description of the repository.
+ - metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata for the repository. Not
+ - top_level_resources (Optional[Mapping[str, [*ResourceDefinition*](resources.mdx#dagster.ResourceDefinition)]]) – A dict of top-level
+
+
+ Example:
+
+ ```python
+ ######################################################################
+ # A simple repository using the first form of the decorated function
+ ######################################################################
+
+ @op(config_schema={n: Field(Int)})
+ def return_n(context):
+ return context.op_config['n']
+
+ @job
+ def simple_job():
+ return_n()
+
+ @job
+ def some_job():
+ ...
+
+ @sensor(job=some_job)
+ def some_sensor():
+ if foo():
+ yield RunRequest(
+ run_key= ...,
+ run_config={
+ 'ops': {'return_n': {'config': {'n': bar()}}}
+ }
+ )
+
+ @job
+ def my_job():
+ ...
+
+ my_schedule = ScheduleDefinition(cron_schedule="0 0 * * *", job=my_job)
+
+ @repository
+ def simple_repository():
+ return [simple_job, some_sensor, my_schedule]
+
+ ######################################################################
+ # A simple repository using the first form of the decorated function
+ # and custom metadata that will be displayed in the UI
+ ######################################################################
+
+ ...
+
+ @repository(
+ name='my_repo',
+ metadata={
+ 'team': 'Team A',
+ 'repository_version': '1.2.3',
+ 'environment': 'production',
+ })
+ def simple_repository():
+ return [simple_job, some_sensor, my_schedule]
+
+ ######################################################################
+ # A lazy-loaded repository
+ ######################################################################
+
+ def make_expensive_job():
+ @job
+ def expensive_job():
+ for i in range(10000):
+ return_n.alias(f'return_n_{i}')()
+
+ return expensive_job
+
+ def make_expensive_schedule():
+ @job
+ def other_expensive_job():
+ for i in range(11000):
+ return_n.alias(f'my_return_n_{i}')()
+
+ return ScheduleDefinition(cron_schedule="0 0 * * *", job=other_expensive_job)
+
+ @repository
+ def lazy_loaded_repository():
+ return {
+ 'jobs': {'expensive_job': make_expensive_job},
+ 'schedules': {'expensive_schedule': make_expensive_schedule}
+ }
+
+
+ ######################################################################
+ # A complex repository that lazily constructs jobs from a directory
+ # of files in a bespoke YAML format
+ ######################################################################
+
+ class ComplexRepositoryData(RepositoryData):
+ def __init__(self, yaml_directory):
+ self._yaml_directory = yaml_directory
+
+ def get_all_jobs(self):
+ return [
+ self._construct_job_def_from_yaml_file(
+ self._yaml_file_for_job_name(file_name)
+ )
+ for file_name in os.listdir(self._yaml_directory)
+ ]
+
+ ...
+
+ @repository
+ def complex_repository():
+ return ComplexRepositoryData('some_directory')
+ ```
+
+
+
+
+
+
class dagster.RepositoryDefinition
+
+
+ Define a repository that contains a group of definitions.
+
+ Users should typically not create objects of this class directly. Instead, use the
+ `@repository()` decorator.
+
+ Parameters:
+ - name (str) – The name of the repository.
+ - repository_data ([*RepositoryData*](#dagster.RepositoryData)) – Contains the definitions making up the repository.
+ - description (Optional[str]) – A string description of the repository.
+ - metadata (Optional[MetadataMapping]) – Arbitrary metadata for the repository. Not
+
+
+
+
get_all_jobs
+
+
+ Return all jobs in the repository as a list.
+
+ Note that this will construct any job in the lazily evaluated dictionary that has
+ not yet been constructed.
+
+ Returns: All jobs in the repository.Return type: List[[JobDefinition](jobs.mdx#dagster.JobDefinition)]
+
+
+
+
+
+
get_asset_value_loader
+
+
+ Returns an object that can load the contents of assets as Python objects.
+
+ Invokes load_input on the [`IOManager`](io-managers.mdx#dagster.IOManager) associated with the assets. Avoids
+ spinning up resources separately for each asset.
+
+ Usage:
+
+ ```python
+ with my_repo.get_asset_value_loader() as loader:
+ asset1 = loader.load_asset_value("asset1")
+ asset2 = loader.load_asset_value("asset2")
+ ```
+
+
+
+
+
+
get_job
+
+
+ Get a job by name.
+
+ If this job is present in the lazily evaluated dictionary passed to the
+ constructor, but has not yet been constructed, only this job is constructed, and
+ will be cached for future calls.
+
+ Parameters: name (str) – Name of the job to retrieve.Returns: The job definition corresponding to
+ the given name.Return type: [JobDefinition](jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
get_schedule_def
+
+
+ Get a schedule definition by name.
+
+ Parameters: name (str) – The name of the schedule.Returns: The schedule definition.Return type: [ScheduleDefinition](schedules-sensors.mdx#dagster.ScheduleDefinition)
+
+
+
+
+
+
get_sensor_def
+
+
+ Get a sensor definition by name.
+
+ Parameters: name (str) – The name of the sensor.Returns: The sensor definition.Return type: [SensorDefinition](schedules-sensors.mdx#dagster.SensorDefinition)
+
+
+
+
+
+
has_job
+
+
+ Check if a job with a given name is present in the repository.
+
+ Parameters: name (str) – The name of the job.Returns: bool
+
+
+
+
+
+
has_schedule_def
+
+ bool: Check if a schedule with a given name is present in the repository.
+
+
+
+
+
has_sensor_def
+
+ bool: Check if a sensor with a given name is present in the repository.
+
+
+
+
+
load_asset_value
+
+
+ Load the contents of an asset as a Python object.
+
+ Invokes load_input on the [`IOManager`](io-managers.mdx#dagster.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()`](#dagster.RepositoryDefinition.get_asset_value_loader), which avoids spinning up
+ resources separately for each asset.
+
+ Parameters:
+ - asset_key (Union[[*AssetKey*](assets.mdx#dagster.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`](io-managers.mdx#dagster.IOManager)
+ - resource_config (Optional[Any]) – A dictionary of resource configurations to be passed
+
+
+ Returns: The contents of an asset as a Python object.
+
+
+
+
+
+
property asset_checks_defs_by_key
+
+
+ The assets checks defined in the repository.
+
+ Type: Mapping[[AssetCheckKey](asset-checks.mdx#dagster.AssetCheckKey), [AssetChecksDefinition](asset-checks.mdx#dagster.AssetChecksDefinition)]
+
+
+
+
+
+
property assets_defs_by_key
+
+
+ The assets definitions defined in the repository.
+
+ Type: Mapping[[AssetKey](assets.mdx#dagster.AssetKey), [AssetsDefinition](assets.mdx#dagster.AssetsDefinition)]
+
+
+
+
+
+
property description
+
+
+ A human-readable description of the repository.
+
+ Type: Optional[str]
+
+
+
+
+
+
property job_names
+
+
+ Names of all jobs in the repository.
+
+ Type: List[str]
+
+
+
+
+
+
property metadata
+
+
+ Arbitrary metadata for the repository.
+
+ Type: Optional[MetadataMapping]
+
+
+
+
+
+
property name
+
+
+ The name of the repository.
+
+ Type: str
+
+
+
+
+
+
property schedule_defs
+
+
+ All schedules in the repository.
+
+ Type: List[[ScheduleDefinition](schedules-sensors.mdx#dagster.ScheduleDefinition)]
+
+
+
+
+
+
property sensor_defs
+
+
+ All sensors in the repository.
+
+ Type: Sequence[[SensorDefinition](schedules-sensors.mdx#dagster.SensorDefinition)]
+
+
+
+
+
+
property source_assets_by_key
+
+
+ The source assets defined in the repository.
+
+ Type: Mapping[[AssetKey](assets.mdx#dagster.AssetKey), [SourceAsset](assets.mdx#dagster.SourceAsset)]
+
+
+
+
+
+
+
+
+
+
class dagster.RepositoryData
+
+
+ Users should usually rely on the [`@repository`](#dagster.repository) decorator to create new
+ repositories, which will in turn call the static constructors on this class. However, users may
+ subclass [`RepositoryData`](#dagster.RepositoryData) for fine-grained control over access to and lazy creation
+ of repository members.
+
+
+
abstract get_all_jobs
+
+
+ Return all jobs in the repository as a list.
+
+ Returns: All jobs in the repository.Return type: List[[JobDefinition](jobs.mdx#dagster.JobDefinition)]
+
+
+
+
+
+
get_all_schedules
+
+
+ Return all schedules in the repository as a list.
+
+ Returns: All jobs in the repository.Return type: List[[ScheduleDefinition](schedules-sensors.mdx#dagster.ScheduleDefinition)]
+
+
+
+
+
+
get_all_sensors
+
+ Sequence[SensorDefinition]: Return all sensors in the repository as a list.
+
+
+
+
+
get_asset_checks_defs_by_key
+
+ Mapping[AssetCheckKey, AssetChecksDefinition]: Get the asset checks definitions for the repository.
+
+
+
+
+
get_assets_defs_by_key
+
+ Mapping[AssetKey, AssetsDefinition]: Get the asset definitions for the repository.
+
+
+
+
+
get_job
+
+
+ Get a job by name.
+
+ Parameters: job_name (str) – Name of the job to retrieve.Returns: The job definition corresponding to the given name.Return type: [JobDefinition](jobs.mdx#dagster.JobDefinition)
+
+
+
+
+
+
get_job_names
+
+
+ Get the names of all jobs in the repository.
+
+ Returns: List[str]
+
+
+
+
+
+
get_schedule
+
+
+ Get a schedule by name.
+
+ Parameters: schedule_name (str) – name of the schedule to retrieve.Returns: The schedule definition corresponding to the given name.Return type: [ScheduleDefinition](schedules-sensors.mdx#dagster.ScheduleDefinition)
+
+
+
+
+
+
get_schedule_names
+
+
+ Get the names of all schedules in the repository.
+
+ Returns: List[str]
+
+
+
+
+
+
get_sensor
+
+
+ Get a sensor by name.
+
+ Parameters: sensor_name (str) – name of the sensor to retrieve.Returns: The sensor definition corresponding to the given name.Return type: [SensorDefinition](schedules-sensors.mdx#dagster.SensorDefinition)
+
+
+
+
+
+
get_sensor_names
+
+ Sequence[str]: Get the names of all sensors in the repository.
+
+
+
+
+
get_source_assets_by_key
+
+ Mapping[AssetKey, SourceAsset]: Get the source assets for the repository.
+
+
+
+
+
has_job
+
+
+ Check if a job with a given name is present in the repository.
+
+ Parameters: job_name (str) – The name of the job.Returns: bool
+
+
+
+
+
+
has_schedule
+
+ Check if a schedule with a given name is present in the repository.
+
+
+
+
+
has_sensor
+
+ Check if a sensor with a given name is present in the repository.
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/resources.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/resources.mdx
new file mode 100644
index 0000000000000..c86d98a9b55d9
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/resources.mdx
@@ -0,0 +1,474 @@
+---
+title: 'resources'
+title_meta: 'resources API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'resources Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Resources
+
+
+## Pythonic resource system
+
+The following classes are used as part of the new [Pythonic resources system](https://docs.dagster.io/guides/build/external-resources/).
+
+
+
class dagster.ConfigurableResource
+
+
+ Base class for Dagster resources that utilize structured config.
+
+ This class is a subclass of both [`ResourceDefinition`](#dagster.ResourceDefinition) and [`Config`](config.mdx#dagster.Config).
+
+ Example definition:
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ prefix: str
+
+ def output(self, text: str) -> None:
+ print(f"{self.prefix}{text}")
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def asset_that_uses_writer(writer: WriterResource):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[asset_that_uses_writer],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+ You can optionally use this class to model configuration only and vend an object
+ of a different type for use at runtime. This is useful for those who wish to
+ have a separate object that manages configuration and a separate object at runtime. Or
+ where you want to directly use a third-party class that you do not control.
+
+ To do this you override the create_resource methods to return a different object.
+
+ ```python
+ class WriterResource(ConfigurableResource):
+ str: prefix
+
+ def create_resource(self, context: InitResourceContext) -> Writer:
+ # Writer is pre-existing class defined else
+ return Writer(self.prefix)
+ ```
+ Example usage:
+
+ ```python
+ @asset
+ def use_preexisting_writer_as_resource(writer: ResourceParam[Writer]):
+ writer.output("text")
+
+ defs = Definitions(
+ assets=[use_preexisting_writer_as_resource],
+ resources={"writer": WriterResource(prefix="a_prefix")},
+ )
+ ```
+
+
+
+
+
+
class dagster.ResourceDefinition
+
+
+ Core class for defining resources.
+
+ Resources are scoped ways to make external resources (like database connections) available to
+ ops and assets during job execution and to clean up after execution resolves.
+
+ If resource_fn yields once rather than returning (in the manner of functions decorable with
+ `@contextlib.contextmanager`) then the body of the
+ function after the yield will be run after execution resolves, allowing users to write their
+ own teardown/cleanup logic.
+
+ Depending on your executor, resources may be instantiated and cleaned up more than once in a
+ job execution.
+
+ Parameters:
+ - resource_fn (Callable[[[*InitResourceContext*](#dagster.InitResourceContext)], Any]) – User-provided function to instantiate
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The schema for the config. If set, Dagster will check
+ - description (Optional[str]) – A human-readable description of the resource.
+ - required_resource_keys – (Optional[Set[str]]) Keys for the resources required by this
+ - version (Optional[str]) – experimental
+
+
+
+
static hardcoded_resource
+
+
+ A helper function that creates a `ResourceDefinition` with a hardcoded object.
+
+ Parameters:
+ - value (Any) – The value that will be accessible via context.resources.resource_name.
+ - description ([Optional[str]]) – The description of the resource. Defaults to None.
+
+
+ Returns: A hardcoded resource.Return type: [[ResourceDefinition](#dagster.ResourceDefinition)]
+
+
+
+
+
+
static mock_resource
+
+
+ A helper function that creates a `ResourceDefinition` which wraps a `mock.MagicMock`.
+
+ Parameters: description ([Optional[str]]) – The description of the resource. Defaults to None.Returns:
+ A resource that creates the magic methods automatically and helps
+ you mock existing resources.
+
+ Return type: [[ResourceDefinition](#dagster.ResourceDefinition)]
+
+
+
+
+
+
static none_resource
+
+
+ A helper function that returns a none resource.
+
+ Parameters: description ([Optional[str]]) – The description of the resource. Defaults to None.Returns: A resource that does nothing.Return type: [[ResourceDefinition](#dagster.ResourceDefinition)]
+
+
+
+
+
+
static string_resource
+
+
+ Creates a `ResourceDefinition` which takes in a single string as configuration
+ and returns this configured string to any ops or assets which depend on it.
+
+ Parameters: description ([Optional[str]]) – The description of the string resource. Defaults to None.Returns:
+ A resource that takes in a single string as configuration and
+ returns that string.
+
+ Return type: [[ResourceDefinition](#dagster.ResourceDefinition)]
+
+
+
+
+
+
property description
+
+ A human-readable description of the resource.
+
+
+
+
+
property required_resource_keys
+
+ A set of the resource keys that this resource depends on. These keys will be made available
+ to the resource’s init context during execution, and the resource will not be instantiated
+ until all required resources are available.
+
+
+
+
+
property version
+
+ A string which can be used to identify a particular code version of a resource definition.
+
+
+
+
+
+
+
+
+
class dagster.InitResourceContext
+
+
+ The context object available as the argument to the initialization function of a [`dagster.ResourceDefinition`](#dagster.ResourceDefinition).
+
+ Users should not instantiate this object directly. To construct an InitResourceContext for testing purposes, use [`dagster.build_init_resource_context()`](#dagster.build_init_resource_context).
+
+ Example:
+
+ ```python
+ from dagster import resource, InitResourceContext
+
+ @resource
+ def the_resource(init_context: InitResourceContext):
+ init_context.log.info("Hello, world!")
+ ```
+
+
property instance
+
+ The Dagster instance configured for the current execution context.
+
+
+
+
+
property log
+
+ The Dagster log manager configured for the current execution context.
+
+
+
+
+
property log_manager
+
+ The log manager for this run of the job.
+
+
+
+
+
property resource_config
+
+ The configuration data provided by the run config. The schema
+ for this data is defined by the `config_field` argument to
+ [`ResourceDefinition`](#dagster.ResourceDefinition).
+
+
+
+
+
property resource_def
+
+ The definition of the resource currently being constructed.
+
+
+
+
+
property resources
+
+ The resources that are available to the resource that we are initializing.
+
+
+
+
+
property run_id
+
+ The id for this run of the job or pipeline. When initializing resources outside of
+ execution context, this will be None.
+
+
+
+
+
+
+
+
+
+
dagster.make_values_resource
+
+
+ A helper function that creates a `ResourceDefinition` to take in user-defined values.
+
+ >
+
+ This is useful for sharing values between ops.
+
+
+
+Parameters: **kwargs – Arbitrary keyword arguments that will be passed to the config schema of the
+returned resource definition. If not set, Dagster will accept any config provided for
+the resource.
+For example:
+
+ ```python
+ @op(required_resource_keys={"globals"})
+ def my_op(context):
+ print(context.resources.globals["my_str_var"])
+
+ @job(resource_defs={"globals": make_values_resource(my_str_var=str, my_int_var=int)})
+ def my_job():
+ my_op()
+ ```
+Returns: A resource that passes in user-defined values.Return type: [ResourceDefinition](#dagster.ResourceDefinition)
+
+
+
+
+
+
dagster.build_init_resource_context
+
+
+ Builds resource initialization context from provided parameters.
+
+ `build_init_resource_context` can be used as either a function or context manager. If there is a
+ provided resource to `build_init_resource_context` that is a context manager, then it must be
+ used as a context manager. This function can be used to provide the context argument to the
+ invocation of a resource.
+
+ Parameters:
+ - resources (Optional[Dict[str, Any]]) – The resources to provide to the context. These can be
+ - config (Optional[Any]) – The resource config to provide to the context.
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The dagster instance configured for the context.
+
+
+ Examples:
+
+ ```python
+ context = build_init_resource_context()
+ resource_to_init(context)
+
+ with build_init_resource_context(
+ resources={"foo": context_manager_resource}
+ ) as context:
+ resource_to_init(context)
+ ```
+
+
+
+
+
+
dagster.build_resources
+
+
+ Context manager that yields resources using provided resource definitions and run config.
+
+ This API allows for using resources in an independent context. Resources will be initialized
+ with the provided run config, and optionally, dagster_run. The resulting resources will be
+ yielded on a dictionary keyed identically to that provided for resource_defs. Upon exiting the
+ context, resources will also be torn down safely.
+
+ Parameters:
+ - resources (Mapping[str, Any]) – Resource instances or definitions to build. All
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The dagster instance configured to instantiate
+ - resource_config (Optional[Mapping[str, Any]]) – A dict representing the config to be
+ - dagster_run (Optional[PipelineRun]) – The pipeline run to provide during resource
+ - log_manager (Optional[[*DagsterLogManager*](loggers.mdx#dagster.DagsterLogManager)]) – Log Manager to use during resource
+
+
+ Examples:
+
+ ```python
+ from dagster import resource, build_resources
+
+ @resource
+ def the_resource():
+ return "foo"
+
+ with build_resources(resources={"from_def": the_resource, "from_val": "bar"}) as resources:
+ assert resources.from_def == "foo"
+ assert resources.from_val == "bar"
+ ```
+
+
+
+
+
+
dagster.with_resources
+
+
+ Adds dagster resources to copies of resource-requiring dagster definitions.
+
+ An error will be thrown if any provided definitions have a conflicting
+ resource definition provided for a key provided to resource_defs. Resource
+ config can be provided, with keys in the config dictionary corresponding to
+ the keys for each resource definition. If any definition has unsatisfied
+ resource keys after applying with_resources, an error will be thrown.
+
+ Parameters:
+ - definitions (Iterable[ResourceAddable]) – Dagster definitions to provide resources to.
+ - resource_defs (Mapping[str, object]) – Mapping of resource keys to objects to satisfy
+ - resource_config_by_key (Optional[Mapping[str, Any]]) – Specifies config for provided resources. The key in this dictionary
+
+
+ Examples:
+
+ ```python
+ from dagster import asset, resource, with_resources
+
+ @resource(config_schema={"bar": str})
+ def foo_resource():
+ ...
+
+ @asset(required_resource_keys={"foo"})
+ def asset1(context):
+ foo = context.resources.foo
+ ...
+
+ @asset(required_resource_keys={"foo"})
+ def asset2(context):
+ foo = context.resources.foo
+ ...
+
+ asset1_with_foo, asset2_with_foo = with_resources(
+ [asset1, asset2],
+ resource_defs={
+ "foo": foo_resource
+ },
+ resource_config_by_key={
+ "foo": {
+ "config": {"bar": ...}
+ }
+ }
+ )
+ ```
+
+
+
+
+
+
+
+
+
+
+## Utilities
+
+
+
class dagster.EnvVar
+
+
+ Class used to represent an environment variable in the Dagster config system.
+
+ This class is intended to be used to populate config fields or resources.
+ The environment variable will be resolved to a string value when the config is
+ loaded.
+
+ To access the value of the environment variable, use the get_value method.
+
+
+
+
+
+
+
+
+
+
+## Legacy resource system
+
+The following classes are used as part of the [legacy resource system](https://legacy-docs.dagster.io/concepts/resources-legacy).
+
+
+
@dagster.resource
+
+
+ Define a resource.
+
+ The decorated function should accept an [`InitResourceContext`](#dagster.InitResourceContext) and return an instance of
+ the resource. This function will become the `resource_fn` of an underlying
+ [`ResourceDefinition`](#dagster.ResourceDefinition).
+
+ If the decorated function yields once rather than returning (in the manner of functions
+ decorable with `@contextlib.contextmanager`) then
+ the body of the function after the yield will be run after execution resolves, allowing users
+ to write their own teardown/cleanup logic.
+
+ Parameters:
+ - config_schema (Optional[[*ConfigSchema*](config.mdx#dagster.ConfigSchema)]) – The schema for the config. Configuration data available in
+ - description (Optional[str]) – A human-readable description of the resource.
+ - version (Optional[str]) – (Experimental) The version of a resource function. Two wrapped
+ - required_resource_keys (Optional[Set[str]]) – Keys for the resources required by this resource.
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/schedules-sensors.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/schedules-sensors.mdx
new file mode 100644
index 0000000000000..ca5dd08c5cd9d
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/schedules-sensors.mdx
@@ -0,0 +1,1307 @@
+---
+title: 'schedules and sensors'
+title_meta: 'schedules and sensors API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'schedules and sensors Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Schedules and sensors
+
+Dagster offers several ways to run data pipelines without manual intervation, including traditional scheduling and event-based triggers. [Automating your Dagster pipelines](https://docs.dagster.io/guides/automate/) can boost efficiency and ensure that data is produced consistently and reliably.
+
+
+
+
+## Run requests
+
+
+
class dagster.RunRequest
+
+
+
+
+
+
+
class dagster.SkipReason
+
+
+ Represents a skipped evaluation, where no runs are requested. May contain a message to indicate
+ why no runs were requested.
+
+
+
skip_message
+
+
+ A message displayed in the Dagster UI for why this evaluation resulted
+ in no requested runs.
+
+ Type: Optional[str]
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Schedules
+
+[Schedules](https://docs.dagster.io/guides/automate/schedules/) are Dagster’s way to support traditional ways of automation, such as specifying a job should run at Mondays at 9:00AM. Jobs triggered by schedules can contain a subset of [assets](https://docs.dagster.io/guides/build/assets/) or [ops](https://legacy-docs.dagster.io/concepts/ops-jobs-graphs/ops).
+
+
+
@dagster.schedule
+
+
+ Creates a schedule following the provided cron schedule and requests runs for the provided job.
+
+ The decorated function takes in a [`ScheduleEvaluationContext`](#dagster.ScheduleEvaluationContext) as its only
+ argument, and does one of the following:
+
+ 1. Return a [`RunRequest`](#dagster.RunRequest) object.
+ 2. Return a list of [`RunRequest`](#dagster.RunRequest) objects.
+ 3. Return a [`SkipReason`](#dagster.SkipReason) object, providing a descriptive message of why no runs were requested.
+ 4. Return nothing (skipping without providing a reason)
+ 5. Return a run config dictionary.
+ 6. Yield a [`SkipReason`](#dagster.SkipReason) or yield one ore more [`RunRequest`](#dagster.RunRequest) objects.
+ Returns a [`ScheduleDefinition`](#dagster.ScheduleDefinition).
+
+ Parameters:
+ - cron_schedule (Union[str, Sequence[str]]) – A valid cron string or sequence of cron strings
+ - name (Optional[str]) – The name of the schedule.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the schedule and can
+ - tags_fn (Optional[Callable[[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)], Optional[Dict[str, str]]]]) – A function
+ - metadata (Optional[Mapping[str, Any]]) – A set of metadata entries that annotate the
+ - should_execute (Optional[Callable[[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)], bool]]) – A function that runs at
+ - execution_timezone (Optional[str]) – Timezone in which the schedule should run.
+ - description (Optional[str]) – A human-readable description of the schedule.
+ - job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The job
+ - default_status (DefaultScheduleStatus) – If set to `RUNNING`, the schedule will immediately be active when starting Dagster. The default status can be overridden from the Dagster UI or via the GraphQL API.
+ - required_resource_keys (Optional[Set[str]]) – The set of resource keys required by the schedule.
+ - target (Optional[Union[CoercibleToAssetSelection, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The target that the schedule will execute.
+
+
+
+
+
+
+
+
class dagster.ScheduleDefinition
+
+
+ Defines a schedule that targets a job.
+
+ Parameters:
+ - name (Optional[str]) – The name of the schedule to create. Defaults to the job name plus
+ - cron_schedule (Union[str, Sequence[str]]) – A valid cron string or sequence of cron strings
+ - execution_fn (Callable[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)]) –
+
+ The core evaluation function for the schedule, which is run at an interval to determine whether a run should be launched or not. Takes a [`ScheduleEvaluationContext`](#dagster.ScheduleEvaluationContext).
+
+ - run_config (Optional[Union[[*RunConfig*](config.mdx#dagster.RunConfig), Mapping]]) – The config that parameterizes this execution,
+ - run_config_fn (Optional[Callable[[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)], [Mapping]]]) – A function that
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the schedule
+ - tags_fn (Optional[Callable[[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)], Optional[Mapping[str, str]]]]) – A
+ - should_execute (Optional[Callable[[[*ScheduleEvaluationContext*](#dagster.ScheduleEvaluationContext)], bool]]) – A function that runs
+ - execution_timezone (Optional[str]) –
+
+ - description (Optional[str]) – A human-readable description of the schedule.
+ - job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition)]]) – The job that should execute when this
+ - default_status (DefaultScheduleStatus) – If set to `RUNNING`, the schedule will start as running. The default status can be overridden from the Dagster UI or via the GraphQL API.
+ - required_resource_keys (Optional[Set[str]]) – The set of resource keys required by the schedule.
+ - target (Optional[Union[CoercibleToAssetSelection, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The target that the schedule will execute.
+ - metadata (Optional[Mapping[str, Any]]) – A set of metadata entries that annotate the
+
+
+
+
property cron_schedule
+
+
+ The cron schedule representing when this schedule will be evaluated.
+
+ Type: Union[str, Sequence[str]]
+
+
+
+
+
+
property default_status
+
+
+ The default status for this schedule when it is first loaded in
+ a code location.
+
+ Type: DefaultScheduleStatus
+
+
+
+
+
+
property description
+
+
+ A description for this schedule.
+
+ Type: Optional[str]
+
+
+
+
+
+
property environment_vars
+
+
+ :::danger[deprecated]
+ This API will be removed in version 2.0.
+ Setting this property no longer has any effect..
+
+ :::
+
+ Environment variables to export to the cron schedule.
+
+ Type: Mapping[str, str]
+
+
+
+
+
+
property execution_timezone
+
+
+ The timezone in which this schedule will be evaluated.
+
+ Type: Optional[str]
+
+
+
+
+
+
property job
+
+
+ The job that is
+ targeted by this schedule.
+
+ Type: Union[[JobDefinition](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]
+
+
+
+
+
+
property job_name
+
+
+ The name of the job targeted by this schedule.
+
+ Type: str
+
+
+
+
+
+
property metadata
+
+
+ The metadata for this schedule.
+
+ Type: Mapping[str, str]
+
+
+
+
+
+
property name
+
+
+ The name of the schedule.
+
+ Type: str
+
+
+
+
+
+
property required_resource_keys
+
+
+ The set of keys for resources that must be provided to this schedule.
+
+ Type: Set[str]
+
+
+
+
+
+
property tags
+
+
+ The tags for this schedule.
+
+ Type: Mapping[str, str]
+
+
+
+
+
+
+
+
+
+
class dagster.ScheduleEvaluationContext
+
+
+ The context object available as the first argument to various functions defined on a [`dagster.ScheduleDefinition`](#dagster.ScheduleDefinition).
+
+ A `ScheduleEvaluationContext` object is passed as the first argument to `run_config_fn`, `tags_fn`,
+ and `should_execute`.
+
+ Users should not instantiate this object directly. To construct a `ScheduleEvaluationContext` for testing purposes, use [`dagster.build_schedule_context()`](#dagster.build_schedule_context).
+
+ Example:
+
+ ```python
+ from dagster import schedule, ScheduleEvaluationContext
+
+ @schedule
+ def the_schedule(context: ScheduleEvaluationContext):
+ ...
+ ```
+
+
property instance
+
+
+ The current [`DagsterInstance`](internals.mdx#dagster.DagsterInstance).
+
+ Type: [DagsterInstance](internals.mdx#dagster.DagsterInstance)
+
+
+
+
+
+
property resources
+
+ Mapping of resource key to resource definition to be made available
+ during schedule execution.
+
+
+
+
+
property scheduled_execution_time
+
+ The time in which the execution was scheduled to happen. May differ slightly
+ from both the actual execution time and the time at which the run config is computed.
+
+
+
+
+
+
+
+
+
dagster.build_schedule_context
+
+
+ Builds schedule execution context using the provided parameters.
+
+ The instance provided to `build_schedule_context` must be persistent;
+ [`DagsterInstance.ephemeral()`](internals.mdx#dagster.DagsterInstance) will result in an error.
+
+ Parameters:
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The Dagster instance configured to run the schedule.
+ - scheduled_execution_time (datetime) – The time in which the execution was scheduled to
+
+
+ Examples:
+
+ ```python
+ context = build_schedule_context(instance)
+ ```
+
+
+
+
+
+
dagster.build_schedule_from_partitioned_job
+
+
+ Creates a schedule from a job that targets
+ time window-partitioned or statically-partitioned assets. The job can also be
+ multi-partitioned, as long as one of the partition dimensions is time-partitioned.
+
+ The schedule executes at the cadence specified by the time partitioning of the job or assets.
+
+ Example:
+ ```python
+ ######################################
+ # Job that targets partitioned assets
+ ######################################
+
+ from dagster import (
+ DailyPartitionsDefinition,
+ asset,
+ build_schedule_from_partitioned_job,
+ define_asset_job,
+ Definitions,
+ )
+
+ @asset(partitions_def=DailyPartitionsDefinition(start_date="2020-01-01"))
+ def asset1():
+ ...
+
+ asset1_job = define_asset_job("asset1_job", selection=[asset1])
+
+ # The created schedule will fire daily
+ asset1_job_schedule = build_schedule_from_partitioned_job(asset1_job)
+
+ defs = Definitions(assets=[asset1], schedules=[asset1_job_schedule])
+
+ ################
+ # Non-asset job
+ ################
+
+ from dagster import DailyPartitionsDefinition, build_schedule_from_partitioned_job, jog
+
+
+ @job(partitions_def=DailyPartitionsDefinition(start_date="2020-01-01"))
+ def do_stuff_partitioned():
+ ...
+
+ # The created schedule will fire daily
+ do_stuff_partitioned_schedule = build_schedule_from_partitioned_job(
+ do_stuff_partitioned,
+ )
+
+ defs = Definitions(schedules=[do_stuff_partitioned_schedule])
+ ```
+
+
+
+ Default scheduler implementation that submits runs from the long-lived `dagster-daemon`
+ process. Periodically checks each running schedule for execution times that don’t yet
+ have runs and launches them.
+
+
+
+
+
+
+
+
+
+
+## Sensors
+
+[Sensors](https://docs.dagster.io/guides/automate/sensors/) are typically used to poll, listen, and respond to external events. For example, you could configure a sensor to run a job or materialize an asset in response to specific events.
+
+
+
@dagster.sensor
+
+
+ Creates a sensor where the decorated function is used as the sensor’s evaluation function.
+
+ The decorated function may:
+
+ 1. Return a RunRequest object.
+ 2. Return a list of RunRequest objects.
+ 3. Return a SkipReason object, providing a descriptive message of why no runs were requested.
+ 4. Return nothing (skipping without providing a reason)
+ 5. Yield a SkipReason or yield one or more RunRequest objects.
+ Takes a `SensorEvaluationContext`.
+
+ Parameters:
+ - name (Optional[str]) – The name of the sensor. Defaults to the name of the decorated
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The job to be executed when the sensor fires.
+ - jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]]) – (experimental) A list of jobs to be executed when the sensor fires.
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - asset_selection (Optional[Union[str, Sequence[str], Sequence[[*AssetKey*](assets.mdx#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]], [*AssetSelection*](assets.mdx#dagster.AssetSelection)]]) – (Experimental) an asset selection to launch a run for if the sensor condition is met.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+ - target (Optional[Union[CoercibleToAssetSelection, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The target that the sensor will execute.
+
+
+
+
+
+
+
+
class dagster.SensorDefinition
+
+
+ Define a sensor that initiates a set of runs based on some external state.
+
+ Parameters:
+ - evaluation_fn (Callable[[SensorEvaluationContext]]) –
+
+ The core evaluation function for the
+ sensor, which is run at an interval to determine whether a run should be launched or
+ not. Takes a `SensorEvaluationContext`.
+
+ - name (Optional[str]) – The name of the sensor to create. Defaults to name of evaluation_fn
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - job (Optional[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJob]) – The job to execute when this sensor fires.
+ - jobs (Optional[Sequence[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJob]]) – (experimental) A list of jobs to execute when this sensor fires.
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - asset_selection (Optional[Union[str, Sequence[str], Sequence[[*AssetKey*](assets.mdx#dagster.AssetKey)], Sequence[Union[[*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*SourceAsset*](assets.mdx#dagster.SourceAsset)]], [*AssetSelection*](assets.mdx#dagster.AssetSelection)]]) – (Experimental) an asset selection to launch a run for if the sensor condition is met.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+ - target (Optional[Union[CoercibleToAssetSelection, [*AssetsDefinition*](assets.mdx#dagster.AssetsDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The target that the sensor will execute.
+
+
+
+
property default_status
+
+
+ The default status for this sensor when it is first loaded in
+ a code location.
+
+ Type: DefaultSensorStatus
+
+
+
+
+
+
property description
+
+
+ A description for this sensor.
+
+ Type: Optional[str]
+
+
+
+
+
+
property job
+
+
+ The job that is
+ targeted by this schedule.
+
+ Type: Union[[GraphDefinition](graphs.mdx#dagster.GraphDefinition), [JobDefinition](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]
+
+
+
+
+
+
property job_name
+
+
+ The name of the job that is targeted by this sensor.
+
+ Type: Optional[str]
+
+
+
+
+
+
property jobs
+
+
+ A list of jobs
+ that are targeted by this schedule.
+
+ Type: List[Union[[GraphDefinition](graphs.mdx#dagster.GraphDefinition), [JobDefinition](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]
+
+
+
+
+
+
property minimum_interval_seconds
+
+
+ The minimum number of seconds between sequential evaluations of this sensor.
+
+ Type: Optional[int]
+
+
+
+
+
+
property name
+
+
+ The name of this sensor.
+
+ Type: str
+
+
+
+
+
+
property required_resource_keys
+
+
+ The set of keys for resources that must be provided to this sensor.
+
+ Type: Set[str]
+
+
+
+
+
+
+
+
+
+
class dagster.SensorEvaluationContext
+
+
+ The context object available as the argument to the evaluation function of a [`dagster.SensorDefinition`](#dagster.SensorDefinition).
+
+ Users should not instantiate this object directly. To construct a
+ SensorEvaluationContext for testing purposes, use `dagster.
+ build_sensor_context()`.
+
+
+
instance_ref
+
+
+ The serialized instance configured to run the schedule
+
+ Type: Optional[[InstanceRef](internals.mdx#dagster._core.instance.InstanceRef)]
+
+
+
+
+
+
cursor
+
+
+ The cursor, passed back from the last sensor evaluation via
+ the cursor attribute of SkipReason and RunRequest
+
+ Type: Optional[str]
+
+
+
+
+
+
last_tick_completion_time
+
+
+ The last time that the sensor was evaluated (UTC).
+
+ Type: float
+
+
+
+
+
+
last_run_key
+
+
+ DEPRECATED The run key of the RunRequest most recently created by this
+ sensor. Use the preferred cursor attribute instead.
+
+ Type: str
+
+
+
+
+
+
log_key
+
+
+ The log key to use for this sensor tick.
+
+ Type: Optional[List[str]]
+
+
+
+
+
+
repository_name
+
+
+ The name of the repository that the sensor belongs to.
+
+ Type: Optional[str]
+
+
+
+
+
+
repository_def
+
+
+ The repository or that
+ the sensor belongs to. If needed by the sensor top-level resource definitions will be
+ pulled from this repository. You can provide either this or definitions.
+
+ Type: Optional[[RepositoryDefinition](repositories.mdx#dagster.RepositoryDefinition)]
+
+
+
+
+
+
instance
+
+
+ The deserialized instance can also be passed in
+ directly (primarily useful in testing contexts).
+
+ Type: Optional[[DagsterInstance](internals.mdx#dagster.DagsterInstance)]
+
+
+
+
+
+
definitions
+
+
+ Definitions object that the sensor is defined in.
+ If needed by the sensor, top-level resource definitions will be pulled from these
+ definitions. You can provide either this or repository_def.
+
+ Type: Optional[[Definitions](definitions.mdx#dagster.Definitions)]
+
+
+
+
+
+
resources
+
+
+ A dict of resource keys to resource
+ definitions to be made available during sensor execution.
+
+ Type: Optional[Dict[str, Any]]
+
+
+
+
+
+
last_sensor_start_time
+
+
+ The last time that the sensor was started (UTC).
+
+ Type: float
+
+
+
+
+
+
code_location_origin
+
+
+ The code location that the sensor is in.
+
+ Type: Optional[CodeLocationOrigin]
+
+
+
+ Updates the cursor value for this sensor, which will be provided on the context for the
+ next sensor evaluation.
+
+ This can be used to keep track of progress and avoid duplicate work across sensor
+ evaluations.
+
+ Parameters: cursor (Optional[str])
+
+
+
+
+
+
property cursor
+
+ The cursor value for this sensor, which was set in an earlier sensor evaluation.
+
+
+
+
+
property instance
+
+
+ The current DagsterInstance.
+
+ Type: [DagsterInstance](internals.mdx#dagster.DagsterInstance)
+
+
+
+
+
+
property is_first_tick_since_sensor_start
+
+ Flag representing if this is the first tick since the sensor was started.
+
+
+
+
+
property last_run_key
+
+
+ The run key supplied to the most recent RunRequest produced by this sensor.
+
+ Type: Optional[str]
+
+
+
+
+
+
property last_sensor_start_time
+
+
+ Timestamp representing the last time this sensor was started. Can be
+ used in concert with last_tick_completion_time to determine if this is the first tick since the
+ sensor was started.
+
+ Type: Optional[float]
+
+
+
+
+
+
property last_tick_completion_time
+
+
+ Timestamp representing the last time this sensor completed an evaluation.
+
+ Type: Optional[float]
+
+
+
+
+
+
property repository_def
+
+
+ The RepositoryDefinition that this sensor resides in.
+
+ Type: Optional[[RepositoryDefinition](repositories.mdx#dagster.RepositoryDefinition)]
+
+
+
+
+
+
property repository_name
+
+
+ The name of the repository that this sensor resides in.
+
+ Type: Optional[str]
+
+
+
+
+
+
property resources
+
+
+ A mapping from resource key to instantiated resources for this sensor.
+
+ Type: Resources
+
+
+
+
+
+
+
+
+
+
dagster.build_sensor_context
+
+
+ Builds sensor execution context using the provided parameters.
+
+ This function can be used to provide a context to the invocation of a sensor definition.If
+ provided, the dagster instance must be persistent; DagsterInstance.ephemeral() will result in an
+ error.
+
+ Parameters:
+ - instance (Optional[[*DagsterInstance*](internals.mdx#dagster.DagsterInstance)]) – The dagster instance configured to run the sensor.
+ - cursor (Optional[str]) – A cursor value to provide to the evaluation of the sensor.
+ - repository_name (Optional[str]) – The name of the repository that the sensor belongs to.
+ - repository_def (Optional[[*RepositoryDefinition*](repositories.mdx#dagster.RepositoryDefinition)]) – The repository that the sensor belongs to.
+ - resources (Optional[Mapping[str, [*ResourceDefinition*](resources.mdx#dagster.ResourceDefinition)]]) – A set of resource definitions
+ - definitions (Optional[[*Definitions*](definitions.mdx#dagster.Definitions)]) – Definitions object that the sensor is defined in.
+ - last_sensor_start_time (Optional[float]) – The last time the sensor was started.
+
+
+ Examples:
+
+ ```python
+ context = build_sensor_context()
+ my_sensor(context)
+ ```
+
+
+
+
+
+
@dagster.asset_sensor
+
+
+ Creates an asset sensor where the decorated function is used as the asset sensor’s evaluation
+ function.
+
+ If the asset has been materialized multiple times between since the last sensor tick, the
+ evaluation function will only be invoked once, with the latest materialization.
+
+ The decorated function may:
+
+ 1. Return a RunRequest object.
+ 2. Return a list of RunRequest objects.
+ 3. Return a SkipReason object, providing a descriptive message of why no runs were requested.
+ 4. Return nothing (skipping without providing a reason)
+ 5. Yield a SkipReason or yield one or more RunRequest objects.
+ Takes a `SensorEvaluationContext` and an EventLogEntry corresponding to an
+ AssetMaterialization event.
+
+ Parameters:
+ - asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – The asset_key this sensor monitors.
+ - name (Optional[str]) – The name of the sensor. Defaults to the name of the decorated
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The
+ - jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]]) – (experimental) A list of jobs to be executed when the sensor fires.
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+
+
+ Example:
+
+ ```python
+ from dagster import AssetKey, EventLogEntry, SensorEvaluationContext, asset_sensor
+
+
+ @asset_sensor(asset_key=AssetKey("my_table"), job=my_job)
+ def my_asset_sensor(context: SensorEvaluationContext, asset_event: EventLogEntry):
+ return RunRequest(
+ run_key=context.cursor,
+ run_config={
+ "ops": {
+ "read_materialization": {
+ "config": {
+ "asset_key": asset_event.dagster_event.asset_key.path,
+ }
+ }
+ }
+ },
+ )
+ ```
+
+
+
+
+
+
class dagster.AssetSensorDefinition
+
+
+ Define an asset sensor that initiates a set of runs based on the materialization of a given
+ asset.
+
+ If the asset has been materialized multiple times between since the last sensor tick, the
+ evaluation function will only be invoked once, with the latest materialization.
+
+ Parameters:
+ - name (str) – The name of the sensor to create.
+ - asset_key ([*AssetKey*](assets.mdx#dagster.AssetKey)) – The asset_key this sensor monitors.
+ - asset_materialization_fn (Callable[[SensorEvaluationContext, [*EventLogEntry*](internals.mdx#dagster.EventLogEntry)], Union[Iterator[Union[[*RunRequest*](#dagster.RunRequest), [*SkipReason*](#dagster.SkipReason)]], [*RunRequest*](#dagster.RunRequest), [*SkipReason*](#dagster.SkipReason)]]) –
+
+ The core
+ evaluation function for the sensor, which is run at an interval to determine whether a
+ run should be launched or not. Takes a `SensorEvaluationContext` and
+ an EventLogEntry corresponding to an AssetMaterialization event.
+
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The job
+ - jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]]) – (experimental) A list of jobs to be executed when the sensor fires.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+
+
+
+
property asset_key
+
+
+ The key of the asset targeted by this sensor.
+
+ Type: [AssetKey](assets.mdx#dagster.AssetKey)
+
+
+
+
+
+
+
+
+
+
class dagster.RunStatusSensorDefinition
+
+
+ Define a sensor that reacts to a given status of job execution, where the decorated
+ function will be evaluated when a run is at the given status.
+
+ Parameters:
+ - name (str) – The name of the sensor. Defaults to the name of the decorated function.
+ - run_status ([*DagsterRunStatus*](internals.mdx#dagster.DagsterRunStatus)) – The status of a run which will be
+ - run_status_sensor_fn (Callable[[[*RunStatusSensorContext*](#dagster.RunStatusSensorContext)], Union[[*SkipReason*](#dagster.SkipReason), DagsterRunReaction]]) – The core
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), UnresolvedAssetJobDefinition, [*JobSelector*](#dagster.JobSelector), [*RepositorySelector*](#dagster.RepositorySelector), CodeLocationSelector]]]) – The jobs in the current repository that will be monitored by this sensor. Defaults to
+ - monitor_all_code_locations (bool) – If set to True, the sensor will monitor all runs in the
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - request_job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition)]]) – The job a RunRequest should
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+ - request_jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition)]]]) – (experimental)
+
+
+
+
+
+
+
+
class dagster.RunStatusSensorContext
+
+
+ The `context` object available to a decorated function of `run_status_sensor`.
+
+
+
property dagster_event
+
+ The event associated with the job run status.
+
+
+
+
+
property dagster_run
+
+ The run of the job.
+
+
+
+
+
property instance
+
+ The current instance.
+
+
+
+
+
property log
+
+ The logger for the current sensor evaluation.
+
+
+
+
+
property partition_key
+
+
+ The partition key of the relevant run.
+
+ Type: Optional[str]
+
+
+
+
+
+
property sensor_name
+
+ The name of the sensor.
+
+
+
+
+
+
+
+
+
class dagster.RunFailureSensorContext
+
+
+ The `context` object available to a decorated function of `run_failure_sensor`.
+
+
+
+ The step failure event for each step in the run that failed.
+
+ Examples:
+
+ ```python
+ error_strings_by_step_key = {
+ # includes the stack trace
+ event.step_key: event.event_specific_data.error.to_string()
+ for event in context.get_step_failure_events()
+ }
+ ```
+
+
+
+
+
+
property failure_event
+
+
+ The run failure event.
+
+ If the run failed because of an error inside a step, get_step_failure_events will have more
+ details on the step failure.
+
+
+
+
+
+
+
+
+
+
+
class dagster.JobSelector
+
+
+
+
+
+
+
class dagster.RepositorySelector
+
+
+
+
+
+
+
dagster.build_run_status_sensor_context
+
+
+ Builds run status sensor context from provided parameters.
+
+ This function can be used to provide the context argument when directly invoking a function
+ decorated with @run_status_sensor or @run_failure_sensor, such as when writing unit tests.
+
+ Parameters:
+ - sensor_name (str) – The name of the sensor the context is being constructed for.
+ - dagster_event ([*DagsterEvent*](execution.mdx#dagster.DagsterEvent)) – A DagsterEvent with the same event type as the one that
+ - dagster_instance ([*DagsterInstance*](internals.mdx#dagster.DagsterInstance)) – The dagster instance configured for the context.
+ - dagster_run ([*DagsterRun*](internals.mdx#dagster.DagsterRun)) – DagsterRun object from running a job
+ - resources (Optional[Mapping[str, object]]) – A dictionary of resources to be made available
+ - repository_def (Optional[[*RepositoryDefinition*](repositories.mdx#dagster.RepositoryDefinition)]) – experimental
+
+
+ Examples:
+
+ ```python
+ instance = DagsterInstance.ephemeral()
+ result = my_job.execute_in_process(instance=instance)
+
+ dagster_run = result.dagster_run
+ dagster_event = result.get_job_success_event() # or get_job_failure_event()
+
+ context = build_run_status_sensor_context(
+ sensor_name="run_status_sensor_to_invoke",
+ dagster_instance=instance,
+ dagster_run=dagster_run,
+ dagster_event=dagster_event,
+ )
+ run_status_sensor_to_invoke(context)
+ ```
+
+
+
+
+
+
@dagster.run_status_sensor
+
+
+ Creates a sensor that reacts to a given status of job execution, where the decorated
+ function will be run when a job is at the given status.
+
+ Takes a [`RunStatusSensorContext`](#dagster.RunStatusSensorContext).
+
+ Parameters:
+ - run_status ([*DagsterRunStatus*](internals.mdx#dagster.DagsterRunStatus)) – The status of run execution which will be
+ - name (Optional[str]) – The name of the sensor. Defaults to the name of the decorated function.
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), UnresolvedAssetJobDefinition, [*RepositorySelector*](#dagster.RepositorySelector), [*JobSelector*](#dagster.JobSelector), CodeLocationSelector]]]) – Jobs in the current code locations that will be monitored by this sensor. Defaults to None, which means the alert will
+ - monitor_all_code_locations (Optional[bool]) – If set to True, the sensor will monitor all runs in the Dagster deployment.
+ - job_selection (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*RepositorySelector*](#dagster.RepositorySelector), [*JobSelector*](#dagster.JobSelector), CodeLocationSelector]]]) – deprecatedmonitored_jobs instead.) (deprecated in favor of monitored_jobs) Jobs in the current code location that will be
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - request_job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]) – The job that should be
+ - request_jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJobDefinition]]]) – (experimental)
+ - monitor_all_repositories (Optional[bool]) – deprecatedmonitor_all_code_locations instead.) (deprecated in favor of monitor_all_code_locations) If set to True, the sensor will monitor all runs in the Dagster instance.
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+
+
+
+
+
+
+
+
@dagster.run_failure_sensor
+
+
+ Creates a sensor that reacts to job failure events, where the decorated function will be
+ run when a run fails.
+
+ Takes a [`RunFailureSensorContext`](#dagster.RunFailureSensorContext).
+
+ Parameters:
+ - name (Optional[str]) – The name of the job failure sensor. Defaults to the name of the
+ - minimum_interval_seconds (Optional[int]) – The minimum number of seconds that will elapse
+ - description (Optional[str]) – A human-readable description of the sensor.
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), UnresolvedAssetJobDefinition, [*RepositorySelector*](#dagster.RepositorySelector), [*JobSelector*](#dagster.JobSelector), CodeLocationSelector]]]) – The jobs in the current repository that will be monitored by this failure sensor.
+ - monitor_all_code_locations (bool) – If set to True, the sensor will monitor all runs in the
+ - job_selection (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*RepositorySelector*](#dagster.RepositorySelector), [*JobSelector*](#dagster.JobSelector), CodeLocationSelector]]]) – deprecatedmonitored_jobs instead.) (deprecated in favor of monitored_jobs) The jobs in the current repository that will be
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - request_job (Optional[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJob]]) – The job a RunRequest should
+ - request_jobs (Optional[Sequence[Union[[*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), UnresolvedAssetJob]]]) – (experimental)
+ - monitor_all_repositories (bool) – deprecatedmonitor_all_code_locations instead.) (deprecated in favor of monitor_all_code_locations) If set to True,
+ - tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can
+ - metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the
+
+
+
+
+
+
+
+
class dagster.SensorResult
+
+
+ The result of a sensor evaluation.
+
+
+
run_requests
+
+
+ A list
+ of run requests to be executed.
+
+ Type: Optional[Sequence[[RunRequest](#dagster.RunRequest)]]
+
+
+
+
+
+
skip_reason
+
+
+ A skip message indicating why sensor
+ evaluation was skipped.
+
+ Type: Optional[Union[str, [SkipReason](#dagster.SkipReason)]]
+
+
+
+
+
+
cursor
+
+
+ The cursor value for this sensor, which will be provided on the
+ context for the next sensor evaluation.
+
+ Type: Optional[str]
+
+
+ AddDynamicPartitionsRequest]]]): A list of dynamic partition requests to request dynamic
+ partition addition and deletion. Run requests will be evaluated using the state of the
+ partitions with these changes applied. We recommend limiting partition additions
+ and deletions to a maximum of 25K partitions per sensor evaluation, as this is the maximum
+ recommended partition limit per asset.
+
+
+
+
+
asset_events
+
+
+ (Experimental) A
+ list of materializations, observations, and asset check evaluations that the system
+ will persist on your behalf at the end of sensor evaluation. These events will be not
+ be associated with any particular run, but will be queryable and viewable in the asset catalog.
+
+ Type: Optional[Sequence[Union[[AssetObservation](assets.mdx#dagster.AssetObservation), [AssetMaterialization](ops.mdx#dagster.AssetMaterialization), AssetCheckEvaluation]]]
+
+
+
+
+
+
+
+
+
+
class dagster.AddDynamicPartitionsRequest
+
+ A request to add partitions to a dynamic partitions definition, to be evaluated by a sensor or schedule.
+
+
+
+
+
class dagster.DeleteDynamicPartitionsRequest
+
+ A request to delete partitions to a dynamic partitions definition, to be evaluated by a sensor or schedule.
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/types.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/types.mdx
new file mode 100644
index 0000000000000..1a5c846eb696b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/types.mdx
@@ -0,0 +1,374 @@
+---
+title: 'types'
+title_meta: 'types API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'types Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+
+# Types
+
+Dagster includes facilities for typing the input and output values of ops (“runtime” types).
+
+
+
+
+
+## Built-in types
+
+
+
dagster.Nothing
+
+
+ Use this type only for inputs and outputs, in order to establish an execution dependency without
+ communicating a value. Inputs of this type will not be passed to the op compute function, so
+ it is necessary to use the explicit [`In`](ops.mdx#dagster.In) API to define them rather than
+ the Python 3 type hint syntax.
+
+ All values are considered to be instances of `Nothing`.
+
+ Examples:
+
+ ```python
+ @op
+ def wait(_) -> Nothing:
+ time.sleep(1)
+ return
+
+ @op(
+ ins={"ready": In(dagster_type=Nothing)},
+ )
+ def done(_) -> str:
+ return 'done'
+
+ @job
+ def nothing_job():
+ done(wait())
+
+ # Any value will pass the type check for Nothing
+ @op
+ def wait_int(_) -> Int:
+ time.sleep(1)
+ return 1
+
+ @job
+ def nothing_int_job():
+ done(wait_int())
+ ```
+
+
+
+
+
+
+
+
+
+
+## Making New Types
+
+
+
class dagster.DagsterType
+
+
+ Define a type in dagster. These can be used in the inputs and outputs of ops.
+
+ Parameters:
+ - type_check_fn (Callable[[[*TypeCheckContext*](execution.mdx#dagster.TypeCheckContext), Any], [Union[bool, [*TypeCheck*](ops.mdx#dagster.TypeCheck)]]]) – The function that defines the type check. It takes the value flowing
+ - key (Optional[str]) –
+
+ The unique key to identify types programmatically.
+ The key property always has a value. If you omit key to the argument
+ to the init function, it instead receives the value of `name`. If
+ neither `key` nor `name` is provided, a `CheckError` is thrown.
+
+ In the case of a generic type such as `List` or `Optional`, this is
+ generated programmatically based on the type parameters.
+
+ - name (Optional[str]) – A unique name given by a user. If `key` is `None`, `key`
+ - description (Optional[str]) – A markdown-formatted string, displayed in tooling.
+ - loader (Optional[[*DagsterTypeLoader*](#dagster.DagsterTypeLoader)]) – An instance of a class that
+ - required_resource_keys (Optional[Set[str]]) – Resource keys required by the `type_check_fn`.
+ - is_builtin (bool) – Defaults to False. This is used by tools to display or
+ - kind (DagsterTypeKind) – Defaults to None. This is used to determine the kind of runtime type
+ - typing_type – Defaults to None. A valid python typing type (e.g. Optional[List[int]]) for the
+
+
+
+
type_check
+
+
+ Type check the value against the type.
+
+ Parameters:
+ - context ([*TypeCheckContext*](execution.mdx#dagster.TypeCheckContext)) – The context of the type check.
+ - value (Any) – The value to check.
+
+
+ Returns: The result of the type check.Return type: [TypeCheck](ops.mdx#dagster.TypeCheck)
+
+
+
+
+
+
property description
+
+
+ Description of the type, or None if not provided.
+
+ Type: Optional[str]
+
+
+
+
+
+
property display_name
+
+ Either the name or key (if name is None) of the type, overridden in many subclasses.
+
+
+
+
+
property has_unique_name
+
+
+ Whether the type has a unique name.
+
+ Type: bool
+
+
+
+
+
+
property loader
+
+
+ Loader for this type, if any.
+
+ Type: Optional[[DagsterTypeLoader](#dagster.DagsterTypeLoader)]
+
+
+
+
+
+
property required_resource_keys
+
+
+ Set of resource keys required by the type check function.
+
+ Type: AbstractSet[str]
+
+
+
+
+
+
property typing_type
+
+
+ The python typing type for this type.
+
+ Type: Any
+
+
+
+
+
+
property unique_name
+
+ The unique name of this type. Can be None if the type is not unique, such as container types.
+
+
+
+
+
+
+
+
+
dagster.PythonObjectDagsterType
+
+
+ Define a type in dagster whose typecheck is an isinstance check.
+
+ Specifically, the type can either be a single python type (e.g. int),
+ or a tuple of types (e.g. (int, float)) which is treated as a union.
+
+ Examples:
+
+ ```python
+ ntype = PythonObjectDagsterType(python_type=int)
+ assert ntype.name == 'int'
+ assert_success(ntype, 1)
+ assert_failure(ntype, 'a')
+ ```
+ ```python
+ ntype = PythonObjectDagsterType(python_type=(int, float))
+ assert ntype.name == 'Union[int, float]'
+ assert_success(ntype, 1)
+ assert_success(ntype, 1.5)
+ assert_failure(ntype, 'a')
+ ```
+ Parameters:
+ - python_type (Union[Type, Tuple[Type, ...]) – The dagster typecheck function calls instanceof on
+ - name (Optional[str]) – Name the type. Defaults to the name of `python_type`.
+ - key (Optional[str]) – Key of the type. Defaults to name.
+ - description (Optional[str]) – A markdown-formatted string, displayed in tooling.
+ - loader (Optional[[*DagsterTypeLoader*](#dagster.DagsterTypeLoader)]) – An instance of a class that
+
+
+
+
+
+
+
+
dagster.dagster_type_loader
+
+
+ Create an dagster type loader that maps config data to a runtime value.
+
+ The decorated function should take the execution context and parsed config value and return the
+ appropriate runtime value.
+
+ Parameters: config_schema ([*ConfigSchema*](config.mdx#dagster.ConfigSchema)) – The schema for the config that’s passed to the decorated
+ function.
+ Examples:
+
+ ```python
+ @dagster_type_loader(Permissive())
+ def load_dict(_context, value):
+ return value
+ ```
+
+
+
+
+
+
class dagster.DagsterTypeLoader
+
+
+ Dagster type loaders are used to load unconnected inputs of the dagster type they are attached
+ to.
+
+ The recommended way to define a type loader is with the
+ [`@dagster_type_loader`](#dagster.dagster_type_loader) decorator.
+
+
+
+
+
+
+
class dagster.DagsterTypeLoaderContext
+
+
+ The context object provided to a [`@dagster_type_loader`](#dagster.dagster_type_loader)-decorated function during execution.
+
+ Users should not construct this object directly.
+
+
+
property job_def
+
+ The underlying job definition being executed.
+
+
+
+
+
property op_def
+
+ The op for which type loading is occurring.
+
+
+
+
+
property resources
+
+ The resources available to the type loader, specified by the required_resource_keys argument of the decorator.
+
+
+
+
+
+
+
+
+
dagster.usable_as_dagster_type
+
+
+ Decorate a Python class to make it usable as a Dagster Type.
+
+ This is intended to make it straightforward to annotate existing business logic classes to
+ make them dagster types whose typecheck is an isinstance check against that python class.
+
+ Parameters:
+ - python_type (cls) – The python type to make usable as python type.
+ - name (Optional[str]) – Name of the new Dagster type. If `None`, the name (`__name__`) of
+ - description (Optional[str]) – A user-readable description of the type.
+ - loader (Optional[[*DagsterTypeLoader*](#dagster.DagsterTypeLoader)]) – An instance of a class that
+
+
+ Examples:
+
+ ```python
+ # dagster_aws.s3.file_manager.S3FileHandle
+ @usable_as_dagster_type
+ class S3FileHandle(FileHandle):
+ def __init__(self, s3_bucket, s3_key):
+ self._s3_bucket = check.str_param(s3_bucket, 's3_bucket')
+ self._s3_key = check.str_param(s3_key, 's3_key')
+
+ @property
+ def s3_bucket(self):
+ return self._s3_bucket
+
+ @property
+ def s3_key(self):
+ return self._s3_key
+
+ @property
+ def path_desc(self):
+ return self.s3_path
+
+ @property
+ def s3_path(self):
+ return 's3://{bucket}/{key}'.format(bucket=self.s3_bucket, key=self.s3_key)
+ ```
+
+
+
+
+
+
dagster.make_python_type_usable_as_dagster_type
+
+ Take any existing python type and map it to a dagster type (generally created with
+ [`DagsterType`](#dagster.DagsterType)) This can only be called once
+ on a given python type.
+
+
+
+
+
+### Testing Types
+
+
+
dagster.check_dagster_type
+
+
+ Test a custom Dagster type.
+
+ Parameters:
+ - dagster_type (Any) – The Dagster type to test. Should be one of the
+ - value (Any) – The runtime value to test.
+
+
+ Returns: The result of the type check.Return type: [TypeCheck](ops.mdx#dagster.TypeCheck)
+ Examples:
+
+ ```python
+ assert check_dagster_type(Dict[Any, Any], {'foo': 'bar'}).success
+ ```
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/utilities.mdx b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/utilities.mdx
new file mode 100644
index 0000000000000..a6e620570530b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/api/python-api/utilities.mdx
@@ -0,0 +1,223 @@
+---
+title: 'utilities'
+title_meta: 'utilities API Documentation - Build Better Data Pipelines | Python Reference Documentation for Dagster'
+description: 'utilities Dagster API | Comprehensive Python API documentation for Dagster, the data orchestration platform. Learn how to build, test, and maintain data pipelines with our detailed guides and examples.'
+last_update:
+ date: '2025-02-03'
+---
+
+
+
+# Utilities
+
+
+
dagster.file_relative_path
+
+
+ Get a path relative to the currently executing Python file.
+
+ This function is useful when one needs to load a file that is relative to the position of
+ the current file. (Such as when you encode a configuration file path in source file and want
+ in runnable in any current working directory)
+
+ Parameters:
+ - dunderfile (str) – Should always be `__file__`.
+ - relative_path (str) – Path to get relative to the currently executing file.
+
+
+ Examples:
+
+ ```python
+ file_relative_path(__file__, 'path/relative/to/file')
+ ```
+
+
+
+
+
+
dagster.config_from_files
+
+
+ Constructs run config from YAML files.
+
+ Parameters: config_files (List[str]) – List of paths or glob patterns for yaml files
+ to load and parse as the run config.Returns: A run config dictionary constructed from provided YAML files.Return type: Dict[str, Any]Raises:
+ - FileNotFoundError – When a config file produces no results
+ - [DagsterInvariantViolationError](errors.mdx#dagster.DagsterInvariantViolationError)DagsterInvariantViolationError – When one of the YAML files is invalid and has a parse
+
+
+
+
+
+
+
+
dagster.config_from_pkg_resources
+
+
+ Load a run config from a package resource, using `pkg_resources.resource_string()`.
+
+ Example:
+
+ ```python
+ config_from_pkg_resources(
+ pkg_resource_defs=[
+ ('dagster_examples.airline_demo.environments', 'local_base.yaml'),
+ ('dagster_examples.airline_demo.environments', 'local_warehouse.yaml'),
+ ],
+ )
+ ```
+ Parameters: pkg_resource_defs (List[(str, str)]) – List of pkg_resource modules/files to
+ load as the run config.Returns: A run config dictionary constructed from the provided yaml stringsReturn type: Dict[Str, Any]Raises: [DagsterInvariantViolationError](errors.mdx#dagster.DagsterInvariantViolationError)DagsterInvariantViolationError – When one of the YAML documents is invalid and has a
+ parse error.
+
+
+
+
+
+
dagster.config_from_yaml_strings
+
+
+ Static constructor for run configs from YAML strings.
+
+ Parameters: yaml_strings (List[str]) – List of yaml strings to parse as the run config.Returns: A run config dictionary constructed from the provided yaml stringsReturn type: Dict[Str, Any]Raises: [DagsterInvariantViolationError](errors.mdx#dagster.DagsterInvariantViolationError)DagsterInvariantViolationError – When one of the YAML documents is invalid and has a
+ parse error.
+
+
+
+
+
+
dagster.get_dagster_logger
+
+
+ Creates a python logger whose output messages will be captured and converted into Dagster log
+ messages. This means they will have structured information such as the step_key, run_id, etc.
+ embedded into them, and will show up in the Dagster event log.
+
+ This can be used as a more convenient alternative to context.log in most cases. If log level
+ is not set explicitly, defaults to DEBUG.
+
+ Parameters: name (Optional[str]) – If supplied, will create a logger with the name “dagster.builtin.\{name}”,
+ with properties inherited from the base Dagster logger. If omitted, the returned logger
+ will be named “dagster.builtin”.Returns: A logger whose output will be captured by Dagster.Return type: `logging.Logger`
+ Example:
+
+ ```python
+ from dagster import get_dagster_logger, op
+
+ @op
+ def hello_op():
+ log = get_dagster_logger()
+ for i in range(5):
+ # do something
+ log.info(f"Did {i+1} things!")
+ ```
+
+
+
+
+
+
class dagster.ExperimentalWarning
+
+
+
+
+
+
+
class dagster._utils.warnings.PreviewWarning
+
+
+
+
+
+
+
class dagster._utils.warnings.BetaWarning
+
+
+
+
+
+
+
dagster.make_email_on_run_failure_sensor
+
+
+ Create a job failure sensor that sends email via the SMTP protocol.
+
+ Parameters:
+ - email_from (str) – The sender email address to send the message from.
+ - email_password (str) – The password of the sender.
+ - email_to (List[str]) – The receipt email addresses to send the message to.
+ - email_body_fn (Optional(Callable[[[*RunFailureSensorContext*](schedules-sensors.mdx#dagster.RunFailureSensorContext)], str])) – Function which
+ - email_subject_fn (Optional(Callable[[[*RunFailureSensorContext*](schedules-sensors.mdx#dagster.RunFailureSensorContext)], str])) – Function which
+ - smtp_host (str) – The hostname of the SMTP server. Defaults to “smtp.gmail.com”.
+ - smtp_type (str) – The protocol; either “SSL” or “STARTTLS”. Defaults to SSL.
+ - smtp_port (Optional[int]) – The SMTP port. Defaults to 465 for SSL, 587 for STARTTLS.
+ - smtp_user (Optional[str]) – The SMTP user for authenticatication in the SMTP server. Defaults to the value of email_from.
+ - name – (Optional[str]): The name of the sensor. Defaults to “email_on_job_failure”.
+ - webserver_base_url – (Optional[str]): The base url of your dagster-webserver instance. Specify this to allow
+ - monitored_jobs (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*RepositorySelector*](schedules-sensors.mdx#dagster.RepositorySelector), [*JobSelector*](schedules-sensors.mdx#dagster.JobSelector)]]]) – The jobs that will be monitored by this failure sensor. Defaults to None, which means the alert will
+ - monitor_all_code_locations (bool) – If set to True, the sensor will monitor all runs in the
+ - job_selection (Optional[List[Union[[*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*GraphDefinition*](graphs.mdx#dagster.GraphDefinition), [*JobDefinition*](jobs.mdx#dagster.JobDefinition), [*RepositorySelector*](schedules-sensors.mdx#dagster.RepositorySelector), [*JobSelector*](schedules-sensors.mdx#dagster.JobSelector)]]]) – deprecatedmonitored_jobs instead.) (deprecated in favor of monitored_jobs) The jobs that will be monitored by this failure
+ - default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default
+ - monitor_all_repositories (bool) – deprecatedmonitor_all_code_locations instead.) If set to True, the sensor will monitor all runs in the
+
+
+ Examples:
+
+ ```python
+ email_on_run_failure = make_email_on_run_failure_sensor(
+ email_from="no-reply@example.com",
+ email_password=os.getenv("ALERT_EMAIL_PASSWORD"),
+ email_to=["xxx@example.com"],
+ )
+
+ @repository
+ def my_repo():
+ return [my_job + email_on_run_failure]
+ ```
+ ```python
+ def my_message_fn(context: RunFailureSensorContext) -> str:
+ return (
+ f"Job {context.dagster_run.job_name} failed!"
+ f"Error: {context.failure_event.message}"
+ )
+
+ email_on_run_failure = make_email_on_run_failure_sensor(
+ email_from="no-reply@example.com",
+ email_password=os.getenv("ALERT_EMAIL_PASSWORD"),
+ email_to=["xxx@example.com"],
+ email_body_fn=my_message_fn,
+ email_subject_fn=lambda _: "Dagster Alert",
+ webserver_base_url="http://mycoolsite.com",
+ )
+ ```
+
+
+
+
+
+
class dagster._utils.forked_pdb.ForkedPdb
+
+
+ A pdb subclass that may be used from a forked multiprocessing child.
+
+ Examples:
+
+ ```python
+ from dagster._utils.forked_pdb import ForkedPdb
+
+ @solid
+ def complex_solid(_):
+ # some complicated stuff
+
+ ForkedPdb().set_trace()
+
+ # some other complicated stuff
+ ```
+ You can initiate pipeline execution via the webserver and use the pdb debugger to examine/step through
+ execution at the breakpoint.
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-locations/code-location-history.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-locations/code-location-history.md
new file mode 100644
index 0000000000000..37f49f5bf62a3
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-locations/code-location-history.md
@@ -0,0 +1,53 @@
+---
+title: "Code location history and rollbacks"
+sidebar_position: 100
+---
+
+Dagster+ automatically tracks metadata every time a code location is loaded. This can be used to understand when changes have been made, and what those changes were. In addition, this metadata can be used to quickly redeploy an older version.
+
+
+ Prerequisites
+
+Before continuing, you should be familiar with:
+
+- [Code Locations](/dagster-plus/deployment/code-locations)
+
+
+
+## Viewing code location history
+
+1. In the Dagster+ UI, navigate to the **Deployment** tab.
+2. In the row associated with the code location you're interested in, click **View history** in the **Updated** column.
+
+![Screenshot highlighting the "Updated" column for a code location](/images/dagster-plus/deployment/code-locations/view-code-location-history.png)
+
+This will bring up a modal showing a history of every time that code location has been loaded, and metadata associated with that load. If you have connected Dagster+ to a GitHub or GitLab repository, each row will have a link to the commit that was deployed at that point in time.
+
+If a code location has been deployed multiple times with identical metadata, these rows will be collapsed together. You can expand them by deselecting **Collapse similar entries** in the top left corner of the modal.
+
+This metadata will also include information regarding assets that have been **added**, **removed**, or **changed**. In the **Assets** column, you can see the keys of assets in any of these categories.
+
+![Screenshot highlighting the column that displays these keys](/images/dagster-plus/deployment/code-locations/code-location-history-metadata.png)
+
+Currently, changes to **code version**, **tags**, **metadata**, **dependencies** and **partitions definition** are tracked. Clicking on any of these assets brings you to its **Asset details** page. Here, you can find the **Change history** tab, and see detailed information regarding each time the asset definition has changed.
+
+![Screenshot highlighting the Change History tab for an asset](/images/dagster-plus/deployment/code-locations/asset-change-history.png)
+
+## Rolling back to a previous code location version
+
+:::note
+To initiate a rollback, you'll need **Organization**, **Admin**, or **Editor** permissions
+:::
+
+If you notice an issue with newly deployed code, or your code fails to deploy successfully, you can quickly roll back to a previously deployed image that's known to work properly.
+
+1. In the Dagster+ UI, navigate to the **Deployment** tab.
+2. In the row associated with the code location you're interested in, click **View history** in the **Updated** column.
+3. In the **Actions** column click the dropdown menu to the right of **View metadata**, select **Rollback to this version**.
+
+![Screenshot highlighting the "Updated" column for a code location](/images/dagster-plus/deployment/code-locations/rollback-code-location.png)
+
+## Next steps
+
+- Learn more about [Code Locations](/dagster-plus/deployment/code-locations)
+- Learn how to [Alert when a code location fails to load](/dagster-plus/features/alerts/creating-alerts#alerting-when-a-code-location-fails-to-load)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-locations/dagster-cloud-yaml.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-locations/dagster-cloud-yaml.md
new file mode 100644
index 0000000000000..12fe2be66cedc
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-locations/dagster-cloud-yaml.md
@@ -0,0 +1,235 @@
+---
+title: dagster_cloud.yaml reference
+sidebar_position: 200
+---
+
+:::note
+This reference is applicable to Dagster+.
+:::
+
+`dagster_cloud.yaml` is used to define multiple code locations for Dagster+. It is similar to `workspace.yaml` in Dagster open source. For [Hybrid deployments](/dagster-plus/deployment/deployment-types/hybrid/), `dagster_cloud.yaml` can be used to manage environment variables and secrets.
+
+
+## File location
+
+The `dagster_cloud.yaml` file should be placed in the root of your Dagster project, similar to the example below:
+
+```shell
+example_etl
+├── README.md
+├── example_etl
+│ ├── __init__.py
+│ ├── assets
+│ ├── docker_image
+├── ml_project
+│ ├── example_ml
+│ ├── __init__.py
+│ ├── ml_assets
+├── random_assets.py
+├── example_etl_tests
+├── dagster_cloud.yaml
+├── pyproject.toml
+├── setup.cfg
+└── setup.py
+```
+
+If your repository contains multiple Dagster projects in subdirectories - otherwise known as a monorepository - add the `dagster_cloud.yaml` file to the root of where the Dagster projects are stored.
+
+## File structure
+
+Settings are formatted using YAML. For example, using the file structure above as an example:
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: data-eng-pipeline
+ code_source:
+ package_name: example_etl
+ build:
+ directory: ./example_etl
+ registry: localhost:5000/docker_image
+ - location_name: ml-pipeline
+ code_source:
+ package_name: example_ml
+ working_directory: ./ml_project
+ executable_path: venvs/path/to/ml_tensorflow/bin/python
+ - location_name: my_random_assets
+ code_source:
+ python_file: random_assets.py
+ container_context:
+ k8s:
+ env_vars:
+ - database_name
+ - database_username=hooli_testing
+ env_secrets:
+ - database_password
+```
+
+## Settings
+
+The `dagster_cloud.yaml` file contains a single top-level key, `locations`. This key accepts a list of code locations; for each code location, you can configure the following:
+
+- [Location name](#location-name)
+- [Code source](#code-source)
+- [Working directory](#working-directory)
+- [Build](#build)
+- [Python executable](#python-executable)
+- [Container context](#container-context)
+
+### Location name
+
+**This key is required.** The `location_name` key specifies the name of the code location. The location name will always be paired with a [code source](#code-source).
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: data-eng-pipeline
+ code_source:
+ package_name: example_etl
+```
+
+| Property | Description | Format |
+|-----------------|----------------------------------------------------------------------------------------|----------|
+| `location_name` | The name of your code location that will appear in the Dagster UI Code locations page. | `string` |
+
+### Code source
+
+**This section is required.** The `code_source` defines how a code location is sourced.
+
+A `code_source` key must contain either a `module_name`, `package_name`, or `file_name` parameter that specifies where to find the definitions in the code location.
+
+
+
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: data-eng-pipeline
+ code_source:
+ package_name: example_etl
+```
+
+
+
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: data-eng-pipeline
+ code_source:
+ package_name: example_etl
+ - location_name: machine_learning
+ code_source:
+ python_file: ml/ml_model.py
+```
+
+
+
+
+| Property | Description | Format |
+|----------------------------|-----------------------------------------------------------------------------------|--------------------------|
+| `code_source.package_name` | The name of a package containing Dagster code | `string` (folder name) |
+| `code_source.python_file` | The name of a Python file containing Dagster code (e.g. `analytics_pipeline.py` ) | `string` (.py file name) |
+| `code_source.module_name` | The name of a Python module containing Dagster code (e.g. `analytics_etl`) | `string` (module name) |
+
+### Working directory
+
+Use the `working_directory` setting to load Dagster code from a different directory than the root of your code repository. This setting allows you to specify the directory you want to load your code from.
+
+Consider the following project:
+
+```shell
+example_etl
+├── README.md
+├── project_directory
+│ ├── example_etl
+│ ├── __init__.py
+│ ├── assets
+│ ├── example_etl_tests
+├── dagster_cloud.yaml
+├── pyproject.toml
+├── setup.cfg
+└── setup.py
+```
+
+To load from `/project_directory`, the `dagster_cloud.yaml` code location would look like this:
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: data-eng-pipeline
+ code_source:
+ package_name: example_etl
+ working_directory: ./project_directory
+```
+
+| Property | Description | Format |
+|---------------------|-------------------------------------------------------------------------|-----------------|
+| `working_directory` | The path of the directory that Dagster should load the code source from | `string` (path) |
+
+### Build
+
+The `build` section contains two parameters:
+
+- `directory` - Setting a build directory is useful if your `setup.py` or `requirements.txt` is in a subdirectory instead of the project root. This is common if you have multiple Python modules within a single Dagster project.
+- `registry` - **Applicable only to Hybrid deployments.** Specifies the Docker registry to push the code location to.
+
+In the example below, the Docker image for the code location is in the root directory and the registry and image defined:
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: data-eng-pipeline
+ code_source:
+ package_name: example_etl
+ build:
+ directory: ./
+ registry: your-docker-image-registry/image-name # e.g. localhost:5000/myimage
+```
+
+
+| Property | Description | Format | Default |
+|-------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------------------------|---------|
+| `build.directory` | The path to the directory in your project that you want to deploy. If there are subdirectories, you can specify the path to only deploy a specific project directory. | `string` (path) | `.` |
+| `build.registry` | **Applicable to Hybrid deployments.** The Docker registry to push your code location to | `string` (docker registry) | |
+
+
+### Python executable
+
+For Dagster+ Hybrid deployments, the Python executable that is installed globally in the image, or the default Python executable on the local system if you use the local agent, will be used. To use a different Python executable, specify it using the `executable_path` setting. It can be useful to have different Python executables for different code locations.
+
+For Dagster+ Serverless deployments, you can specify a different Python version by [following these instructions](/dagster-plus/deployment/deployment-types/serverless/runtime-environment#use-a-different-python-version).
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: data-eng-pipeline
+ code_source:
+ package_name: example_etl
+ executable_path: venvs/path/to/dataengineering_spark_team/bin/python
+ - location_name: machine_learning
+ code_source:
+ python_file: ml_model.py
+ executable_path: venvs/path/to/ml_tensorflow/bin/python
+```
+
+| Property | Description | Format |
+|-------------------|-----------------------------------------------|-----------------|
+| `executable_path` | The file path of the Python executable to use | `string` (path) |
+
+### Container context
+
+If using Hybrid deployment, you can define additional configuration options for code locations using the `container_context` parameter. Depending on the Hybrid agent you're using, the configuration settings under `container_context` will vary.
+
+Refer to the configuration reference for your agent for more info:
+
+- [Docker agent configuration reference](/dagster-plus/deployment/deployment-types/hybrid/docker/configuration)
+- [Amazon ECS agent configuration reference](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/configuration-reference)
+- [Kubernetes agent configuration reference](/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-locations/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-locations/index.md
new file mode 100644
index 0000000000000..a7bf70acf8f12
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-locations/index.md
@@ -0,0 +1,237 @@
+---
+title: "Code locations"
+sidebar_position: 40
+---
+
+Separate code locations allow you to deploy different projects that still roll up into a single Dagster+ deployment with one global lineage graph.
+
+This guide will cover three options for adding a new code location:
+- Adding a code location manually
+- Adding a code location in a new Git repository
+- Adding a new code location to an existing Git monorepo
+
+
+Prerequisites
+
+1. An existing Dagster project. Refer to the [recommended project structure](/guides/build/projects/structuring-your-dagster-project) and [code requirements](/dagster-plus/deployment/code-requirements) pages for more information.
+
+2. Editor, Admin, or Organization Admin permissions in Dagster+.
+
+3. To know your Dagster+ deployment type. An administrator can help find this information, or you can locate it by clicking *Deployment > Agents tab* in the Dagster UI. Dagster+ Serverless organizations will have a *Managed by Dagster+* label next to the agent.
+
+
+
+Adding a code location follows two steps:
+- For Dagster+ Hybrid, ensuring the Dagster code is in a place accessible by your agent, usually by building a Docker image with your code that's pushed to a registry. For Dagster+ Serverless you can skip this step.
+- Notifying Dagster+ of the new or updated code location. This will be done by using the Dagster+ Python client.
+
+Often these two steps are handled by CI/CD connected to your Git repository.
+
+
+## Add a new code location manually
+
+Start by installing the `dagster-cloud` Python client:
+
+```
+pip install dagster-cloud
+```
+
+Next you will want need to authenticate this Python client:
+
+1. In the Dagster+ UI, click the user icon in the upper right corner.
+2. Click **Organization settings**, then the **Tokens** tab.
+3. Click the **Create user token** button.
+4. Copy the token.
+5. Set the following environment variables:
+
+ ```bash
+ export DAGSTER_CLOUD_ORGANIZATION="organization-name" # if your URL is https://acme.dagster.plus your organization name is "acme"
+ export DAGSTER_CLOUD_API_TOKEN="your-token"
+ ```
+
+Now add the code location. The following example assumes you are running the command from the top-level working directory of your Dagster project with a project named "quickstart" structured as a Python module named "quickstart".
+
+```bash
+/quickstart
+ setup.py
+ pyproject.toml
+ /quickstart
+ __init__.py
+ /assets
+ /resources
+```
+
+The commands below take two main arguments:
+- `module_name` is determined by your code structure
+- `location_name` is the unique label for this code location used in Dagster+
+
+
+
+
+If you are using Dagster+ Serverless, run the following command to add a code location:
+
+```bash
+dagster-cloud serverless deploy-python-executable --deployment prod --location-name quickstart --module-name quickstart
+```
+
+Running the command multiple times with the same location name will *update* the code location. Running the command with a new location name will *add* a code location.
+
+
+
+If you are using Dagster+ Hybrid, make sure you have deployed the code appropriately by [building a Docker image and pushing it to an image registry](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart). Then run this command, using the image URI which is available from your registry:
+
+```bash
+dagster-cloud deployment add-location --deployment prod --location-name quickstart --module-name quickstart --image 764506304434.dkr.ecr.us-west-2.amazonaws.com/hooli-data-science-prod:latest
+```
+
+
+
+After running the command you can verify the code location was deployed by navigating to the *Deployments* tab on Dagster+.
+
+## Adding a code location in a new Git repository
+
+Adding a code location to a Git repository follows the same steps as adding a code location manually, but automates those steps by running them through CI/CD.
+
+To get started, review the appropriate example repository and then create your Git repository with the same structure.
+
+- [GitHub repository with Dagster+ Serverless](https://github.com/dagster-io/dagster-cloud-serverless-quickstart/)
+- [GitHub repository with Dagster+ Hybrid](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/)
+- [GitLab CI/CD for Dagster+ Serverless](https://github.com/dagster-io/dagster-cloud-action/blob/main/gitlab/serverless-ci.yml)
+- [GitLab CI/CD for Dagster+ Hybrid](https://github.com/dagster-io/dagster-cloud-action/blob/main/gitlab/hybrid-ci.yml)
+
+
+Overall, the Git repository should contain:
+
+1. Your Dagster code, structured as a Python module. For Dagter+ Hybrid you may need a [Dockerfile](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/blob/main/Dockerfile) as well. The repository might look like this:
+
+ ```bash
+ README.md
+ dagster_cloud.yaml
+ Dockerfile
+ /.github
+ /workflows
+ dagster-cloud-deploy.yml
+ setup.py
+ pyproject.toml
+ /quickstart
+ __init__.py
+ definitions.py
+ /assets
+ ...
+ /resources
+ ...
+ ```
+
+2. A [`dagster_cloud.yaml` file](/dagster-plus/deployment/code-locations/dagster-cloud-yaml) with the settings for your code location. Here is an example:
+
+ ```yaml title="dagster_cloud.yaml
+ locations:
+ - location_name: quickstart
+ code_source:
+ package_name: quickstart
+ ```
+
+3. A CI/CD workflow file that contains the steps for adding your code location. These are the same steps outlined in the preceding section. Here is a minimal example workflow file for a Dagster+ Hybrid organization based on [this GitLab template](https://github.com/dagster-io/dagster-cloud-action/blob/main/gitlab/hybrid-ci.yml).
+
+ ```yaml
+ variables:
+ DAGSTER_CLOUD_ORGANIZATION:
+ DAGSTER_PROJECT_DIR: .
+ IMAGE_REGISTRY: .dkr.ecr.us-west-2.amazonaws.com/
+ IMAGE_TAG: $CI_COMMIT_SHORT_SHA-$CI_PIPELINE_ID
+
+ stages:
+ - build
+ - deploy
+
+ build:
+ stage: build
+ image: docker:latest
+ services:
+ - docker:dind
+ before_script:
+ # # For Gitlab Container Registry
+ # - echo $CI_JOB_TOKEN | docker login --username $CI_REGISTRY_USER --password-stdin $REGISTRY_URL
+ # # For DockerHub
+ # - echo $DOCKERHUB_TOKEN | docker login --username $DOCKERHUB_USERNAME --password-stdin $REGISTRY_URL
+ # # For AWS Elastic Container Registry (ECR)
+ # - apk add --no-cache curl jq python3 py3-pip
+ # - pip install awscli
+ # - echo $AWS_ECR_PASSWORD | docker login --username AWS --password-stdin $IMAGE_REGISTRY
+ # # For Google Container Registry (GCR)
+ # - echo $GCR_JSON_KEY | docker login --username _json_key --password-stdin $REGISTRY_URL
+ script:
+ - docker build . -t $IMAGE_REGISTRY:$IMAGE_TAG
+ - docker push $IMAGE_REGISTRY:$IMAGE_TAG
+
+ deploy:
+ stage: deploy
+ dependencies:
+ - build
+ image: ghcr.io/dagster-io/dagster-cloud-action:0.1.43
+ script:
+ - dagster-cloud deployment add-location --deployment prod --image
+ $IMAGE_REGISTRY:$IMAGE_TAG --location-name quickstart --package-name quickstart
+ ```
+
+Once your Git repository has this structure, you will want to run your CI/CD process. The CI/CD process will add the code location to Dagster+ which can be verified by viewing the *Deployments* tab.
+
+## Adding a new code location to a Git monorepo
+
+Many organizations use a Git monorepo to contain multiple Dagster projects. Here is an example of DagsterLab's own [internal data engineering Git repository](https://github.com/dagster-io/dagster-open-platform).
+
+To add a new code location to a monorepo, create a new directory that contains your Dagster project. The final repository structure might look like this:
+
+```
+README.md
+dagster_cloud.yaml
+/.github
+ ...
+/shared
+ setup.py
+ pyproject.toml
+ /shared
+ __init__.py
+ utilities.py
+/core
+ setup.py
+ pyproject.toml
+ Dockerfile
+ /core
+ definitions.py
+ __init__.py
+ /assets
+/new-code-location
+ setup.py
+ pyproject.toml
+ Dockerfile
+ /new-code-location
+ definitions.py
+ __init__.py
+ /assets
+```
+
+Then update the `dagster_cloud.yaml` file in the root of the Git repository, adding a location section for your project including the location name, code source, and build directory. For Dagster+ Hybrid, include the registry. If you don't know the registry, consult your administrator or the team that set up CI/CD for the Git repository.
+
+```yaml
+locations:
+ - location_name: core
+ code_source:
+ package_name: core
+ build:
+ directory: ./core
+ registry: your-registry/image # eg 764506304434.dkr.ecr.us-west-2.amazonaws.com/core
+ - location_name: new
+ code_source:
+ package_name: new
+ build:
+ directory: ./new
+ registry: your-registry/image # eg 764506304434.dkr.ecr.us-west-2.amazonaws.com/new
+```
+
+The monorepo should have CI/CD configured to deploy your changes and add or update your new code location. After adding your code and updating the `dagster_cloud.yaml` file, trigger the CI/CD process to add your code location to Dagster+. Navigate to the *Deployments* tab in Dagster+ to confirm your code location was added.
+
+## Next steps
+
+- After adding a code location, you may want to setup access controls
+- You may want to add additional configuration to your code location. This configuration will vary by agent type, but see examples for [setting default resource limits for Kubernetes](/dagster-plus/deployment/deployment-types/hybrid/kubernetes) or [changing the IAM role for ECS](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/configuration-reference).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-requirements.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-requirements.md
new file mode 100644
index 0000000000000..a20009079b20b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/code-requirements.md
@@ -0,0 +1,45 @@
+---
+title: 'Dagster+ code requirements'
+sidebar_label: "Code requirements"
+sidebar_position: 10
+---
+
+Your Dagster project must meet a few requirements to run in Dagster+.
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- A basic understanding of Python project structure and Docker
+
+
+## General requirements
+
+:::tip
+**Learn by example?** Check out [an example repo](https://github.com/dagster-io/hooli-data-eng-pipelines) which is set up to run in Dagster+.
+:::
+
+To work with Dagster+, your Dagster code:
+
+- **Must be loaded from a single entry point: either a Python file or package.** This entry point can load repositories from other files or packages.
+
+- **Must run in an environment where the `dagster` and `dagster-cloud` 0.13.2 or later Python packages are installed.**
+
+**Note**:
+
+- Different code locations can use different versions of Dagster
+- Dagster+ doesn't require a [`workspace.yaml` file](/guides/deploy/code-locations/workspace-yaml). You can still create a `workspace.yaml` file to load your code in an open source Dagster webserver instance, but doing so won't affect how your code is loaded in Dagster+.
+
+## Hybrid deployment requirements
+
+If you're using [Hybrid Deployment](/dagster-plus/deployment/deployment-types/hybrid), there are a few additional requirements.
+
+- **If using an Amazon Elastic Container Service (ECS), Kubernetes, or Docker agent**, your code must be packaged into a Docker image and pushed to a registry your agent can access. Dagster+ doesn't need access to your image - the agent only needs to be able to pull it.
+
+ Additionally, the Dockerfile for your image doesn't need to specify an entry point or command. These will be supplied by the agent when it runs your code using the supplied image.
+
+- **If using a local agent**, your code must be in a Python environment that can be accessed on the same machine as your agent.
+
+Additionally, your code doesn't need to use the same version of Dagster as your agent.
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/configuration-reference.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/configuration-reference.md
new file mode 100644
index 0000000000000..13aff85854f9a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/configuration-reference.md
@@ -0,0 +1,258 @@
+---
+title: Configuration reference
+sidebar_position: 400
+---
+
+:::note
+
+This guide is applicable to Dagster+.
+
+:::
+
+This reference describes the various configuration options Dagster+ currently supports for [Amazon ECS agents](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs).
+
+## Per-location configuration
+
+When [adding a code location](/dagster-plus/deployment/code-locations/) to Dagster+ with an Amazon ECS agent, you can use the `container_context` key on the location configuration to add additional ECS-specific configuration that will be applied to any ECS tasks associated with that code location.
+
+**Note**: If you're using the Dagster+ Github action, the `container_context` key can also be set for each location in your `dagster_cloud.yaml` file.
+
+The following example [`dagster_cloud.yaml`](/dagster-plus/deployment/code-locations/dagster-cloud-yaml) file illustrates the available fields:
+
+```yaml
+locations:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ ecs:
+ env_vars:
+ - DATABASE_NAME=staging
+ - DATABASE_PASSWORD
+ secrets:
+ - name: "MY_API_TOKEN"
+ valueFrom: "arn:aws:secretsmanager:us-east-1:123456789012:secret:FOO-AbCdEf:token::"
+ - name: "MY_PASSWORD"
+ valueFrom: "arn:aws:secretsmanager:us-east-1:123456789012:secret:FOO-AbCdEf:password::"
+ secrets_tags:
+ - "my_tag_name"
+ server_resources: # Resources for code servers launched by the agent for this location
+ cpu: 256
+ memory: 512
+ replica_count: 1
+ run_resources: # Resources for runs launched by the agent for this location
+ cpu: 4096
+ memory: 16384
+ execution_role_arn: arn:aws:iam::123456789012:role/MyECSExecutionRole
+ task_role_arn: arn:aws:iam::123456789012:role/MyECSTaskRole
+ mount_points:
+ - sourceVolume: myEfsVolume
+ containerPath: "/mount/efs"
+ readOnly: True
+ volumes:
+ - name: myEfsVolume
+ efsVolumeConfiguration:
+ fileSystemId: fs-1234
+ rootDirectory: /path/to/my/data
+ server_sidecar_containers:
+ - name: DatadogAgent
+ image: public.ecr.aws/datadog/agent:latest
+ environment:
+ - name: ECS_FARGATE
+ value: true
+ run_sidecar_containers:
+ - name: DatadogAgent
+ image: public.ecr.aws/datadog/agent:latest
+ environment:
+ - name: ECS_FARGATE
+ value: true
+ server_ecs_tags:
+ - key: MyEcsTagKey
+ value: MyEcsTagValue
+ run_ecs_tags:
+ - key: MyEcsTagKeyWithoutValue
+ repository_credentials: MyRepositoryCredentialsSecretArn
+```
+
+### Environment variables and secrets
+
+Using the `container_context.ecs.env_vars` and `container_context.ecs.secrets` properties, you can configure environment variables and secrets for a specific code location.
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ ecs:
+ env_vars:
+ - DATABASE_NAME=testing
+ - DATABASE_PASSWORD
+ secrets:
+ - name: "MY_API_TOKEN"
+ valueFrom: "arn:aws:secretsmanager:us-east-1:123456789012:secret:FOO-AbCdEf:token::"
+ - name: "MY_PASSWORD"
+ valueFrom: "arn:aws:secretsmanager:us-east-1:123456789012:secret:FOO-AbCdEf:password::"
+ secrets_tags:
+ - "my_tag_name"
+```
+
+| Property | Description |
+|----------|-------------|
+| container_context.ecs.env_vars | A list of keys or key-value pairs for task inclusion. If value unspecified, pulls from agent task. Example: `FOO_ENV_VAR` set to `foo_value`, `BAR_ENV_VAR` set to agent task value. |
+| container_context.ecs.secrets | Individual secrets specified using the [same structure as the ECS API.](https://docs.aws.amazon.com/AmazonECS/latest/APIReference/API_Secret.html) |
+| container_context.ecs.secrets_tags | A list of tag names. Each secret tagged with any of those tag names in AWS Secrets Manager will be included in the launched tasks as environment variables. The name of the environment variable will be the name of the secret, and the value of the environment variable will be the value of the secret. |
+
+
+Refer to the following guides for more info about environment variables:
+
+- [Dagster+ environment variables and secrets](/dagster-plus/deployment/management/environment-variables/)
+- [Using environment variables and secrets in Dagster code](/guides/deploy/using-environment-variables-and-secrets)
+
+## Per-job configuration: Resource limits
+
+You can use job tags to customize the CPU and memory of every run for that job:
+
+```python
+from dagster import job, op
+
+@op()
+def my_op(context):
+ context.log.info('running')
+
+@job(
+ tags = {
+ "ecs/cpu": "256",
+ "ecs/memory": "512",
+ }
+)
+def my_job():
+ my_op()
+```
+
+[Fargate tasks only support certain combinations of CPU and memory.](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-cpu-memory-error.html)
+
+If the `ecs/cpu` or `ecs/memory` tags are set, they will override any defaults set on the code location or the deployment.
+
+## Per-deployment configuration
+
+This section describes the properties of the `dagster.yaml` configuration file used by Amazon ECS agents. Typically, this file is created by the CloudFormation template that deploys the agent and can be found within the agent task definition's command.
+
+To change these properties, edit the CloudFormation template and redeploy the CloudFormation stack.
+
+```yaml
+instance_class:
+ module: dagster_cloud
+ class: DagsterCloudAgentInstance
+
+dagster_cloud_api:
+ agent_token:
+ deployments:
+ -
+ -
+ branch_deployments:
+
+user_code_launcher:
+ module: dagster_cloud.workspace.ecs
+ class: EcsUserCodeLauncher
+ config:
+ cluster:
+ subnets:
+ -
+ -
+ security_group_ids:
+ -
+ service_discovery_namespace_id:
+ execution_role_arn:
+ task_role_arn:
+ log_group:
+ launch_type: <"FARGATE"|"EC2">
+ server_process_startup_timeout:
+ server_resources:
+ cpu:
+ memory:
+ server_sidecar_containers:
+ - name: SidecarName
+ image: SidecarImage
+
+ run_resources:
+ cpu:
+ memory:
+ run_sidecar_containers:
+ - name: SidecarName
+ image: SidecarImage
+
+ mount_points:
+ -
+ volumes:
+ -
+ server_ecs_tags:
+ - key: MyEcsTagKey
+ value: MyEcsTagValue
+ run_ecs_tags:
+ - key: MyEcsTagKeyWithoutValue
+ repository_credentials: MyRepositoryCredentialsSecretArn
+
+isolated_agents:
+ enabled:
+agent_queues:
+ include_default_queue:
+ additional_queues:
+ -
+ -
+```
+
+### dagster_cloud_api properties
+
+| Property | Description |
+|----------|-------------|
+| dagster_cloud_api.agent_token | An agent token for the agent to use for authentication. |
+| dagster_cloud_api.deployments | The names of full deployments for the agent to serve. |
+| dagster_cloud_api.branch_deployments | Whether the agent should serve all branch deployments. |
+
+
+### user_code_launcher properties
+
+| Property | Description |
+|----------|-------------|
+| config.cluster | Name of ECS cluster with Fargate or EC2 capacity provider |
+| config.launch_type | ECS launch type: • `FARGATE` • `EC2` **Note:** Using this launch type requires you to have an EC2 capacity provider installed and additional operational overhead to run the agent.|
+| config.subnets | **At least one subnet is required**. Dagster+ tasks require a route to the internet so they can access our API server. How this requirement is satisfied depends on the type of subnet provided: • **Public subnets** - The ECS agent will assign each task a public IP address. Note that [ECS tasks on EC2](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-networking-awsvpc.html) launched within public subnets do not have access to the internet, so a public subnet will only work for Fargate tasks. •**Private subnets** - The ECS agent assumes you've configured a NAT gateway with an attached NAT gateway. Tasks will **not** be assigned a public IP address. |
+| config.security_group_ids | A list of [security groups](https://docs.aws.amazon.com/AWSCloudFormation/latest/UserGuide/aws-resource-ec2-securitygroup.html) to use for tasks launched by the agent. |
+| config.service_discovery_namespace_id | The name of a [private DNS namespace](https://docs.aws.amazon.com/AWSCloudFormation/latest/UserGuide/aws-resource-servicediscovery-privatednsnamespace.html). The ECS agent launches each code location as its own ECS service. The agent communicates with these services via [AWS CloudMap service discovery.](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/service-discovery.html) |
+| config.execution_role_arn | The ARN of the [Amazon ECS task execution IAM role](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task_execution_IAM_role.html). This role allows ECS to interact with AWS resources on your behalf, such as getting an image from ECR or pushing logs to CloudWatch. **Note**:This role must include a trust relationship that allows ECS to use it.|
+| config.task_role_arn | The ARN of the [Amazon ECS task IAM role](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-iam-roles.html). This role allows the containers running in the ECS task to interact with AWS. **Note**: This role must include a trust relationship that allows ECS to use it. |
+| config.log_group | The name of a CloudWatch [log group](https://docs.aws.amazon.com/AmazonCloudWatch/latest/logs/Working-with-log-groups-and-streams.html#Create-Log-Group). |
+| config.server_process_startup_timeout | The amount of time, in seconds, to wait for code to import when launching a new service for a code location. If your code takes an unusually long time to load after your ECS task starts up and results in timeouts in the **Deployment** tab, you can increase this setting above the default. **Note** This setting isn't applicable to the time it takes for a job to execute. • **Default** - 180 (seconds)|
+| config.ecs_timeout | How long (in seconds) to wait for ECS to spin up a new service and task for a code server. If your ECS tasks take an unusually long time to start and result in timeouts, you can increase this setting above the default. • **Default** - 300 (seconds) |
+| config.ecs_grace_period | How long (in seconds) to continue polling if an ECS API endpoint fails during creation of a new code server (because the ECS API is eventually consistent). • **Default** - 30 (seconds)|
+| config.server_resources | The resources that the agent should allocate to the ECS service for each code location that it creates. If set, must be a dictionary with a `cpu` and/or `memory` key. **Note**: [Fargate tasks only support certain combinations of CPU and memory.](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-cpu-memory-error.html) |
+| config.server_sidecar_containers | Additional sidecar containers to include along with the Dagster container. If set, must be a list of dictionaries with valid ECS container definitions. |
+| config.run_resources | The resources that the agent should allocate to the ECS task that it creates for each run. If set, must be a dictionary with a `cpu` and/or `memory` key. **Note**: [Fargate tasks only support certain combinations of CPU and memory.](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-cpu-memory-error.html) |
+| config.run_sidecar_containers | Additional sidecar containers to include along with the Dagster container. If set, must be a list of dictionaries with valid ECS container definitions. |
+| config.mount_points | Mount points to include in the Dagster container. If set, should be a list of dictionaries matching the `mountPoints` field when specifying a container definition to boto3. |
+| config.volumes | Additional volumes to include in the task definition. If set, should be a list of dictionaries matching the volumes argument to `register_task_definition` in boto3. |
+| config.server_ecs_tags | Additional ECS tags to include in the service for each code location. If set, must be a list of dictionaries, each with a `key` key and optional `value` key. |
+| config.run_ecs_tags | AAdditional ECS tags to include in the task for each run. If set, must be a list of dictionaries, each with a `key` key and optional `value` key. |
+| config.repository_credentials | Optional arn of the secret to authenticate into your private container registry. This does not apply if you are leveraging ECR for your images, see the [AWS private auth guide.](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/private-auth.html) |
+| config.enable_ecs_exec | Boolean that determines whether tasks created by the agent should be configured with the needed linuxParameters and permissions to use [ECS Exec](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/ecs-exec.html) to shell into the task. Also grants the `SYS_PTRACE` linux capability to enable running tools like py-spy to debug slow or hanging tasks. Defaults to false. **Note**: For ECS Exec to work, the task IAM role must be granted [certain permissions](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-iam-roles.html#ecs-exec-required-iam-permissions). |
+
+
+### isolated_agents properties
+
+| Property | Description |
+|----------|-------------|
+| isolated_agents.enabled | When enabled, agents are isolated and will not be able to access each others' resources. See the [Running multiple agents guide](/dagster-plus/deployment/deployment-types/hybrid/multiple) for more information. |
+
+### agent_queues properties
+
+These settings specify the queue(s) the agent will obtain requests from. See [Routing requests to specific agents](/dagster-plus/deployment/deployment-types/hybrid/multiple).
+
+| Property | Description |
+|----------|-------------|
+| agent_queues.include_default_queue | If true, agent processes requests from default queue |
+| agent_queues.additional_queues | List of additional queues for agent processing |
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/existing-vpc.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/existing-vpc.md
new file mode 100644
index 0000000000000..870bcc1ac186c
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/existing-vpc.md
@@ -0,0 +1,80 @@
+---
+title: Existing VPC setup
+sidebar_position: 200
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+In this guide, you'll set up and deploy an Amazon Elastic Container Service (ECS) agent in an existing VPC using CloudFormation. Amazon ECS agents are used to launch user code in ECS tasks.
+
+Our CloudFormation template allows you to quickly spin up the ECS agent stack in an existing VPC. It also supports using a new or existing ECS cluster. The template code can be found [here](https://s3.amazonaws.com/dagster.cloud/cloudformation/ecs-agent.yaml). Refer to the [CloudFormation docs](https://docs.aws.amazon.com/AWSCloudFormation/latest/UserGuide/Welcome.html) for more info about CloudFormation.
+
+**For info about deploying an ECS agent in a new VPC**, check out the [ECS agents in new VPCs guide](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/new-vpc).
+
+## Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- **In Dagster+**:
+
+ - **Your organization and deployment names.**
+ - **Permissions in Dagster+ that allow you to manage agent tokens**. Refer to the [User permissions documentation](/dagster-plus/features/authentication-and-access-control/rbac/users) for more info.
+
+- **In Amazon Web Services (AWS)**:
+ - **An existing VPC with the following:**
+ - **Subnets with access to the public internet**. Refer to the [AWS Work with VPCs guide](https://docs.aws.amazon.com/vpc/latest/userguide/working-with-vpcs.html) for more info.
+ - **Enabled `enableDnsHostnames` and `enableDnsSupport` DNS attributes**. Refer to the [AWS DNS attributes documentation](https://docs.aws.amazon.com/vpc/latest/userguide/vpc-dns.html#vpc-dns-support) for more info.
+ - **Optional**: An existing ECS cluster with a [Fargate or EC2 capacity provider](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/cluster-capacity-providers.html). The CloudFormation template will create a cluster for you if one isn't specified.
+
+## Step 1: Generate a Dagster+ agent token
+
+In this step, you'll generate a token for the Dagster+ agent. The Dagster+ agent will use this to authenticate to the agent API.
+
+1. Sign in to your Dagster+ instance.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. In the **Organization Settings** page, click the **Tokens** tab.
+4. Click the **+ Create agent token** button.
+5. After the token has been created, click **Reveal token**.
+
+Keep the token somewhere handy - you'll need it to complete the setup.
+
+## Step 2: Install the CloudFormation stack in AWS
+
+Click the **Launch Stack** button to install the CloudFormation stack in your AWS account:
+
+[](https://console.aws.amazon.com/cloudformation/home#/stacks/create/review?templateURL=https://s3.amazonaws.com/dagster.cloud/cloudformation/ecs-agent.yaml)
+
+**Note**: Creating the CloudFormation stack may take a few minutes. Refresh the [AWS console **Stacks** page](https://console.aws.amazon.com/cloudformation/home#/stacks) to check the status.
+
+## Step 3: Configure the agent
+
+After the stack is installed, you'll be prompted to configure it. In the ECS wizard, fill in the following fields:
+
+- **Dagster+ Organization**: Enter the name of your Dagster+ organization.
+- **Dagster+ Deployment**: Enter the name of the Dagster+ deployment you want to use. Leave this field empty if the agent will only serve Branch deployments.
+- **Enable Branch Deployments**: Whether to have this agent serve your ephemeral [Branch deployments](/dagster-plus/features/ci-cd/branch-deployments). Only a single agent should have this setting enabled.
+- **Agent Token**: Paste the agent token you generated in [Step 1](#step-1-generate-a-dagster-agent-token).
+- **Deploy VPC**: The existing VPC to deploy the agent into.
+- **Deploy VPC Subnet**: A public subnet of the existing VPC to deploy the agent into.
+- **Existing ECS Cluster**: Optionally, the name of an existing ECS cluster to deploy the agent in. Leave blank to create a new cluster
+- **Task Launch Type**: Optionally, the launch type to use for new tasks created by the agent (FARGATE or EC2). Defaults to FARGATE.
+
+The page should look similar to the following image. In this example, our organization name is `hooli` and our deployment is `prod`:
+
+![Example Configuration for the ECS Agent CloudFormation Template](/images/dagster-plus/deployment/agents/aws-ecs-stack-wizard-existing.png)
+
+After you've finished configuring the stack in AWS, you can view the agent in Dagster+. To do so, navigate to the **Status** page and click the **Agents** tab. You should see the agent running in the **Agent statuses** section:
+
+![Instance Status](/images/dagster-plus/deployment/agents/dagster-cloud-instance-status.png)
+
+## Next steps
+
+Now that you've got your agent running, what's next?
+
+- **If you're getting Dagster+ set up**, the next step is to [add a code location](/dagster-plus/deployment/code-locations) using the agent.
+
+- **If you're ready to load your Dagster code**, refer to the [Adding Code to Dagster+](/dagster-plus/deployment/code-locations) guide for more info.
+
+If you need to upgrade your ECS agent's CloudFormation template, refer to the [upgrade guide](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/upgrading-cloudformation) for more info.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/index.md
new file mode 100644
index 0000000000000..0ab0a059c490b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/index.md
@@ -0,0 +1,8 @@
+---
+title: Amazon ECS agent
+sidebar_position: 30
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/manual-provision.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/manual-provision.md
new file mode 100644
index 0000000000000..02108e5253ace
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/manual-provision.md
@@ -0,0 +1,135 @@
+---
+title: Manual provision setup
+sidebar_position: 300
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+In this guide, you'll manually set up and deploy an Amazon Elastic Container Service (ECS) agent. Amazon ECS agents are used to launch user code in ECS tasks.
+
+This method of setting up an Amazon ECS agent is a good option if you're comfortable with infrastructure management and want to fully define your agent.
+
+## Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- **In Dagster+**:
+
+ - **Your organization and deployment names.**
+ - **Permissions in Dagster+ that allow you to manage agent tokens**. Refer to the [User permissions documentation](/dagster-plus/features/authentication-and-access-control/rbac/users) for more info.
+
+- **Permissions in Amazon Web Services (AWS) that allow you to:**
+
+ - Create and configure ECS services.
+ - Create and configure IAM roles.
+
+- **Familiarity with infrastructure management and tooling.**
+
+## Step 1: Generate a Dagster+ agent token
+
+In this step, you'll generate a token for the Dagster+ agent. The Dagster+ agent will use this to authenticate to the agent API.
+
+1. Sign in to your Dagster+ instance.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. In the **Organization Settings** page, click the **Tokens** tab.
+4. Click the **+ Create agent token** button.
+5. After the token has been created, click **Reveal token**.
+
+Keep the token somewhere handy - you'll need it to complete the setup.
+
+## Step 2: Create ECS IAM roles
+
+To successfully run your ECS agent, you'll need to have the following IAM roles in your AWS account:
+
+- [**Task execution IAM role**](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task_execution_IAM_role.html) - This role allows ECS to interact with AWS resources on your behalf, such as pulling an image from ECR or pushing logs to CloudWatch.
+
+ Amazon publishes a managed policy called `AmazonECSTaskExecutionRolePolicy` with the required permissions. Refer to the [AWS docs](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task_execution_IAM_role.html#create-task-execution-role) for more info about creating this role.
+
+- [**Task IAM role**](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-iam-roles.html) - This role allows the containers running in the ECS task to interact with AWS.
+
+ When creating this role, include the permissions required to describe and launch ECS tasks. For example:
+
+ ```json
+ {
+ "Version": "2012-10-17",
+ "Statement": [
+ {
+ "Sid": "",
+ "Effect": "Allow",
+ "Action": [
+ "ec2:DescribeNetworkInterfaces",
+ "ec2:DescribeRouteTables",
+ "ecs:CreateService",
+ "ecs:DeleteService",
+ "ecs:DescribeServices",
+ "ecs:DescribeTaskDefinition",
+ "ecs:DescribeTasks",
+ "ecs:ListAccountSettings",
+ "ecs:ListServices",
+ "ecs:ListTagsForResource",
+ "ecs:ListTasks",
+ "ecs:RegisterTaskDefinition",
+ "ecs:RunTask",
+ "ecs:StopTask",
+ "ecs:TagResource",
+ "ecs:UpdateService",
+ "iam:PassRole",
+ "logs:GetLogEvents",
+ "secretsmanager:DescribeSecret",
+ "secretsmanager:GetSecretValue",
+ "secretsmanager:ListSecrets",
+ "servicediscovery:CreateService",
+ "servicediscovery:DeleteService",
+ "servicediscovery:ListServices",
+ "servicediscovery:GetNamespace",
+ "servicediscovery:ListTagsForResource",
+ "servicediscovery:TagResource"
+ ],
+ "Resource": "*"
+ }
+ ]
+ }
+ ```
+
+ You can also include any additional permissions required to run your ops, such as permissions to interact with an S3 bucket.
+
+**Note**: Both roles must include a trust relationship that allows ECS to use them:
+
+```json
+{
+ "Version": "2012-10-17",
+ "Statement": [
+ {
+ "Sid": "",
+ "Effect": "Allow",
+ "Principal": {
+ "Service": "ecs-tasks.amazonaws.com"
+ },
+ "Action": "sts:AssumeRole"
+ }
+ ]
+}
+```
+
+## Step 3: Create an ECS service
+
+1. Create an ECS service to run the agent. You can do this [in the Amazon ECS console](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/create-service-console-v2.html) or [via the CreateService API](https://docs.aws.amazon.com/AmazonECS/latest/APIReference/API_CreateService.html).
+
+ Use the [official dagster/dagster-cloud-agent image](https://hub.docker.com/r/dagster/dagster-cloud-agent) as the service's **Task definition**. This image can be used as-is or as a base layer for your own image.
+
+2. Add a configured `dagster.yaml` file to your container. You can do this by:
+
+ - Building it into your image
+ - Echoing it to a file in your task definition's command **before starting the agent**
+
+ Refer to the [ECS configuration reference](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/configuration-reference#per-deployment-configuration) for more info about the required fields.
+
+## Next steps
+
+Now that you've got your agent running, what's next?
+
+- **If you're getting Dagster+ set up**, the next step is to [add a code location](/dagster-plus/deployment/code-locations) using the agent.
+
+- **If you're ready to load your Dagster code**, refer to the [Adding Code to Dagster+](/dagster-plus/deployment/code-locations) guide for more info.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/new-vpc.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/new-vpc.md
new file mode 100644
index 0000000000000..0a0adee951094
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/new-vpc.md
@@ -0,0 +1,85 @@
+---
+title: New VPC setup
+sidebar_position: 100
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+In this guide, you'll set up and deploy an Amazon Elastic Container Service (ECS) agent in a new VPC using CloudFormation. Amazon ECS agents are used to launch user code in ECS tasks.
+
+Our CloudFormation template allows you to quickly spin up the ECS agent stack. This template sets up an ECS agent from scratch, creating a new VPC and ECS cluster for the agent to run in. The template code can be found [here](https://s3.amazonaws.com/dagster.cloud/cloudformation/ecs-agent-vpc.yaml). Refer to the [CloudFormation docs](https://docs.aws.amazon.com/AWSCloudFormation/latest/UserGuide/Welcome.html) for more info about CloudFormation.
+
+**For info about deploying an ECS agent in an existing VPC**, check out the [ECS agents in existing VPCs guide](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/existing-vpc).
+
+## Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- **In Dagster+**:
+
+ - **Your organization and deployment names.**
+ - **Permissions in Dagster+ that allow you to manage agent tokens**. Refer to the [User permissions documentation](/dagster-plus/features/authentication-and-access-control/rbac/users) for more info.
+
+- **In Amazon Web Services (AWS), you'll need an account**:
+
+ - **Under its VPC quota limit in the region where you're spinning up the agent.** By default, AWS allows **five VPCs per region**. If you're already at your limit, refer to the [AWS VPC quotas documentation](https://docs.aws.amazon.com/vpc/latest/userguide/amazon-vpc-limits.html) for info on requesting a quota increase.
+
+ - **With an ECS service-linked IAM role**. This role is required to complete the setup in ECS. AWS will automatically create the role in your account the first time you create an ECS cluster in the console. However, the IAM role isn't automatically created when ECS clusters are created via CloudFormation.
+
+ If your account doesn't have this IAM role, running the CloudFormation template may fail.
+
+ If you haven't created an ECS cluster before, complete one of the following before proceeding:
+
+ - Create one using the [first run wizard](https://console.aws.amazon.com/ecs/home#/firstRun), or
+ - Create the IAM role using the [AWS CLI](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/using-service-linked-roles.html#create-service-linked-role)
+
+## Step 1: Generate a Dagster+ agent token
+
+In this step, you'll generate a token for the Dagster+ agent. The Dagster+ agent will use this to authenticate to the agent API.
+
+1. Sign in to your Dagster+ instance.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. In the **Organization Settings** page, click the **Tokens** tab.
+4. Click the **+ Create agent token** button.
+5. After the token has been created, click **Reveal token**.
+
+Keep the token somewhere handy - you'll need it to complete the setup.
+
+## Step 2: Install the CloudFormation stack in AWS
+
+Click the **Launch Stack** button to install the CloudFormation stack in your AWS account:
+
+[](https://console.aws.amazon.com/cloudformation/home#/stacks/create/review?templateURL=https://s3.amazonaws.com/dagster.cloud/cloudformation/ecs-agent-vpc.yaml)
+
+**Note**: Creating the CloudFormation stack may take a few minutes. Refresh the [AWS console **Stacks** page](https://console.aws.amazon.com/cloudformation/home#/stacks) to check the status.
+
+If the installation fails, verify that your AWS account [meets the requirements listed above](#prerequisites).
+
+## Step 3: Configure the agent
+
+After the stack is installed, you'll be prompted to configure it. In the ECS wizard, fill in the following fields:
+
+- **Dagster+ Organization**: Enter the name of your Dagster+ organization.
+- **Dagster+ Deployment**: Enter the name of the Dagster+ deployment you want to use. Leave this field empty if the agent will only serve Branch deployments.
+- **Enable Branch Deployments**: Whether to have this agent serve your ephemeral [Branch deployments](/dagster-plus/features/ci-cd/branch-deployments). Only a single agent should have this setting enabled.
+- **Agent Token**: Paste the agent token you generated in [Step 1](#step-1-generate-a-dagster-agent-token).
+
+The page should look similar to the following image. In this example, our organization name is `hooli` and our deployment is `prod`:
+
+![Example Configuration for the ECS Agent CloudFormation Template](/images/dagster-plus/deployment/agents/aws-ecs-stack-wizard-new.png)
+
+After you've finished configuring the stack in AWS, you can view the agent in Dagster+. To do so, navigate to the **Status** page and click the **Agents** tab. You should see the agent running in the **Agent statuses** section:
+
+![Instance Status](/images/dagster-plus/deployment/agents/dagster-cloud-instance-status.png)
+
+## Next steps
+
+Now that you've got your agent running, what's next?
+
+- **If you're getting Dagster+ set up**, the next step is to [add a code location](/dagster-plus/deployment/code-locations) using the agent.
+
+- **If you're ready to load your Dagster code**, refer to the [Adding Code to Dagster+](/dagster-plus/deployment/code-locations) guide for more info.
+
+If you need to upgrade your ECS agent's CloudFormation template, refer to the [upgrade guide](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/upgrading-cloudformation) for more info.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/upgrading-cloudformation.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/upgrading-cloudformation.md
new file mode 100644
index 0000000000000..bee2bfe5ae77d
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/upgrading-cloudformation.md
@@ -0,0 +1,35 @@
+---
+title: Upgrading CloudFormation for an Amazon ECS agent
+sidebar_label: Upgrading CloudFormation
+sidebar_position: 500
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+In this guide, we'll show you how to upgrade an existing [Amazon Elastic Container Services (ECS) agent](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/new-vpc)'s CloudFormation template.
+
+**Note**: To complete the steps in this guide, you'll need [permissions in Amazon Web Services (AWS) that allow you to manage ECS agents](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/security-iam-awsmanpol.html).
+
+1. Sign in to your AWS console.
+
+2. Navigate to the deployed stack and click **Update**.
+
+3. Select **Replace current template**. You can specify a specific Dagster+ version or upgrade to the latest template.
+
+ **If you have deployed your agent into [its own VPC](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/new-vpc), use the following:**
+
+ To use the [latest template](https://s3.amazonaws.com/dagster.cloud/cloudformation/ecs-agent-vpc.yaml)
+
+ To specify a [version](https://s3.amazonaws.com/dagster.cloud/cloudformation/ecs-agent-vpc-1-0-3.yaml)
+
+ **If you are deploying the agent into an [existing VPC](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/existing-vpc), use the following:**
+
+ To use the [latest template](https://s3.amazonaws.com/dagster.cloud/cloudformation/ecs-agent.yaml)
+
+ To specify a [version](https://s3.amazonaws.com/dagster.cloud/cloudformation/ecs-agent-1-0-3.yaml)
+
+4. Proceed through the remaining steps in the wizard.
+
+When finished, the agent will be re-deployed using the newest image and Dagster version.
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/architecture.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/architecture.md
new file mode 100644
index 0000000000000..3260890c05c96
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/architecture.md
@@ -0,0 +1,79 @@
+---
+title: 'Dagster+ Hybrid architecture'
+sidebar_position: 10
+---
+
+The Hybrid architecture is the most flexible and secure way to deploy Dagster+. It allows you to run your user code in your environment while leveraging Dagster+'s infrastructure for orchestration and metadata management
+
+## Hybrid architecture overview
+
+A **hybrid deployment** utilizes a combination of your infrastructure and Dagster-hosted backend services.
+
+The Dagster backend services - including the web frontend, GraphQL API, metadata database, and daemons (responsible for executing schedules and sensors) - are hosted in Dagster+. You are responsible for running an [agent](index.md#dagster-hybrid-agents) in your environment.
+
+![Dagster+ Hybrid deployment architecture](/images/dagster-plus/deployment/hybrid-architecture.png)
+
+Work is enqueued for your agent when:
+
+- Users interact with the web front end,
+- The GraphQL API is queried, or
+- Schedules and sensors tick
+
+The agent polls the agent API to see if any work needs to be done and launches user code as appropriate to fulfill requests. User code then streams metadata back to the agent API (GraphQL over HTTPS) to make it available in Dagster+.
+
+All user code runs within your environment, in isolation from Dagster system code.
+
+## The agent
+
+Because the agent communicates with the Dagster+ control plane over the agent API, it's possible to support agents that operate in arbitrary compute environments.
+
+This means that over time, Dagster+'s support for different user deployment environments will expand and custom agents can take advantage of bespoke compute environments such as HPC.
+
+See the [setup page](index.md#dagster-hybrid-agents) for a list of agents that are currently supported.
+
+## Security
+
+Dagster+ Hybrid relies on a shared security model.
+
+The Dagster+ control plane is SOC 2 Type II certified and follows best practices such as:
+- encrypting data at rest (AES 256) and in transit (TLS 1.2+)
+- highly available, with disaster recovery and backup strategies
+- only manages metadata such as pipeline names, execution status, and run duration
+
+The execution environment is managed by the customer:
+- Dagster+ doesn't have access to user code—your code never leaves your environment. Metadata about the code is fetched over constrained APIs.
+- All connections to databases, file systems, and other resources are made from your environment.
+- The execution environment only requires egress access to Dagster+. No ingress is required from Dagster+ to user environments.
+
+Additionally, the Dagster+ agent is [open source and auditable](https://github.com/dagster-io/dagster-cloud)
+
+The following highlights are described in more detail below:
+
+- [Interactions and queries](#interactions-and-queries)
+- [Runs](#runs)
+- [Ingress](#ingress)
+
+### Interactions and queries
+
+When Dagster+ needs to interact with user code - for instance, to display the structure of a job in the Dagster+ user interface, to run the body of a sensor definition, or to launch a run for a job - it enqueues a message for the Dagster+ Agent. The Dagster+ Agent picks up this message and then launches or queries user code running on the appropriate compute substrate.
+
+Depending on the agent implementation, user code may run in isolated OS processes, in Docker containers, in ECS Tasks, in Kubernetes Jobs and Services, or in a custom isolation strategy.
+
+Queries to user code run over a well-defined gRPC interface. Dagster+ uses this interface to:
+
+- Retrieve the names, config schemas, descriptions, tags, and structures of jobs, ops, repositories, partitions, schedules, and sensors defined in your code
+- Evaluate schedule and sensor ticks and determine whether a run should be launched
+
+When the agent queries user code, it writes the response back to Dagster+ over a well-defined GraphQL interface.
+
+### Runs
+
+Runs are launched by calling the `dagster api` CLI command in a separate process/container as appropriate to the agent type. Run termination is handled by interrupting the user code process/container as appropriate for the compute substrate.
+
+When runs are launched, the user code process/container streams structured metadata (containing everything that's viewable in the integrated logs viewer in the Dagster+ UI) back to Dagster+ over a well-defined GraphQL interface. Structured metadata is stored in Amazon RDS, encrypted at rest.
+
+By default, the run worker also uploads the compute logs (raw `stdout` and `stderr` from runs) to Dagster+. If you don't want to upload logs, you can disable this feature in the [agent settings](/dagster-plus/deployment/management/settings/hybrid-agent-settings).
+
+### Ingress
+
+No ingress is required from Dagster+ to user environments. All network requests are outbound from user environments to Dagster+.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/acr-user-code.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/acr-user-code.md
new file mode 100644
index 0000000000000..dea16460f440f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/acr-user-code.md
@@ -0,0 +1,168 @@
+---
+title: "Deploy user code in Azure Container Registry with Dagster+"
+sidebar_position: 200
+---
+
+This guide will walk you through setting up a new repository for your Dagster code, setting up CI/CD with GitHub Actions backed by Azure Container Registry (ACR), and deploying your code to your Azure Kubernetes Service (AKS) cluster.
+
+This guide assumes you already have an AKS agent running. You can follow along [here](aks-agent) if you still need to set up an AKS agent.
+
+## Prerequisites
+
+This guide will use a Github repository to store the Dagster code, and GitHub Actions to deploy the code to Azure Container Registry. If you need to use another CI/CD provider, such as Azure DevOps, the steps here will need to be adapted. For more information on configuring CI/CD using the `dagster-cloud` CLI, see "[Configuring CI/CD for your project](/dagster-plus/features/ci-cd/configuring-ci-cd#non-github).
+
+- The azure CLI installed on your machine. You can download it [here](https://docs.microsoft.com/en-us/cli/azure/install-azure-cli).
+- A GitHub account, and the ability to run GitHub Actions workflows in a repository.
+
+## Step 1: Creating a repository for Dagster code
+
+We'll create a new repository based on the [Dagster+ hybrid quickstart repository](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart). We'll go through these steps using a brand new repository in GitHub, but you should be able to adapt these steps to an existing repository or other version control systems.
+
+First, [create a new repository in GitHub](https://docs.github.com/en/repositories/creating-and-managing-repositories/creating-a-new-repository). Going forward, we'll refer to this repository as `dagster-plus-code`.
+
+Next, we'll run a few commands which clone both our new repository and the Dagster+ hybrid quickstart repository to our local machine.
+
+```bash
+git clone dagster-plus-code
+git clone git@github.com:dagster-io/dagster-cloud-hybrid-quickstart.git
+```
+
+We'll copy the contents of the `dagster-cloud-hybrid-quickstart` repository into our `dagster-plus-code` repository, and commit the changes.
+
+```bash
+rsync -av --exclude='.git' dagster-cloud-hybrid-quickstart/ dagster-plus-code/
+cd dagster-plus-code
+git add .
+git commit -m "Initial commit"
+git push
+```
+
+### Project structure
+
+The project has the following structure:
+
+```plaintext
+├── .github
+│ └── workflows
+│ └── dagster-cloud-deploy.yml # GitHub Actions workflow for re-deploying code location
+├── .vscode # Standard VSCode settings for working with a Dagster repository
+├── Dockerfile # Dockerfile for building the user code image
+├── README.md
+├── dagster_cloud.yaml # Configuration file describing all code locations in the repository
+├── pyproject.toml # Python project configuration file for the code location
+├── quickstart_etl # Python package containing the user code
+│ ├── __init__.py
+│ ├── assets
+│ │ ├── __init__.py
+│ │ └── hackernews.py
+│ └── definitions.py
+├── quickstart_etl_tests # User code tests
+│ ├── __init__.py
+│ └── test_assets.py
+├── setup.cfg
+└── setup.py
+```
+
+## Step 2: Setting up an Azure Container Registry
+
+Next, we'll set up an Azure Container Registry to store our Docker images. We'll use the Azure CLI to create the registry.
+
+```bash
+az login
+az acr create --resource-group --name --sku Basic
+```
+
+Then, we'll make images from our ACR available to our AKS cluster.
+
+```bash
+az aks update -n -g --attach-acr
+```
+
+## Step 3: Setting up GitHub Actions
+
+Now, we'll set up a Github Actions workflow to build and push our Docker image to Azure Container Registry.
+
+We already have a GitHub Actions workflow in our repository, located at `.github/workflows/dagster-cloud-deploy.yml`. This workflow will build the Docker image, push it to ACR, and update the code location in Dagster+. To get it working with your repository, you'll need to do a few things.
+
+#### Generate Azure credentials
+
+First, we'll need to generate a service principal for GitHub Actions to use to authenticate with Azure. We'll use the Azure CLI to create the service principal.
+
+```bash
+az ad sp create-for-rbac --name "github-actions-acr" --role contributor --scopes /subscriptions//resourceGroups//providers/Microsoft.ContainerRegistry/registries/
+```
+
+This command will output a JSON object with the service principal details. Make sure to save the `appId` and `password` values - we'll use them in the next step.
+
+### Add secrets to your repository
+
+We'll add the service principal details as secrets in our repository. Go to your repository in GitHub, and navigate to `Settings` -> `Secrets`. Add the following secrets:
+
+- `DAGSTER_CLOUD_API_TOKEN`: An agent token. For more details see [Managing agent tokens](/dagster-plus/deployment/management/tokens/agent-tokens).
+- `AZURE_CLIENT_ID`: The `appId` from the service principal JSON object.
+- `AZURE_CLIENT_SECRET`: The `password` from the service principal JSON object.
+
+### Update the GitHub Actions workflow
+
+For this step, open `.github/workflows/dagster-cloud-deploy.yml` in your repository with your preferred text editor to perform the changes below.
+
+In the `env` section of the workflow, update the following variables:
+
+- `DAGSTER_CLOUD_ORGANIZATION`: The name of your Dagster Cloud organization.
+- `IMAGE_REGISTRY`: The URL of your Azure Container Registry: `.azurecr.io`.
+
+We'll update the workflow to use the Azure Container Registry by uncommenting its section and providing the principal details. It should look like this:
+
+```yaml
+# Azure Container Registry (ACR)
+# https://github.com/docker/login-action#azure-container-registry-acr
+- name: Login to Azure Container Registry
+ if: steps.prerun.outputs.result != 'skip'
+ uses: docker/login-action@v3
+ with:
+ registry: ${{ env.IMAGE_REGISTRY }}
+ username: ${{ secrets.AZURE_CLIENT_ID }}
+ password: ${{ secrets.AZURE_CLIENT_SECRET }}
+```
+
+Finally, update the tags in the "Build and upload Docker image" step to match the full URL of your image in ACR:
+
+```yaml
+- name: Build and upload Docker image for "quickstart_etl"
+ if: steps.prerun.outputs.result != 'skip'
+ uses: docker/build-push-action@v4
+ with:
+ context: .
+ push: true
+ tags: ${{ env.IMAGE_REGISTRY }}/:${{ env.IMAGE_TAG }}
+ cache-from: type=gha
+ cache-to: type=gha,mode=max
+```
+
+### Update the `dagster_cloud.yaml` build configuration to use the Azure Container Registry
+
+Edit the `dagster_cloud.yaml` file in the root of your repository. Update the `build` section to use the Azure Container Registry, and provide an image name specific to the code location. This must match the registry and image name used in the previous step.
+
+```yaml
+locations:
+ - location_name: quickstart_etl
+ code_source:
+ package_name: quickstart_etl.definitions
+ build:
+ directory: ./
+ registry: .azurecr.io/
+```
+
+### Push and run the workflow
+
+Now, commit and push the changes to your repository. The GitHub Actions workflow should run automatically. You can check the status of the workflow in the `Actions` tab of your repository.
+
+![GitHub Actions workflow for deploying user code to Azure Container Registry](/images/dagster-plus/deployment/azure/github-actions-workflow.png)
+
+When the workflow completes, you should see the new code location in Dagster+. Navigate to the `Status` page, and click the `Code Locations` tab. You should see your new code location listed.
+
+![Dagster+ code locations page showing the new code location](/images/dagster-plus/deployment/azure/dagster-cloud-code-locations.png)
+
+## Next steps
+
+Now that you have your code location deployed, you can follow the guide [here](blob-compute-logs) to set up logging in your AKS cluster.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/aks-agent.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/aks-agent.md
new file mode 100644
index 0000000000000..87f87d2fc3d11
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/aks-agent.md
@@ -0,0 +1,55 @@
+---
+title: "Deploy a Dagster+ agent on an Azure Kubernetes Service cluster"
+sidebar_position: 100
+---
+
+This guide will walk you through deploying a Dagster+ agent on an Azure Kubernetes Service (AKS) cluster.
+
+This guide is intended to be a quickstart, and you should always defer to organization-specific guidelines for creating and managing new infrastructure.
+
+We'll start from a brand new organization in Dagster+, and finish with a full hybrid deployment of Dagster+ using Azure infrastructure.
+
+## Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- The azure CLI installed on your machine. You can download it [here](https://docs.microsoft.com/en-us/cli/azure/install-azure-cli).
+- `kubectl` installed on your machine. You can download it [here](https://kubernetes.io/docs/tasks/tools/install-kubectl/).
+- `helm` installed on your machine. You can download it [here](https://helm.sh/docs/intro/install/).
+- An existing AKS cluster. If you need to create a new AKS cluster, refer to the [Azure documentation](https://learn.microsoft.com/en-us/azure/aks/learn/quick-kubernetes-deploy-portal?tabs=azure-cli).
+- A Dagster+ organization, with an agent token for that organization.
+
+## Step 1: Generate a Dagster+ agent token.
+
+In this step, you'll generate a token for the Dagster+ agent. The Dagster+ agent will use this to authenticate to the agent API.
+
+1. Sign in to your Dagster+ instance.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. In the **Organization Settings** page, click the **Tokens** tab.
+4. Click the **+ Create agent token** button.
+5. After the token has been created, click **Reveal token**.
+
+Keep the token somewhere handy - you'll need it to complete the setup.
+
+## Step 2: Log in to your AKS cluster.
+
+We'll use the `azure` CLI to log in to your AKS cluster. Run the following command and follow the prompts to log in:
+
+```bash
+az login
+az aks get-credentials --resource-group --name
+```
+
+We should now be able to verify our installation by running a command that tells us the current context of our kubectl installation. We'd expect it to output the name of the AKS cluster.
+
+```bash
+kubectl config current-context
+```
+
+## Step 3: Install the Dagster+ agent on the AKS cluster.
+
+Next, we'll install the agent helm chart. You should be able to follow the guide [here](/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration) to install the agent on the AKS cluster.
+
+## Next steps
+
+Now that you have an agent running on your AKS cluster, you can start deploying Dagster code to it. You can follow the guide [here](acr-user-code) to deploy user code to your AKS cluster backed by Azure Container Registry (ACR).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/blob-compute-logs.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/blob-compute-logs.md
new file mode 100644
index 0000000000000..8a7c13dc19541
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/blob-compute-logs.md
@@ -0,0 +1,150 @@
+---
+title: "Store compute logs in Azure Blob Storage/Azure Data Lake Storage"
+sidebar_position: 300
+---
+
+In this guide, we'll walk through how to store compute logs in Azure Blob Storage or Azure Data Lake Storage. This guide assumes you have already set up an Azure Kubernetes Service (AKS) agent and deployed user code in Azure Container Registry (ACR).
+
+This guide focuses on using Azure Blob Storage, but the same steps should be applicable for Azure Data Lake Storage.
+
+If you have not yet set up an AKS agent, you can follow the [Deploy an Azure Kubernetes Service (AKS) agent guide](aks-agent). If you have not yet deployed user code in ACR, you can follow the [Deploy user code in Azure Container Registry (ACR) guide](acr-user-code).
+
+## Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- The Azure CLI installed on your machine. You can download it [here](https://docs.microsoft.com/en-us/cli/azure/install-azure-cli).
+- An Azure account with the ability to create resources in Azure Blob Storage or Azure Data Lake Storage.
+- An Azure container in Azure Blob Storage or Azure Data Lake Storage where you want to store logs.
+- Either the `quickstart_etl` module from the [hybrid quickstart repo](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart), or any other code location successfully imported, which contains at least one asset or job that will generate logs for you to test against.
+
+## Step 1: Give AKS agent access to blob storage account
+
+We need to ensure that the AKS agent has the necessary permissions to write logs to Azure Blob Storage or Azure Data Lake Storage. We'll do this with some azure CLI commands.
+
+First, we'll enable the cluster to use workload identity. This will allow the AKS agent to use a managed identity to access Azure resources.
+
+```bash
+az aks update --resource-group --name --enable-workload-identity
+```
+
+Then, we'll create a new managed identity for the AKS agent.
+
+```bash
+az identity create --resource-group --name agent-identity
+```
+
+We will need to find the name of the service account used by the Dagster+ Agent. If you used the [Dagster+ Helm chart](/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration), it should be `user-cloud-dagster-cloud-agent`. You can confirm by using this command:
+
+```bash
+kubectl get serviceaccount -n
+```
+
+Now we need to federate the managed identity with the service account used by the Dagster+ Agent.
+
+```bash
+az identity federated-credential create \
+ --name dagster-agent-federated-id \
+ --identity-name agent-identity \
+ --resource-group \
+ --issuer $(az aks show -g -n --query "oidcIssuerProfile.issuerUrl" -otsv) \
+ --subject system:serviceaccount::
+```
+
+You will need to obtain the client id of this identity for the next few operations. Make sure to save this value:
+
+```bash
+az identity show -g -n agent-identity --query 'clientId' -otsv
+```
+
+We need to grant access to the storage account.
+
+```bash
+az role assignment create \
+ --assignee \
+ --role "Storage Blob Data Contributor" \
+ --scope $(az storage account show -g -n --query 'id' -otsv)
+```
+
+You will need to add new annotations and labels in Kubernetes to enable the use of workload identities. If you're using the Dagster+ Helm Chart, modify your values.yaml to add the following lines:
+
+```yaml
+serviceAccount:
+ annotations:
+ azure.workload.identity/client-id: ""
+
+dagsterCloudAgent:
+ labels:
+ azure.workload.identity/use: "true"
+
+workspace:
+ labels:
+ azure.workload.identity/use: "true"
+```
+
+:::tip
+
+If you need to retrieve the values used by your Helm deployment, you can run:
+ `helm get values user-cloud > values.yaml`.
+
+:::
+
+Finally, update your Helm release with the new values:
+
+```bash
+helm upgrade user-cloud dagster-cloud/dagster-cloud-agent -n -f values.yaml
+```
+
+If everything is set up correctly, you should be able to run the following command and see an access token returned:
+
+```bash
+kubectl exec -n -it -- bash
+# in the pod
+apt update && apt install -y curl # install curl if missing, may vary depending on the base image
+curl -H "Metadata:true" "http://169.254.169.254/metadata/identity/oauth2/token?resource=https://storage.azure.com/&api-version=2018-02-01"
+```
+
+## Step 2: Configure Dagster to use Azure Blob Storage
+
+Once again, you need to update the Helm values to use Azure Blob Storage for logs. You can do this by editing the `values.yaml` file for your user-cloud deployment to include the following lines:
+
+```yaml
+computeLogs:
+ enabled: true
+ custom:
+ module: dagster_azure.blob.compute_log_manager
+ class: AzureBlobComputeLogManager
+ config:
+ storage_account: mystorageaccount
+ container: mycontainer
+ default_azure_credential:
+ exclude_environment_credential: false
+ prefix: dagster-logs
+ local_dir: "/tmp/cool"
+ upload_interval: 30
+```
+
+Finally, update your deployment with the new values:
+
+```bash
+helm upgrade user-cloud dagster-cloud/dagster-cloud-agent -n -f values.yaml
+```
+
+## Step 3: Update your code location to enable the use of the AzureBlobComputeLogManager
+
+- Add `dagster-azure` to your `setup.py` file. This will allow you to import the `AzureBlobComputeLogManager` class.
+
+## Step 4: Verify logs are being written to Azure Blob Storage
+
+It's time to kick off a run in Dagster to test your new configuration. If following along with the quickstart repo, you should be able to kick off a run of the `all_assets_job`, which will generate logs for you to test against. Otherwise, use any job that emits logs. When you go to the stdout/stderr window of the run page, you should see a log file that directs you to the Azure Blob Storage container.
+
+![Azure Blob Storage logs in Dagster](/images/dagster-plus/deployment/azure/azure-blob-storage-logs.png)
+
+:::tip
+
+Whether or not the URL will be clickable depends on whether your logs are
+public or private. If they are private, directly clicking the link would not
+work, and instead you should use either the Azure CLI or the Azure Portal to
+access the logs using the URL.
+
+:::
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/index.md
new file mode 100644
index 0000000000000..96ce6fbe537f8
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/azure/index.md
@@ -0,0 +1,12 @@
+---
+title: "Microsoft Azure"
+sidebar_position: 50
+---
+
+# Deploying Dagster+ hybrid on Azure
+
+This guide will walk you through deploying Dagster+ on Azure. We'll start from a brand new organization in Dagster+ and finish with a full hybrid deployment of Dagster+ using Azure infrastructure.
+
+* [Step 1: Deploy an Azure Kubernetes Service (AKS) agent](aks-agent)
+* [Step 2: Deploy user code in Azure Container Registry (ACR)](acr-user-code)
+* [Step 3: Store compute logs in Azure Blob Storage or Azure Data Lake Storage](blob-compute-logs)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/docker/configuration.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/docker/configuration.md
new file mode 100644
index 0000000000000..56edbeb19a2f9
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/docker/configuration.md
@@ -0,0 +1,37 @@
+---
+title: Docker agent configuration
+sidebar_position: 200
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+This reference describes the various configuration options Dagster+ currently supports for Docker agents
+
+---
+
+## Environment variables and secrets
+
+Using the `container_context.docker.env_vars` property, you can include environment variables and secrets in the Docker container associated with a specific code location. For example:
+
+```yaml
+# dagster_cloud.yaml
+locations:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ docker:
+ env_vars:
+ - DATABASE_NAME
+ - DATABASE_USERNAME=hooli_testing
+```
+
+The `container_context.docker.env_vars` property is a list, where each item can be either `KEY` or `KEY=VALUE`. If only `KEY` is specified, the value will be pulled from the local environment.
+
+Refer to the following guides for more info about environment variables:
+
+- [Dagster+ environment variables and secrets](/dagster-plus/deployment/management/environment-variables/)
+- [Using environment variables and secrets in Dagster code](/guides/deploy/using-environment-variables-and-secrets)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/docker/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/docker/index.md
new file mode 100644
index 0000000000000..81ebfa807f20a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/docker/index.md
@@ -0,0 +1,8 @@
+---
+title: Docker agent
+sidebar_position: 40
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/docker/setup.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/docker/setup.md
new file mode 100644
index 0000000000000..bbb63babd5916
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/docker/setup.md
@@ -0,0 +1,121 @@
+---
+title: Docker agent setup
+sidebar_position: 100
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+In this guide, you'll configure and run a Docker agent. Docker agents are used to launch your code in Docker containers.
+
+## Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- **Permissions in Dagster+ that allow you to manage agent tokens**. Refer to the [User permissions documentation](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions/) for more info.
+- **To have Docker installed**
+- **Access to a container registry to which you can push images with Dagster code.** Additionally, your Docker agent must have the permissions required to pull images from the registry.
+
+ This can be:
+
+ - A self-hosted registry,
+ - A public registry such as [DockerHub](https://hub.docker.com/), or
+ - A managed offering such as [Amazon ECR](https://aws.amazon.com/ecr/), [Azure CR](https://azure.microsoft.com/en-us/services/container-registry/#overview), or [Google CR](https://cloud.google.com/container-registry)
+
+## Step 1: Generate a Dagster+ agent token
+
+In this step, you'll generate a token for the Dagster+ agent. The Dagster+ agent will use this to authenticate to the agent API.
+
+1. Sign in to your Dagster+ instance.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. In the **Organization Settings** page, click the **Tokens** tab.
+4. Click the **+ Create agent token** button.
+5. After the token has been created, click **Reveal token**.
+
+Keep the token somewhere handy - you'll need it to complete the setup.
+
+## Step 2: Create a Docker agent
+
+1. Create a Docker network for your agent:
+
+ ```shell
+ docker network create dagster_cloud_agent
+ ```
+
+2. Create a `dagster.yaml` file:
+
+ ```yaml
+ instance_class:
+ module: dagster_cloud.instance
+ class: DagsterCloudAgentInstance
+
+ dagster_cloud_api:
+ agent_token:
+ branch_deployments: true # enables branch deployments
+ deployment: prod
+
+ user_code_launcher:
+ module: dagster_cloud.workspace.docker
+ class: DockerUserCodeLauncher
+ config:
+ networks:
+ - dagster_cloud_agent
+ ```
+
+3. In the file, fill in the following:
+
+ - `agent_token` - Add the agent token you created in [Step 1](#step-1-generate-a-dagster-agent-token)
+ - `deployment` - Enter the deployment associated with this instance of the agent.
+
+ In the above example, we specified `prod` as the deployment. This is present when Dagster+ organizations are first created.
+
+4. Save the file.
+
+## Step 3: Start the agent
+
+Next, you'll start the agent as a container. Run the following command in the same folder as your `dagster.yaml` file:
+
+```shell
+docker run \
+ --network=dagster_cloud_agent \
+ --volume $PWD/dagster.yaml:/opt/dagster/app/dagster.yaml:ro \
+ --volume /var/run/docker.sock:/var/run/docker.sock \
+ -it docker.io/dagster/dagster-cloud-agent:latest \
+ dagster-cloud agent run /opt/dagster/app
+```
+
+This command:
+
+- Starts the agent with your local `dagster.yaml` mounted as a volume
+- Starts the system Docker socket mounted as a volume, allowing the agent to launch containers.
+
+To view the agent in Dagster+, click the Dagster icon in the top left to navigate to the **Status** page and click the **Agents** tab. You should see the agent running in the **Agent statuses** section:
+
+![Instance Status](/images/dagster-plus/deployment/agents/dagster-cloud-instance-status.png)
+
+## Credential Helpers
+
+If your images are stored in a private registry, configuring a [Docker credentials helper](https://docs.docker.com/engine/reference/commandline/login/#credential-helpers) allows the agent to log in to your registry. The agent image comes with several popular credentials helpers preinstalled:
+
+- [docker-credential-ecr-login](https://github.com/awslabs/amazon-ecr-credential-helper)
+- [docker-credential-gcr](https://github.com/GoogleCloudPlatform/docker-credential-gcr)
+
+These credential helpers generally are configured in `~/.docker.config.json`. To use one, make sure you mount that file as a volume when you start your agent:
+
+```shell
+ docker run \
+ --network=dagster_cloud_agent \
+ --volume $PWD/dagster.yaml:/opt/dagster/app/dagster.yaml:ro \
+ --volume /var/run/docker.sock:/var/run/docker.sock \
+ --volume ~/.docker/config.json:/root/.docker/config.json:ro \
+ -it docker.io/dagster/dagster-cloud-agent:latest \
+ dagster-cloud agent run /opt/dagster/app
+```
+
+## Next steps
+
+Now that you've got your agent running, what's next?
+
+- **If you're getting Dagster+ set up**, the next step is to [add a code location](/dagster-plus/deployment/code-locations/) using the agent.
+- **If you're ready to load your Dagster code**, refer to the [Adding Code to Dagster+](/dagster-plus/deployment/code-locations/) guide for more info.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/index.md
new file mode 100644
index 0000000000000..6889d3650933f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/index.md
@@ -0,0 +1,61 @@
+---
+title: "Hybrid deployment"
+sidebar_label: Hybrid
+sidebar_position: 20
+---
+
+In a Dagster+ Hybrid deployment, the orchestration control plane is run by Dagster+ while your Dagster code is executed within your environment.
+
+:::note
+
+For an overview of the Hybrid design, including security considerations, see [Dagster+ Hybrid architecture](architecture).
+
+:::
+
+## Get started
+
+To get started with a Hybrid deployment, you'll need to:
+
+1. Create a [Dagster+ organization](https://dagster.cloud/signup)
+2. [Install a Dagster+ Hybrid agent](#dagster-hybrid-agents)
+3. [Add a code location](/dagster-plus/deployment/code-locations/), typically using a Git repository and CI/CD
+
+## Dagster+ Hybrid agents
+
+The Dagster+ agent is a long-lived process that polls Dagster+'s API servers for new work. Currently supported agents include:
+
+ - [Kubernetes](/dagster-plus/deployment/deployment-types/hybrid/kubernetes)
+ - [AWS ECS](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/new-vpc)
+ - [Docker](/dagster-plus/deployment/deployment-types/hybrid/docker)
+ - [Local agent](/dagster-plus/deployment/deployment-types/hybrid/local)
+
+:::tip
+
+If you're not sure which agent to use, we recommend the [Dagster+ Kubernetes agent](/dagster-plus/deployment/deployment-types/hybrid/kubernetes/) in most cases.
+
+:::
+
+## What you'll see in your environment
+
+### Code location servers
+
+Dagster+ runs your Dagster projects through code locations. To add a code location, see the [code locations documentation](/dagster-plus/deployment/code-locations/).
+
+When you inform Dagster+ about a new code location, we enqueue instructions for your agent to launch a new code server. The agent uses your container image to launch a code server that interacts with your Dagster definitions. The agent will run one long-standing code server for each code location. Once the code server is running, the agent will send Dagster+ metadata about your Dagster definitions that Dagster+ uses to make orchestration decisions.
+
+
+### Runs
+
+Your definitions might include [automations](/guides/automate/) that launch runs or materialize assets. Or your developers might launch runs directly with the web UI.
+
+When a run needs to be launched, Dagster+ enqueues instructions for your agent to launch a new run. The next time your agent polls Dagster+ for new work, it will see instructions about how to launch your run. It will delegate those instructions to your code server and your code server will launch a run - a new run will typically require its own container.
+
+Your agent will send Dagster+ metadata letting us know the run has been launched. Your run's container will also send Dagster+ metadata informing us of how the run is progressing. The Dagster+ backend services will monitor this stream of metadata to make additional orchestration decisions, monitor for failure, or send alerts.
+
+## Best practices
+
+### Security
+
+You can do the following to make your Dagster+ Hybrid deployment more secure:
+- [Disable log forwarding](/dagster-plus/deployment/management/settings/customizing-agent-settings#disabling-compute-logs)
+- [Manage tokens](/dagster-plus/deployment/management/tokens/agent-tokens)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration.md
new file mode 100644
index 0000000000000..ba7e087ddcb5b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration.md
@@ -0,0 +1,183 @@
+---
+title: Kubernetes agent configuration
+sidebar_position: 200
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+This reference describes the various configuration options Dagster+ currently supports for [Kubernetes agents](setup).
+
+## Viewing the Helm chart
+
+To see the different customizations that can be applied to the Kubernetes agent, you can view the chart's default values:
+
+```shell
+helm repo add dagster-plus https://dagster-io.github.io/helm-user-cloud
+helm repo update
+helm show values dagster-plus/dagster-plus-agent
+```
+
+You can also view the chart values on [ArtifactHub](https://artifacthub.io/packages/helm/dagster-cloud/dagster-cloud-agent?modal=values).
+
+## Per-deployment configuration
+
+The [`workspace`](https://artifacthub.io/packages/helm/dagster-cloud/dagster-cloud-agent?modal=values) value of the Helm chart provides the ability to add configuration for all jobs that are spun up by the agent, across all repositories. To add secrets or mounted volumes to all Kubernetes Pods, you can specify your desired configuration under this value.
+
+Additionally, the [`imagePullSecrets`](https://artifacthub.io/packages/helm/dagster-cloud/dagster-cloud-agent?modal=values) value allows you to specify a list of secrets that should be included when pulling the images for your containers.
+
+## Per-location configuration
+
+When [adding a code location](/dagster-plus/deployment/code-locations/) to Dagster+ with a Kubernetes agent, you can use the `container_context` key on the location configuration to add additional Kubernetes-specific configuration. If you're using the Dagster+ Github action, the `container_context` key can also be set for each location in your `dagster_cloud.yaml` file, using the same format.
+
+The following example [`dagster_cloud.yaml`](/dagster-plus/deployment/code-locations/dagster-cloud-yaml) file illustrates the available fields:
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ k8s:
+ env_config_maps:
+ - my_config_map
+ env_secrets:
+ - my_secret
+ env_vars:
+ - FOO_ENV_VAR=foo_value
+ - BAR_ENV_VAR
+ image_pull_policy: Always
+ image_pull_secrets:
+ - name: my_image_pull_secret
+ labels:
+ my_label_key: my_label_value
+ namespace: my_k8s_namespace
+ service_account_name: my_service_account_name
+ volume_mounts:
+ - mount_path: /opt/dagster/test_mount_path/volume_mounted_file.yaml
+ name: test-volume
+ sub_path: volume_mounted_file.yaml
+ volumes:
+ - name: test-volume
+ config_map:
+ name: test-volume-configmap
+ server_k8s_config: # Raw kubernetes config for code servers launched by the agent
+ pod_spec_config: # Config for the code server pod spec
+ node_selector:
+ disktype: standard
+ pod_template_spec_metadata: # Metadata for the code server pod
+ annotations:
+ mykey: myvalue
+ deployment_metadata: # Metadata for the code server deployment
+ annotations:
+ mykey: myvalue
+ service_metadata: # Metadata for the code server service
+ annotations:
+ mykey: myvalue
+ container_config: # Config for the main dagster container in the code server pod
+ resources:
+ limits:
+ cpu: 100m
+ memory: 128Mi
+ run_k8s_config: # Raw kubernetes config for runs launched by the agent
+ pod_spec_config: # Config for the run's PodSpec
+ node_selector:
+ disktype: ssd
+ container_config: # Config for the main dagster container in the run pod
+ resources:
+ limits:
+ cpu: 500m
+ memory: 1024Mi
+ pod_template_spec_metadata: # Metadata for the run pod
+ annotations:
+ mykey: myvalue
+ job_spec_config: # Config for the Kubernetes job for the run
+ ttl_seconds_after_finished: 7200
+ job_metadata: # Metadata for the Kubernetes job for the run
+ annotations:
+ mykey: myvalue
+```
+
+### Environment variables and secrets
+
+Using the `container_context.k8s.env_vars` and `container_context.k8s.env_secrets` properties, you can specify environment variables and secrets for a specific code location. For example:
+
+```yaml
+# dagster_cloud.yaml
+
+location:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ k8s:
+ env_vars:
+ - database_name
+ - database_username=hooli_testing
+ env_secrets:
+ - database_password
+```
+
+ | Property | Description |
+ |---------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+ | `env_vars` | A list of environment variable names to inject into the job, formatted as `KEY` or `KEY=VALUE`. If only `KEY` is specified, the value will be pulled from the current process. |
+ | `env_secrets` | A list of secret names, from which environment variables for a job are drawn using `envFrom`. Refer to the [Kubernetes documentation](https://kubernetes.io/docs/tasks/inject-data-application/distribute-credentials-secure/#configure-all-key-value-pairs-in-a-secret-as-container-environment-variables) for more info. |
+
+Refer to the following guides for more info about environment variables:
+
+- [Dagster+ environment variables and secrets](/dagster-plus/deployment/management/environment-variables/)
+- [Using environment variables and secrets in Dagster code](/guides/deploy/using-environment-variables-and-secrets)
+
+## Op isolation
+
+By default, each Dagster job will run in its own Kubernetes pod, with each op running in its own subprocess within the pod.
+
+You can also configure your Dagster job with the to run each op in its own Kubernetes pod. For example:
+
+```python
+from dagster import job
+from dagster_k8s import k8s_job_executor
+
+@job(executor_def=k8s_job_executor)
+def k8s_job():
+ ...
+```
+
+## Per-job and per-op configuration
+
+{/* To add configuration to specific Dagster jobs, ops, or assets, use the `dagster-k8s/config` tag. For example, to specify that a job should have certain resource limits when it runs. Refer to [Customizing your Kubernetes deployment for Dagster Open Source](/deployment/guides/kubernetes/customizing-your-deployment#per-job-kubernetes-configuration) for more info. */}
+To add configuration to specific Dagster jobs, ops, or assets, use the `dagster-k8s/config` tag. For example, to specify that a job should have certain resource limits when it runs. Refer to [Customizing your Kubernetes deployment for Dagster Open Source](/guides/deploy/deployment-options/kubernetes/customizing-your-deployment) for more info.
+
+## Running as a non-root user
+
+Starting in 0.14.0, the provided `docker.io/dagster/dagster-cloud-agent` image offers a non-root user with id `1001`. To run the agent with this user, you can specify the [`dagsterCloudAgent`](https://artifacthub.io/packages/helm/dagster-cloud/dagster-cloud-agent?modal=values) value in the Helm chart to be:
+
+```yaml
+dagsterCloudAgent:
+ podSecurityContext:
+ runAsUser: 1001
+```
+
+We plan to make this user the default in a future release.
+
+## Grant AWS permissions
+
+You can provide your Dagster pods with [permissions to assume an AWS IAM role](https://docs.aws.amazon.com/eks/latest/userguide/iam-roles-for-service-accounts.html) using a [Service Account](https://kubernetes.io/docs/tasks/configure-pod-container/configure-service-account/). For example, you might do this to [configure an S3 IO Manager](/guides/deploy/deployment-options/aws#using-s3-for-io-management).
+
+1. [Create an IAM OIDC provider for your EKS cluster](https://docs.aws.amazon.com/eks/latest/userguide/enable-iam-roles-for-service-accounts.html)
+2. [Create an IAM role and and attach IAM policies](https://docs.aws.amazon.com/eks/latest/userguide/associate-service-account-role.html)
+3. Update the [ Helm chart](#viewing-the-helm-chart) to associate the IAM role with a service account:
+
+ ```bash
+ serviceAccount:
+ create: true
+ annotations:
+ eks.amazonaws.com/role-arn: "arn:aws:iam::1234567890:role/my_service_account_role"
+ ```
+
+This will allow your agent and the pods it creates to assume the `my_service_account_role` IAM role.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/kubernetes/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/kubernetes/index.md
new file mode 100644
index 0000000000000..3b4e765984d84
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/kubernetes/index.md
@@ -0,0 +1,8 @@
+---
+title: Kubernetes agent
+sidebar_position: 20
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/kubernetes/setup.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/kubernetes/setup.md
new file mode 100644
index 0000000000000..ee26e44a20b5b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/kubernetes/setup.md
@@ -0,0 +1,454 @@
+---
+title: Kubernetes agent setup
+sidebar_position: 100
+---
+
+This page provides instructions for running the Dagster+ agent on a [Kubernetes](https://kubernetes.io) cluster.
+
+## Installation
+
+
+### Prerequisites
+
+You'll need a Kubernetes cluster. This can be a self-hosted Kubernetes cluster or a managed offering like [Amazon EKS](https://aws.amazon.com/eks/), [Azure AKS](https://azure.microsoft.com/en-us/products/kubernetes-service), or [Google GKE](https://cloud.google.com/kubernetes-engine).
+
+You'll also need access to a container registry to which you can push images and from which pods in the Kubernetes cluster can pull images. This can be a self-hosted registry or a managed offering like [Amazon ECR](https://aws.amazon.com/ecr/), [Azure ACR](https://azure.microsoft.com/en-us/products/container-registry), or [Google GCR](https://cloud.google.com/artifact-registry).
+
+We recommend installing the Dagster+ agent using [Helm](https://helm.sh).
+
+### Step 1: Create a Kubernetes namespace
+
+```shell
+kubectl create namespace dagster-cloud
+```
+
+### Step 2: Create an agent token secret
+
+[Generate an agent token](/dagster-plus/deployment/management/tokens) and set it as a Kubernetes secret:
+
+```shell
+kubectl --namespace dagster-cloud create secret generic dagster-cloud-agent-token --from-literal=DAGSTER_CLOUD_AGENT_TOKEN=
+```
+
+### Step 3: Add the Dagster+ agent Helm chart repository
+
+```shell
+helm repo add dagster-cloud https://dagster-io.github.io/helm-user-cloud
+helm repo update
+```
+
+### Step 4: Install the Dagster+ agent Helm chart
+
+```shell
+helm --namespace dagster-cloud install agent --install dagster-cloud/dagster-cloud-agent
+```
+
+## Upgrading
+
+You can use Helm to do rolling upgrades of your Dagster+ agent. The version of the agent doesn't need to be the same as the version of Dagster used in your projects. The Dagster+ control plane is upgraded automatically but is backwards compatible with older versions of the agent.
+
+:::tip
+We recommend upgrading your Dagster+ agent every 6 months. The version of your agent is visible on the "Deployments", "Agents" tab https://your-org.dagster.plus/deployment/health. The current version of the agent matches the most [recent Dagster release](https://github.com/dagster-io/dagster/releases).
+:::
+
+```yaml
+# values.yaml
+dagsterCloudAgent:
+ image:
+ tag: latest
+```
+
+```shell
+helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+```
+
+## Troubleshooting tips
+
+You can see basic health information about your agent in the Dagster+ UI:
+
+[comment]: <> (TODO: Screenshot of Dagster+ Deployments agents tab)
+
+### View logs
+
+```shell
+kubectl --namespace dagster-cloud logs -l deployment=agent
+```
+
+
+## Common configurations
+
+There are three places to customize how Dagster interacts with Kubernetes:
+- **Per Deployment** by configuring the Dagster+ agent using [Helm values](https://artifacthub.io/packages/helm/dagster-cloud/dagster-cloud-agent?modal=values)
+- **Per Project** by configuring the `dagster_cloud.yaml` file for your [code location](/dagster-plus/deployment/code-locations)
+- **Per Asset or Job** by adding tags to the [asset](/guides/build/assets/defining-assets), [job](/guides/build/assets/asset-jobs), or [customizing the Kubernetes pipes invocation](/guides/build/external-pipelines/kubernetes-pipeline)
+
+Changes apply in a hierarchy, for example, a customization for an asset will override a default set globally in the agent configuration. Attributes that are not customized will use the global defaults.
+
+An exhaustive list of settings is available [here](/dagster-plus/deployment/management/settings/hybrid-agent-settings), but common options are presented below.
+
+
+### Configure your agents to serve branch deployments
+
+[Branch deployments](/dagster-plus/features/ci-cd/branch-deployments/index.md) are lightweight staging environments created for each code change. To configure your Dagster+ agent to manage them:
+
+```yaml
+# values.yaml
+dagsterCloud:
+ branchDeployment: true
+```
+
+```shell
+helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+```
+
+### Deploy a high availability architecture
+
+You can configure your Dagster+ agent to run with multiple replicas. Work will be load balanced across all replicas.
+
+```yaml
+# values.yaml
+dagsterCloudAgent:
+ replicas: 2
+```
+
+```shell
+helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+```
+
+Work load balanced across agents isn't sticky; there's no guarantee the agent that launched a run will be the same one to receive instructions to terminate it. This is fine if both replicas run on the same Kubernetes cluster because either agent can terminate the run. But if your agents are physically isolated (for example, they run on two different Kubernetes clusters), you should configure:
+
+```yaml
+# values.yaml
+isolatedAgents: true
+```
+
+```shell
+helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+```
+
+### Use a secret to pull images
+
+The agent is responsible for managing the lifecycle of your code locations and will typically need to pull images after your CI/CD process builds them and pushes them to your registry. You can specify a secret the agent will use to authenticate to your image registry.
+
+
+:::tip
+For cloud-based Kubernetes deployments such as AWS EKS, AKS, or GCP, you don't need an image pull secret. The role used by Kubernetes will have permission to access the registry, so you can skip this configuration.
+:::
+
+
+First create the secret. This step will vary based on the registry you use, but for DockerHub:
+
+```
+kubectl create secret docker-registry regCred \
+ --docker-server=DOCKER_REGISTRY_SERVER \
+ --docker-username=DOCKER_USER \
+ --docker-password=DOCKER_PASSWORD \
+ --docker-email=DOCKER_EMAIL
+```
+
+Use Helm to configure the agent with the secret:
+
+```yaml file=values.yaml
+# values.yaml
+imagePullSecrets: [regCred]
+```
+
+```shell
+helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+```
+
+### Don't send Dagster+ stdout and stderr
+
+By default, Dagster+ will capture stdout and stderr and present them to users in the runs page. You may not want to send Dagster+ these logs, in which case you should update the compute logs setting.
+
+
+
+
+
+The compute logs for a run will be stored in your S3 bucket and a link will be presented to users in the Dagster run page.
+
+```yaml file=values.yaml
+# values.yaml
+computeLogs:
+ enabled: true
+ custom:
+ module: dagster_aws.s3.compute_log_manager
+ class: S3ComputeLogManager
+ config:
+ show_url_only: true
+ bucket: your-compute-log-storage-bucket
+ region: your-bucket-region
+```
+
+```shell
+helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+```
+
+
+
+
+You can turn off compute logs altogether which will prevent Dagster+ from storing stdout and stderr. This setting will prevent users from accessing these logs.
+
+```yaml file=values.yaml
+# values.yaml
+computeLogs:
+ enabled: false
+```
+
+```shell
+helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+```
+
+
+
+
+
+### Make secrets available to your code
+
+You can make secrets available [through the Dagster+ web interface](/dagster-plus/deployment/management/environment-variables) or through Kubernetes. Configuring secrets through Kubernetes has the benefit that Dagster+ never stores or accesses the secrets, and they can be managed as code. First, create the Kubernetes secret:
+
+```bash
+kubectl create secret generic database-password-kubernetes-secret \
+ --from-literal=DATABASE_PASSWORD=your_password \
+ --namespace dagster-plus
+```
+
+
+Next, determine if the secret should be available to all code locations or a single code location.
+
+
+
+
+```yaml file=values.yaml
+# values.yaml
+workspace:
+ envSecrets:
+ - name: database-password-kubernetes-secret
+```
+
+```shell
+helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+```
+
+`envSecrets` will make the secret available in an environment variable, see the Kubernetes docs on [`envFrom` for details](https://kubernetes.io/docs/tasks/inject-data-application/distribute-credentials-secure/#configure-all-key-value-pairs-in-a-secret-as-container-environment-variables). In this example the environment variable `DATABASE_PASSWORD` would have the value `your_password`.
+
+
+
+
+
+Modify the [`dagster_cloud.yaml` file](/dagster-plus/deployment/code-locations/dagster-cloud-yaml) in your project's Git repository:
+
+```yaml file=dagster_cloud.yaml
+location:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ k8s:
+ env_secrets:
+ - database-password
+```
+
+`env_secrets` will make the secret available in an environment variable, see the Kubernetes docs on [`envFrom` for details](https://kubernetes.io/docs/tasks/inject-data-application/distribute-credentials-secure/#configure-all-key-value-pairs-in-a-secret-as-container-environment-variables). In this example the environment variable `DATABASE_PASSWORD` would have the value `your_password`.
+
+
+
+
+
+:::note
+If you need to request secrets from a secret manager like AWS Secrets Manager or HashiCorp Vault, follow one of the prior methods to give your code access to vault credentials. Then, inside your Dagster code, use those credentials to authenticate a Python client that requests secrets from the secret manager.
+:::
+
+
+### Use a different service account for a specific code location
+
+Modify the [`dagster_cloud.yaml` file](/dagster-plus/deployment/code-locations/dagster-cloud-yaml) in your project's Git repository:
+
+```yaml file=dagster_cloud.yaml
+locations:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ k8s:
+ service_account_name: my_service_account_name
+```
+
+### Run Dagster+ with different Kubernetes clusters
+
+
+
+
+Deploy the agent Helm chart to each cluster, setting the `isolatedAgents.enabled` flag to true.
+
+```yaml file=values.yaml
+# values.yaml
+isolatedAgents:
+ enabled: true
+```
+
+```shell
+helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+```
+
+In this configuration, requests will be randomly distributed across agents in both clusters.
+
+
+
+
+You may wish to run data pipelines from project A in Kubernetes cluster A, and project B in Kubernetes cluster B. For example, you may wish to run some jobs on-premise and other jobs in AWS. To accomplish this task:
+
+- Deploy an agent into each environment and use the `agentQueue` configuration:
+ Specify an agent queue for the on-prem agent:
+ ```yaml file=values.yaml
+ dagsterCloud:
+ agentQueues:
+ additionalQueues:
+ - on-prem-agent-queue
+ ```
+ Deploy onto the on-prem Kubernetes cluster:
+ ```shell
+ helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+ ```
+ Specify an agent queue for the agent on AWS:
+
+ ```yaml file=values.yaml
+ dagsterCloud:
+ agentQueues:
+ additionalQueues:
+ - aws-agent-queue
+ ```
+ Deploy onto the AWS Kubernetes cluster:
+ ```shell
+ helm --namespace dagster-cloud upgrade agent \
+ dagster-cloud/dagster-cloud-agent \
+ --values ./values.yaml
+ ```
+- Create separate code locations for each project
+- Update the `dagster_cloud.yaml` file for each code location
+ ```yaml file=dagster_cloud.yaml
+ locations:
+ - location_name: project-a
+ ...
+ agent_queue: on-prem-agent-queue
+ - location_name: project-b
+ ...
+ agent_queue: aws-agent-queue
+ ```
+
+:::tip
+Code locations without an `agent_queue` will be routed to a default queue. By default, all agents will serve this default queue. You can specify which agent should serve the default queue using the `includeDefaultQueue` setting:
+
+```yaml file=values.yaml
+dagsterCloud:
+ agentQueues:
+ includeDefaultQueue: true
+```
+:::
+
+
+
+
+If you want completely separate environments with their own asset graph, run history, and access controls you should create different Dagster+ deployments. Separate deployments are common for isolated tenants or separate dev, stage, and prod environments. Multiple deployments require a Dagster+ Pro plan. To create separate deployments:
+
+- Navigate to the deployments page for your organization: https://your-organizaiton.dagster.plus/org-settings/deployments
+
+- Click "New Deployment"
+
+- Follow the preceding instructions for creating and deploying agent for this deployment
+
+
+
+
+### Request resources such as CPU, memory, or GPU
+
+:::tip
+
+Dagster+ makes it easy to monitor CPU and memory used by code location servers and individual runs. For more information, see the [run monitoring documentation](/guides/deploy/execution/run-monitoring).
+:::
+
+First determine if you want to change the requested resource for everything in a code location, or for a specific job or asset.
+
+
+
+
+
+Modify the [`dagster_cloud.yaml` file](/dagster-plus/deployment/code-locations/dagster-cloud-yaml) in your project's Git repository:
+
+```yaml file=dagster_cloud.yaml
+locations:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ k8s:
+ server_k8s_config:
+ container_config:
+ resources:
+ limits:
+ cpu: 500m
+ memory: 2560Mi
+ run_k8s_config:
+ container_config:
+ resources:
+ limits:
+ cpu: 500m
+ memory: 2560Mi
+ nvidia.com/gpu: 1
+```
+
+The `server_k8s_config` section sets resources for the code location servers, which is where schedule and sensor evaluations occur.
+
+The `runs_k8s_config` section sets resources for the individual run.
+
+Requests are used by Kubernetes to determine which node to place a pod on, and limits are a strict upper bound on how many resources a pod can use while running. We recommend using both in most cases.
+
+The units for CPU and memory resources are described [in this document](https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/#resource-units-in-kubernetes).
+
+
+
+
+The default behavior in Dagster+ is to create one pod for a run. Each asset targeted by that run is executed in subprocess within the pod. Use a job tag to request resources for this pod, which in turn makes those resources available to the targeted assets.
+
+
+
+Another option is to launch a pod for each asset by telling Dagster to use the Kubernetes job executor. In this case, you can specify resources for each individual asset.
+
+
+
+
+
+
+
+Dagster can launch and manage existing Docker images as Kubernetes jobs using the [Dagster kubernetes pipes integration](/integrations/libraries/kubernetes). To request resources for these jobs by supplying the appropriate Kubernetes pod spec.
+
+
+
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/local.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/local.md
new file mode 100644
index 0000000000000..aa47685c1bf59
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/local.md
@@ -0,0 +1,74 @@
+---
+title: Running a local agent
+sidebar_position: 60
+sidebar_label: Local agent
+---
+
+Local agents are a good way to experiment with Dagster+ before deploying a more scalable Hybrid agent like [Kubernetes](/dagster-plus/deployment/deployment-types/hybrid/kubernetes) or [Amazon ECS](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/).
+
+:::note
+Local agents aren't well suited for most production use cases. If you're running the local agent in production, make sure that:
+
+- You've set up a supervisor to automatically restart the agent process if it crashes
+- You're alerted if the VM or container dies, or to automatically restart it
+:::
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- **Organization Admin** permissions in your Dagster+ account
+- **To install the `dagster-cloud` CLI** in the same environment where the agent will run. We recommend using a Python virtual environment for your Dagster application code and its dependencies.
+
+ ```bash
+ pip install dagster-cloud
+ ```
+
+
+## Step 1: Generate an agent token
+
+Your local agent will need a token to authenticate with your Dagster+ account. To generate an agent token:
+
+1. Click the **user menu (your icon) > Organization Settings**.
+2. In the **Organization Settings** page, click the **Tokens** tab.
+3. Click the **+ Create agent token** button.
+4. After the token has been created, click **Reveal token**.
+5. Save this token as an environment variable on the machine where the local agent will run. You can choose any name for this environment variable, but `$DAGSTER_AGENT_TOKEN` will be used in the rest of this guide.
+6. Give the agent token a description to distinguish it from other tokens in the future.
+
+## Step 2: Configure the local agent
+
+1. Create a directory to act as your Dagster home. This guide uses `~/dagster_home`, but the directory can be located wherever you want.
+2. In the new directory, create a `dagster.yaml` file with the following:
+
+3. In the file, fill in the following:
+ - `agent_token.env` - The name of the environment variable storing the agent token you created in Step 1.
+ - `deployment` - The name of the deployment associated with this instance of the agent. In the preceding example, `prod` was used as the deployment.
+4. Save the file.
+
+For more information about `dagster.yaml` configuration options, check out the [`dagster.yaml` reference](/guides/deploy/dagster-yaml).
+
+### Alternative methods for setting the agent token
+
+If you prefer not to specify the agent token by using an environment variable in `dagster.yaml`, pass it to the `dagster-cloud agent run` command:
+
+```bash
+dagster-cloud agent run ~/dagster_home/ --agent-token
+```
+
+## Step 3: Run the agent
+
+To start the agent, run the following command and pass the path to the `dagster.yaml` file you created in Step 2:
+
+```bash
+dagster-cloud agent run ~/dagster_home/
+```
+
+To view the agent in Dagster+, click the Dagster icon in the top left to navigate to the **Deployment > Agents** page. You should see the agent running in the **Agent statuses** section:
+
+![Screenshot of Dagster Asset Lineage](/images/dagster-plus/deployment/agents/dagster-cloud-instance-status.png)
+
+## Next steps
+
+- Add a [code location](/dagster-plus/deployment/code-locations/) to your Dagster+ deployment.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/multiple.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/multiple.md
new file mode 100644
index 0000000000000..46a596b77aa58
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/hybrid/multiple.md
@@ -0,0 +1,218 @@
+---
+title: Running multiple agents
+sidebar_position: 70
+sidebar_label: Multiple agents
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+Each Dagster+ full deployment (e.g., `prod`) needs to have at least one agent running. A single agent is adequate for many use cases, but you may want to run multiple agents to provide redundancy if a single agent goes down.
+
+---
+
+## Running multiple agents in the same environment
+
+To run multiple agents in the same environment (e.g., multiple Kubernetes agents in the same namespace), you can set the number of replicas in the configuration for your particular agent type:
+
+
+
+
+### In Docker
+
+In Docker, you can set the number of replicas for a service in the `docker-compose.yaml` file if the deployment mode is set to `replicated` (which is the default):
+
+```yaml
+services:
+ dagster-cloud-agent:
+ ...
+ deploy:
+ mode: replicated
+ replicas: 2
+```
+
+
+
+
+### In Kubernetes
+
+In Kubernetes, the number of replicas is set in the Helm chart. You can set the number of replicas in the Helm command:
+
+```shell
+helm upgrade \
+ ...
+ --set replicas=2
+```
+
+or if using a `values.yaml` file:
+
+```yaml
+dagsterCloudAgent:
+ ...
+ replicas: 2
+```
+
+
+
+
+### In Amazon ECS
+
+In Amazon ECS, the number of replicas can be set via the CloudFormation template:
+
+```yaml
+DagsterCloudAgent:
+ Type: AWS::ECS::Service
+ Properties:
+ ...
+ DesiredCount: 2
+```
+
+If using the CloudFormation template provided by Dagster, the number of replicas can be set via the `NumReplicas` parameter in the Amazon Web Services (AWS) UI.
+
+
+
+
+---
+
+## Running multiple agents in different environments
+
+To run multiple agents in an environment where each agent can not access the others' resources (for example, multiple Kubernetes namespaces or different clusters), enable the `isolated_agents` option. This is supported for all agent types.
+
+
+
+### In Docker
+
+Add the following to the `dagster.yaml` file:
+
+```yaml
+isolated_agents:
+ enabled: true
+
+dagster_cloud_api:
+ #
+ agent_label: "My agent" # optional
+```
+
+
+
+
+### In Kubernetes
+
+Add the following options to your Helm command:
+
+```shell
+helm upgrade \
+ ...
+ --set isolatedAgents.enabled=true \
+ --set dagsterCloud.agentLabel="My agent" # optional, only supported on 0.13.14 and later
+```
+
+Or if you're using a `values.yaml` file:
+
+```yaml
+isolatedAgents:
+ enabled: true
+
+dagsterCloud:
+ agentLabel: "My agent" # optional, only supported on 0.13.14 and later
+```
+
+
+
+
+### In Amazon ECS
+
+The `isolated_agents` option can be set as per-deployment configuration on the `dagster.yaml` file used by your agent. See the [ECS configuration reference](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/configuration-reference) guide for more information.
+
+
+
+---
+
+## Routing requests to specific agents
+
+:::note
+Agent queues are a Dagster+ Pro feature.
+:::
+
+Every Dagster+ agent serves requests from one or more queues. By default, requests for each code location are placed on a default queue and your agent will read requests only from that default queue.
+
+In some cases, you might want to route requests for certain code locations to specific agents. For example, routing requests for one code location to an agent running in an on-premise data center, but then routing requests for all other code locations to an agent running in AWS.
+
+To route requests for a code location to a specific agent, annotate the code locations with the name of a custom queue and configure an agent to serve only requests for that queue.
+
+### Step 1: Define an agent queue for the code location
+
+First, set an agent queue for the code location in your [`dagster_cloud.yaml`](/dagster-plus/deployment/code-locations/dagster-cloud-yaml):
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: data-eng-pipeline
+ code_source:
+ package_name: quickstart_etl
+ executable_path: venvs/path/to/dataengineering_spark_team/bin/python
+ agent_queue: special-queue
+```
+
+### Step 2: Configure an agent to handle the agent queue
+
+Next, configure an agent to handle your agent queue.
+
+
+
+#### In Docker
+
+Add the following to your project's [`dagster.yaml`](/guides/deploy/dagster-yaml) file:
+
+```yaml
+agent_queues:
+ include_default_queue: True # Continue to handle requests for code locations that aren't annotated with a specific queue
+ additional_queues:
+ - special-queue
+```
+
+
+
+
+#### In Kubernetes
+
+Add the following options to your Helm command:
+
+```shell
+helm upgrade \
+ ...
+ --set dagsterCloud.agentQueues.additionalQueues={"special-queue"}
+```
+
+Or if you're using a `values.yaml` file:
+
+```yaml
+dagsterCloud:
+ agentQueues:
+ # Continue to handle requests for code locations that aren't
+ # assigned to a specific agent queue
+ includeDefaultQueue: true
+ additionalQueues:
+ - special-queue
+```
+
+
+
+
+#### In Amazon ECS
+
+Modify your ECS Cloud Formation template to add the following configuration to the `config.yaml` passed to the agent:
+
+```yaml
+agent_queues:
+ # Continue to handle requests for code locations that aren't
+ # assigned to a specific agent queue
+ include_default_queue: true
+ additional_queues:
+ - special-queue
+```
+
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/index.md
new file mode 100644
index 0000000000000..bd70a65d90ffd
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/index.md
@@ -0,0 +1,8 @@
+---
+title: Deployment types
+sidebar_position: 20
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/multi-tenancy.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/multi-tenancy.md
new file mode 100644
index 0000000000000..a21c82cdab0a7
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/multi-tenancy.md
@@ -0,0 +1,136 @@
+---
+title: Managing multiple projects and teams with multi-tenancy
+sidebar_label: Multi-tenant
+sidebar_position: 30
+---
+
+In this article, we'll cover some strategies for managing multiple projects/code bases and teams in a Dagster+ account.
+
+## Separating code bases
+
+:::note
+In this section, repository refers to a version control system, such as Git or Mercurial.
+:::
+
+If you want to manage complexity or divide your work into areas of responsibility, consider isolating your code bases into multiple projects with:
+
+- Multiple directories in a single repository, or
+- Multiple repositories
+
+Refer to the following table for more information, including the pros and cons of each approach.
+
+| **Approach** | **How it works** | **Pros** | **Cons** |
+|---------------|------------------|----------|----------|
+| **Multiple directories in a single repository** | You can use a single repository to manage multiple projects by placing each project in a separate directory. Depending on your VCS, you may be able to set [code owners](https://docs.github.com/en/repositories/managing-your-repositorys-settings-and-features/customizing-your-repository/about-code-owners) to restrict who can modify each project. |
Easy to implement
Facilitates code sharing between projects
|
All projects share the same CI/CD pipeline and cannot be deployed independently
Shared dependencies between projects may cause conflicts and require coordination between teams
|
+| **Multiple repositories** | For stronger isolation, you can use multiple repositories to manage multiple projects. |
Stronger isolation between projects and teams
Each project has its own CI/CD pipeline and be deployed independently
Dependencies between projects can be managed independently
| Code sharing between projects requires additional coordination to publish and reuse packages between projects. |
+
+### Deployment configuration
+
+{/* Whether you use a single repository or multiple, you can use a [`dagster_cloud.yaml` file](/dagster-plus/managing-deployments/dagster-cloud-yaml) to define the code locations to deploy. For each repository, follow the [steps appropriate to your CI/CD provider](/dagster-plus/getting-started#step-4-configure-cicd-for-your-project) and include only the code locations that are relevant to the repository in your CI/CD workflow. */}
+Whether you use a single repository or multiple, you can use a [`dagster_cloud.yaml` file](/dagster-plus/deployment/code-locations/dagster-cloud-yaml) to define the code locations to deploy. For each repository, follow the [steps appropriate to your CI/CD provider](/dagster-plus/features/ci-cd/configuring-ci-cd) and include only the code locations that are relevant to the repository in your CI/CD workflow.
+
+#### Example with GitHub CI/CD on Hybrid deployment
+
+1. **For each repository**, use the CI/CD workflow provided in [Dagster+ Hybrid quickstart repository](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/blob/main/.github/workflows/dagster-cloud-deploy.yml).
+
+2. **For each project in the repository**, configure a code location in the [`dagster_cloud.yaml` file](/dagster-plus/deployment/code-locations/dagster-cloud-yaml):
+
+ ```yaml
+ # dagster_cloud.yml
+
+ locations:
+ - location_name: project_a
+ code_source:
+ package_name: project_a
+ build:
+ # ...
+ - location_name: project_b
+ code_source:
+ package_name: project_b
+ build:
+ # ...
+ ```
+
+3. In the repository's `dagster-cloud-deploy.yml` file, modify the CI/CD workflow to deploy all code locations for the repository:
+
+ ```yaml
+ # .github/workflows/dagster-cloud-deploy.yml
+
+ jobs:
+ dagster-cloud-deploy:
+ # ...
+ steps:
+ - name: Update build session with image tag for "project_a" code location
+ id: ci-set-build-output-project-a
+ if: steps.prerun.outputs.result != 'skip'
+ uses: dagster-io/dagster-cloud-action/actions/utils/dagster-cloud-cli@v0.1
+ with:
+ command: "ci set-build-output --location-name=project_a --image-tag=$IMAGE_TAG"
+
+ - name: Update build session with image tag for "project_b" code location
+ id: ci-set-build-output-project-b
+ if: steps.prerun.outputs.result != 'skip'
+ uses: dagster-io/dagster-cloud-action/actions/utils/dagster-cloud-cli@v0.1
+ with:
+ command: "ci set-build-output --location-name=project_b --image-tag=$IMAGE_TAG"
+ # ...
+ ```
+
+---
+
+## Isolating execution context between projects
+
+Separating execution context between projects can have several motivations:
+
+- Facilitating separation of duty between teams to prevent access to sensitive data
+- Differing compute environments and requirements, such as different architecture, cloud provider, etc.
+- Reducing impact on other projects. For example, a project with a large number of runs can impact the performance of other projects
+
+In order from least to most isolated, there are three levels of isolation:
+
+- [Code location](#code-location-isolation)
+- [Agent](#agent-isolation)
+- [Deployment](#deployment-isolation)
+
+### Code location isolation
+
+If you have no specific requirements for isolation beyond the ability to deploy and run multiple projects, you can use a single agent and deployment to manage all your projects as individual code locations.
+
+![Diagram of isolation at the code location level](/images/dagster-plus/deployment/management/managing-deployments/isolation-level-code-locations.png)
+
+| **Pros** | **Cons** |
+|----------|----------|
+|
Simplest and most cost-effective solution
User access control can be set at the code location level
Single glass pane to view all assets
| No isolation between execution environments |
+
+### Agent isolation
+
+:::note
+Agent queues are only available on [Hybrid deployment](/dagster-plus/deployment/deployment-types/hybrid/).
+:::
+
+Using the [agent routing feature](/dagster-plus/deployment/deployment-types/hybrid/multiple#routing-requests-to-specific-agents), you can effectively isolate execution environments between projects by using a separate agent for each project.
+
+Motivations for utilizing this approach could include:
+
+- Different compute requirements, such as different cloud providers or architectures
+- Optimizing for locality or access, such as running the data processing closer or in environment with access to the storage locations
+
+![Diagram of isolation at the agent level](/images/dagster-plus/deployment/management/managing-deployments/isolation-level-agents.png)
+
+| **Pros** | **Cons** |
+|----------|----------|
+|
Isolation between execution environments
User access control can be set at the code location level
Single glass pane to view all assets
| Extra work to set up additional agents and agent queues |
+
+### Deployment isolation
+
+:::note
+Multiple deployments are only available in Dagster+ Pro.
+:::
+
+Of the approaches outlined in this guide, multiple deployments are the most isolated solution. The typical motivation for this isolation level is to separate production and non-production environments.
+
+![Diagram of isolation at the Dagster+ deployment level](/images/dagster-plus/deployment/management/managing-deployments/isolation-level-deployments.png)
+
+| **Pros** | **Cons** |
+|----------|----------|
+|
Isolation between assets and execution environments
User access control can be set at the code location and deployment level
| No single glass pane to view all assets (requires switching between multiple deployments in the UI) |
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/ci-cd-in-serverless.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/ci-cd-in-serverless.md
new file mode 100644
index 0000000000000..709777c9c7f8a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/ci-cd-in-serverless.md
@@ -0,0 +1,85 @@
+---
+title: 'CI/CD in Serverless'
+sidebar_position: 300
+---
+
+If you're a GitHub or GitLab user, you can use our predefined workflows to seamlessly deploy and synchronize your code to Dagster+. You can also use other Git providers or a local Git repository with our [dagster-cloud CLI](/dagster-plus/deployment/management/dagster-cloud-cli) to run your own CI/CD process.
+
+
+
+
+If you're a GitHub user, with a single click our GitHub app with GitHub Actions can set up a repo containing skeleton code and configuration for you consistent with Dagster+'s best practices. Pushing your code changes to the `main` branch will automatically deploy them to your `prod` Serverless deployment. Pull requests will spin up ephemeral [branch deployments](/dagster-plus/features/ci-cd/branch-deployments/) that you can view in the Dagster+ UI for previewing and testing.
+
+:::note
+**If you are importing a Dagster project that's in an existing GitHub repo:**
+
+- The repo will need to allow the [Workflow permission](https://docs.github.com/en/repositories/managing-your-repositorys-settings-and-features/enabling-features-for-your-repository/managing-github-actions-settings-for-a-repository) for `Read and write permissions`. Workflow permissions settings can be found in GitHub's `Settings` > `Actions` > `General` > `Workflow permissions`. In GitHub Enterprise, these permissions [are controlled at the Organization level](https://github.com/orgs/community/discussions/57244).
+
+- An initial commit will need to be able to be merged directly to the repo's `main` branch to automatically add the GitHub Actions workflow files. If [branch protection rules](https://docs.github.com/en/repositories/configuring-branches-and-merges-in-your-repository/managing-protected-branches/about-protected-branches#about-protected-branches) require changes be done through a pull request, it will prevent the automatic setup from completing.
+
+ - You can temporarily disable the branch protection rules and then re-enable them after the automatic setup completes.
+
+:::
+
+
+
+
+
+If you're a GitLab user, with a single click our GitLab app can set up a repo containing skeleton code and CI/CD configuration for you consistent with Dagster+'s best practices. Pushing your code changes to the `main` branch will automatically deploy them to your `prod` Serverless deployment. Pull requests will spin up ephemeral [branch deployments](/dagster-plus/features/ci-cd/branch-deployments/index.md) that you can view in the Dagster+ UI for previewing and testing.
+
+
+
+
+
+If you don't want to use our automated GitHub/GitLab process, we offer [the powerful `dagster-cloud` command-line interface (CLI)](/dagster-plus/deployment/management/dagster-cloud-cli/) that you can use in another CI environment or locally.
+
+First, [create a new project](/getting-started/quickstart) with the Dagster open source CLI.
+
+The example below uses our [quickstart_etl example project](https://github.com/dagster-io/dagster/tree/master/examples/quickstart_etl). For more info about the examples, visit the [Dagster GitHub repository](https://github.com/dagster-io/dagster/tree/master/examples).
+
+```shell
+pip install dagster
+dagster project from-example \
+ --name my-dagster-project \
+ --example quickstart_etl
+```
+
+:::note
+If using a different project, ensure that `dagster-cloud` is included as a dependency in your `setup.py` or `requirements.txt` file.
+
+For example, in `my-dagster-project/setup.py`:
+
+```python
+install_requires=[
+ "dagster",
+ "dagster-cloud", # add this line
+ ...
+]
+```
+
+:::
+
+Next, install the [`dagster-cloud` CLI](/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring) and use its `configure` command to authenticate it to your Dagster+ organization.
+
+**Note:** The CLI requires a recent version of Python 3 and Docker.
+
+```shell
+pip install dagster-cloud
+dagster-cloud configure
+```
+
+You can also configure the `dagster-cloud` tool non-interactively; see [the CLI docs](/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring) for more information.
+
+Finally, deploy your project to Dagster+ using the `serverless` command:
+
+```shell
+dagster-cloud serverless deploy-python-executable ./my-dagster-project \
+ --location-name example \
+ --package-name quickstart_etl \
+ --python-version 3.12
+```
+
+**Note:** Windows users should use the `deploy` command instead of `deploy-python-executable`.
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/dagster-ips.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/dagster-ips.md
new file mode 100644
index 0000000000000..2ea95704944f5
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/dagster-ips.md
@@ -0,0 +1,27 @@
+---
+title: Dagster+ Serverless IP addresses
+sidebar_label: IP addresses
+sidebar_position: 400
+---
+
+Serverless code will make requests from one of the following IP addresses. You may need to whitelist / allowlist them for services your code interacts with.
+
+```plain
+34.216.9.66
+35.162.181.243
+35.83.14.215
+44.230.239.14
+44.240.64.133
+52.34.41.163
+52.36.97.173
+52.37.188.218
+52.38.102.213
+52.39.253.102
+52.40.171.60
+52.89.191.177
+54.201.195.80
+54.68.25.27
+54.71.18.84
+```
+
+**Note**: Additional IP addresses may be added over time. This list was last updated on **October 24, 2024**.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/index.md
new file mode 100644
index 0000000000000..ec81a914487c1
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/index.md
@@ -0,0 +1,38 @@
+---
+title: 'Serverless deployment'
+sidebar_label: Serverless
+sidebar_position: 10
+---
+
+Dagster+ Serverless is a fully managed version of Dagster+ and is the easiest way to get started with Dagster. With a Serverless deployment, you can run your Dagster jobs without spinning up any infrastructure yourself.
+
+## Serverless vs Hybrid
+
+Serverless works best with workloads that primarily orchestrate other services or perform light computation. Most workloads fit into this category, especially those that orchestrate third-party SaaS products like cloud data warehouses and ETL tools.
+
+If any of the following are applicable, you should select [Hybrid deployment](/dagster-plus/deployment/deployment-types/hybrid/):
+
+- You require substantial computational resources. For example, training a large machine learning (ML) model in-process
+- Your dataset is too large to fit in memory. For example, training a large ML model in-process on a terabyte of data
+- You need to distribute computation across many nodes for a single run. Dagster+ runs currently execute on a single node with 4 CPUs
+- You don't want to add Dagster Labs as a data processor
+
+## Limitations
+
+Serverless is subject to the following limitations:
+
+- Maximum of 100 GB of bandwidth per day
+- Maximum of 4500 step-minutes per day
+- Runs receive 4 vCPU cores, 16 GB of RAM, and 128 GB of ephemeral disk
+- Code locations receive 0.25 vCPU cores and 1 GB of RAM
+- All Serverless jobs run in the United States
+- Infrastructure cannot be customized or extended, such as using additional containers
+
+Dagster+ Pro customers may request a quota increase by [contacting Sales](https://dagster.io/contact).
+
+## Next steps
+
+To start using Dagster+ Serverless, you must [create a Dagster+ organization](https://dagster.plus/signup) if you have not already done so. You can sign up with:
+- a Google email address
+- a GitHub account
+- a one-time email link (ideal if you are using a corporate email). You can set up SSO after completing these steps.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/run-isolation.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/run-isolation.md
new file mode 100644
index 0000000000000..86b0d59679cf5
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/run-isolation.md
@@ -0,0 +1,63 @@
+---
+title: 'Serverless run isolation'
+sidebar_label: 'Run isolation'
+sidebar_position: 200
+---
+
+Dagster+ Serverless offers two settings for run isolation: isolated and non-isolated. Isolated runs are the default and are intended for production, while non-isolated runs are intended for faster iteration during development.
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- An active Dagster+ Serverless deployment
+- An understanding of [Dagster+ deployment settings](/dagster-plus/deployment/management/deployments/deployment-settings-reference)
+
+
+## Differences between isolated and non-isolated runs
+
+- [**Isolated runs**](#isolated-runs-default) execute in their own container. They're the default and are intended for production and compute-heavy use cases.
+- [**Non-isolated runs**](#non-isolated-runs) trade off isolation for speed. They must be launched manually and are intended for fast iteration during development.
+
+## Isolated runs (default)
+
+Isolated runs each take place in their own container with their own compute resources: 4 vCPUs and 16GB of RAM.
+
+These runs may take up to 3 minutes to start while these resources are provisioned.
+
+When launching runs manually, select `Isolate run environment` in the Launchpad to launch an isolated runs.
+
+:::note
+
+If non-isolated runs are disabled (see the section below), the toggle won't appear and all runs will be isolated.
+:::
+
+## Non-isolated runs
+
+This can be enabled or disabled in deployment settings with
+
+```yaml
+non_isolated_runs:
+ enabled: True
+```
+
+Non-isolated runs provide a faster start time by using a standing, shared container for each code location.
+
+They have fewer compute resources: 0.25 vCPUs and 1GB of RAM. These resources are shared with other processes running within a code location like sensors. As a result, it's recommended to use isolated runs for compute intensive jobs and asset materializations.
+
+While launching runs from the Launchpad, un-check `Isolate run environment`. When materializing an asset, shift-click `Materialize all` to open the Launchpad and un-check the `Isolate run environment` checkbox.
+
+:::warning
+
+To avoid crashes and memory exhaustion, only one non-isolated run will execute at once by default. While a non-isolated run is in progress, the Launchpad will only allow isolated runs to be launched.
+
+This limit can be configured in [deployment settings](/dagster-plus/deployment/management/deployments/deployment-settings-reference).
+
+```yaml
+non_isolated_runs:
+ enabled: True
+ max_concurrent_non_isolated_runs: 1
+```
+
+:::
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/runtime-environment.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/runtime-environment.md
new file mode 100644
index 0000000000000..47b3ed3c88d40
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/runtime-environment.md
@@ -0,0 +1,245 @@
+---
+title: 'Serverless runtime environment'
+sidebar_label: 'Runtime environment'
+sidebar_position: 100
+---
+
+By default, Dagster+ Serverless will package your code as PEX files and deploys them on Docker images. Using PEX files significantly reduces the time to deploy since it does not require building a new Docker image and provisioning a new container for every code change. However you are able to customize the Serverless runtime environment in various ways:
+
+- [Add dependencies](#add-dependencies)
+- [Use a different Python version](#use-a-different-python-version)
+- [Use a different base image](#use-a-different-base-image)
+- [Include data files](#include-data-files)
+- [Disable PEX deploys](#disable-pex-deploys)
+- [Use private Python packages](#use-private-python-packages)
+
+## Add dependencies
+
+You can add dependencies by including the corresponding Python libraries in your Dagster project's `setup.py` file. These should follow [PEP 508](https://peps.python.org/pep-0508/).
+
+
+
+You can also use a tarball to install a dependency, such as if `pip` is unable to resolve a package using `dependency_links`. For example, `soda` and `soda-snowflake` provide tarballs that you can include in the `install_requires` section:
+
+```python
+from setuptools import find_packages, setup
+
+setup(
+ name="quickstart_etl",
+ packages=find_packages(exclude=["quickstart_etl_tests"]),
+ install_requires=[
+ "dagster",
+ "boto3",
+ "pandas",
+ "matplotlib",
+ 'soda @ https://pypi.cloud.soda.io/packages/soda-1.6.2.tar.gz',
+ 'soda-snowflake @ https://pypi.cloud.soda.io/packages/soda_snowflake-1.6.2.tar.gz'
+ ],
+ extras_require={"dev": ["dagster-webserver", "pytest"]},
+)
+```
+
+To add a package from a private GitHub repository, see [Use private Python packages](#use-private-python-packages)
+
+## Use a different Python version
+
+Python versions 3.9 through 3.12 are all supported for Serverless deployments. You can specify the Python version you want to use in your GitHub or GitLab workflow, or by using the `dagster-cloud` CLI.
+
+
+
+In your `.github/workflows/deploy.yml` file, update the `PYTHON_VERSION` environment variable with your desired Python version:
+
+
+
+
+1. Open your `.gitlab-ci.yml` file. If your `.gitlab-ci.yml` contains an `include` with a link to a Dagster provided CI/CD template:
+
+
+ Follow the link and replace the contents of your `.gitlab-ci.yml` with the YAML document at the link address. Otherwise, continue to the next step.
+
+3. Update the `PYTHON_VERSION` environment variable with your desired Python version
+
+
+
+
+
+You can specify the Python version when you deploy your code with the `dagster-cloud serverless deploy-python-executable` command:
+```bash
+dagster-cloud serverless deploy-python-executable --python-version=3.11 --location-name=my_location
+```
+
+
+
+## Use a different base image
+
+Dagster+ runs your code on a Docker image that we build as follows:
+
+- The standard Python "slim" Docker image, such as python:3.10-slim is used as the base
+- The dagster-cloud[serverless] module installed in the image
+
+You can [add dependencies](#add-dependencies) in your `setup.py` file, but when that is not possible you can build and upload a custom base image that will be used to run your Python code:
+
+:::note
+Setting a custom base image isn't supported for GitLab CI/CD workflows out of the box, but you can write a custom GitLab CI/CD yaml file that implements the manual steps noted.
+:::
+
+1. Include `dagster-cloud[serverless]` as a dependency in your Docker image by adding the following line to your `Dockerfile`:
+ ```
+ RUN pip install "dagster-cloud[serverless]"
+ ```
+2. Build your Docker image, using your usual Docker toolchain.
+3. Upload your Docker image to Dagster+ using the `upload-base-image` command. This command will print out the tag used in Dagster+ to identify your image:
+
+ ```bash
+ $ dagster-cloud serverless upload-base-image local-image:tag
+
+ ...
+ To use the uploaded image run: dagster-cloud serverless deploy-python-executable ... --base-image-tag=sha256_518ad2f92b078c63c60e89f0310f13f19d3a1c7ea9e1976d67d59fcb7040d0d6
+ ```
+
+4. Specify this base image tag in you GitHub workflow, or using the `dagster-cloud` CLI:
+
+
+ In your `.github/workflows/deploy.yml` file, add the `SERVERLESS_BASE_IMAGE_TAG` environment variable and set it to the tag printed out in the previous step:
+
+
+
+
+
+ You can specify the base image when you deploy your code with the `dagster-cloud serverless deploy-python-executable` command:
+
+ ```bash
+ dagster-cloud serverless deploy-python-executable \
+ --base-image-tag=sha256_518ad2f92b078c63c60e89f0310f13f19d3a1c7ea9e1976d67d59fcb7040d0d6 \
+ --location-name=my_location
+ ```
+
+
+
+
+## Include data files
+
+To add data files to your deployment, use the [Data Files Support](https://setuptools.pypa.io/en/latest/userguide/datafiles.html) built into Python's `setup.py`. This requires adding a `package_data` or `include_package_data` keyword in the call to `setup()` in `setup.py`. For example, given this directory structure:
+
+```
+- setup.py
+- quickstart_etl/
+ - __init__.py
+ - definitions.py
+ - data/
+ - file1.txt
+ - file2.csv
+```
+
+If you want to include the data folder, modify your `setup.py` to add the `package_data` line:
+
+
+## Disable PEX deploys
+
+You have the option to disable PEX-based deploys and deploy using a Docker image instead of PEX. You can disable PEX in your GitHub or GitLab workflow, or by using the `dagster-cloud` CLI.
+
+
+
+In your `.github/workflows/deploy.yml` file, update the `ENABLE_FAST_DEPLOYS` environment variable to `false`:
+
+
+
+
+1. Open your `.gitlab-ci.yml` file. If your `.gitlab-ci.yml` contains an `include` with a link to a Dagster provided CI/CD template:
+
+
+ Follow the link and replace the contents of your `.gitlab-ci.yml` with the YAML document at the link address. Otherwise, continue to the next step.
+
+3. Update the `DISABLE_FAST_DEPLOYS` variable to `true`
+
+
+
+
+
+You can deploy using a Docker image instead of PEX by using the `dagster-cloud serverless deploy` command instead of the `dagster-cloud-serverless deploy-python-executable` command:
+
+```bash
+dagster-cloud serverless deploy --location-name=my_location
+```
+
+
+
+
+You can customize the Docker image using lifecycle hooks or by customizing the base image:
+
+
+
+This method is the easiest to set up, and doesn't require setting up any additional infrastructure.
+
+In the root of your repo, you can provide two optional shell scripts: `dagster_cloud_pre_install.sh` and `dagster_cloud_post_install.sh`. These will run before and after Python dependencies are installed. They're useful for installing any non-Python dependencies or otherwise configuring your environment.
+
+
+
+This method is the most flexible, but requires setting up a pipeline outside of Dagster to build a custom base image.
+
+:::note
+Setting a custom base image isn't supported for GitLab CI/CD workflows out of the box, but you can write a custom GitLab CI/CD yaml file that implements the manual steps noted.
+:::
+
+1. Build you base image
+2. Specify this base image tag in your GitHub workflow, or using the `dagster-cloud` CLI:
+
+
+ In your `.github/workflows/deploy.yml` file, add the `SERVERLESS_BASE_IMAGE_TAG` environment variable and set it to the tag printed out in the previous step:
+
+
+
+
+ You can specify the base image when you deploy your code with the `dagster-cloud serverless deploy` command:
+ ```bash
+ dagster-cloud serverless deploy --base-image=my_base_image:latest --location-name=my_location
+ ```
+
+
+
+
+
+
+## Use private Python packages
+
+If you use PEX deploys in your workflow (`ENABLE_FAST_DEPLOYS: 'true'`), the following steps can install a package from a private GitHub repository, e.g. `my-org/private-repo`, as a dependency:
+
+1. In your `deploy.yml` file, add the following to the top of `steps:` section in the `dagster-cloud-default-deploy` job.
+
+ ```yaml
+ - name: Checkout internal repository
+ uses: actions/checkout@v3
+ with:
+ token: ${{ secrets.GH_PAT }}
+ repository: my-org/private-repo
+ path: deps/private-repo
+ ref: some-branch # optional to check out a specific branch
+
+ - name: Build a wheel
+ # adjust the `cd` command to cd into the directory with setup.py
+ run: >
+ cd deps/private-repo &&
+ python setup.py bdist_wheel &&
+ mkdir -p $GITHUB_WORKSPACE/deps &&
+ cp dist/*whl $GITHUB_WORKSPACE/deps
+
+ # If you have multiple private packages, the above two steps should be repeated for each but the following step is only
+ # needed once
+ - name: Configure dependency resolution to use the wheel built above
+ run: >
+ echo "[global]" > $GITHUB_WORKSPACE/deps/pip.conf &&
+ echo "find-links = " >> $GITHUB_WORKSPACE/deps/pip.conf &&
+ echo " file://$GITHUB_WORKSPACE/deps/" >> $GITHUB_WORKSPACE/deps/pip.conf &&
+ echo "PIP_CONFIG_FILE=$GITHUB_WORKSPACE/deps/pip.conf" > $GITHUB_ENV
+ ```
+
+2. Create a GitHub personal access token and set it as the `GH_PAT` secret for your Actions.
+3. In your Dagster project's `setup.py` file, add your package name to the `install_requires` section:
+ ```python
+ install_requires=[
+ "dagster",
+ "dagster-cloud",
+ "private-package", # add this line - must match your private Python package name
+ ```
+
+Once the `deploy.yml` is updated and changes pushed to your repo, then any subsequent code deploy should checkout your private repository, build the package and install it as a dependency in your Dagster+ project. Repeat the above steps for your `branch_deployments.yml` if needed.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/security.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/security.md
new file mode 100644
index 0000000000000..0e4e0dcd451bf
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/deployment-types/serverless/security.md
@@ -0,0 +1,37 @@
+---
+title: Serverless security & data protection
+sidebar_label: 'Security & data protection'
+sidebar_position: 500
+---
+
+Unlike [Hybrid deployments](/dagster-plus/deployment/deployment-types/hybrid), Serverless deployments on Dagster+ require direct access to your data, secrets and source code.
+
+Secrets and source code are built into the image directly. Images are stored in a per-customer container registry with restricted access.
+User code is securely sandboxed using modern container sandboxing techniques.
+
+All production access is governed by industry-standard best practices which are regularly audited.
+
+## I/O management in Serverless
+
+:::warning
+The default I/O manager cannot be used if you are a Serverless user who:
+
+- Works with personally identifiable information (PII)
+- Works with private health information (PHI)
+- Has signed a business association agreement (BAA), or
+- Are otherwise working with data subject to GDPR or other such regulations
+ :::
+
+In Serverless, code that uses the default [I/O manager](/guides/build/io-managers/) is automatically adjusted to save data in Dagster+ managed storage. This automatic change is useful because the Serverless filesystem is ephemeral, which means the default I/O manager wouldn't work as expected.
+
+However, this automatic change also means potentially sensitive data could be **stored** and not just processed or orchestrated by Dagster+.
+
+To prevent this, you can use [another I/O manager](/guides/build/io-managers/#built-in) that stores data in your infrastructure or [adapt your code to avoid using an I/O manager](/guides/build/io-managers/#before-you-begin).
+
+:::note
+You must have [boto3](https://pypi.org/project/boto3/) or `dagster-cloud[serverless]` installed as a project dependency otherwise the Dagster+ managed storage can fail and silently fall back to using the default I/O manager.
+:::
+
+## Adding environment variables and secrets
+
+Often you'll need to securely access secrets from your jobs. Dagster+ supports several methods for adding secrets—refer to the [Dagster+ environment variables documentation](/dagster-plus/deployment/management/environment-variables) for more information.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/dagster-cloud-cli/dagster-cloud-cli-reference.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/dagster-cloud-cli/dagster-cloud-cli-reference.md
new file mode 100644
index 0000000000000..6c7521800425f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/dagster-cloud-cli/dagster-cloud-cli-reference.md
@@ -0,0 +1,24 @@
+---
+title: dagster-cloud CLI reference
+sidebar_position: 200
+---
+
+## Custom configuration file path
+
+Point the CLI at an alternate config location by specifying the `DAGSTER_CLOUD_CLI_CONFIG` environment variable.
+
+## Environment variables and CLI options
+
+Environment variables and CLI options can be used in place of or to override the CLI configuration file.
+
+The priority of these items is as follows:
+
+- **CLI options** - highest
+- **Environment variables**
+- **CLI configuration** - lowest
+
+| Setting | Environment variable | CLI flag | CLI config value |
+| ------------ | ---------------------------- | ---------------------- | -------------------- |
+| Organization | `DAGSTER_CLOUD_ORGANIZATION` | `--organization`, `-o` | `organization` |
+| Deployment | `DAGSTER_CLOUD_DEPLOYMENT` | `--deployment`, `-d` | `default_deployment` |
+| User Token | `DAGSTER_CLOUD_API_TOKEN` | `--user-token`, `-u` | `user_token` |
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/dagster-cloud-cli/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/dagster-cloud-cli/index.md
new file mode 100644
index 0000000000000..7569b8915f65a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/dagster-cloud-cli/index.md
@@ -0,0 +1,10 @@
+---
+title: dagster-cloud CLI
+sidebar_position: 40
+---
+
+The dagster-cloud CLI is a command-line toolkit designed to work with Dagster+.
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring.md
new file mode 100644
index 0000000000000..e603d98a2c9ac
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring.md
@@ -0,0 +1,131 @@
+---
+title: Installing and configuring the dagster-cloud CLI
+sidebar_position: 100
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+The `dagster-cloud` CLI is a command-line toolkit designed to work with Dagster+.
+
+In this guide, we'll cover how to install and configure the `dagster-cloud` CLI, get help, and use some helpful environment variables and CLI options.
+
+## Installing the CLI
+
+The Dagster+ Agent library is available in PyPi. To install, run:
+
+```shell
+pip install dagster-cloud
+```
+
+Refer to the [configuration section](#configuring-the-cli) for next steps.
+
+### Completions
+
+Optionally, you can install command-line completions to make using the `dagster-cloud` CLI easier.
+
+To have the CLI install these completions to your shell, run:
+
+```shell
+dagster-cloud --install-completion
+```
+
+To print out the completion for copying or manual installation:
+
+```shell
+dagster-cloud --show-completion
+```
+
+## Configuring the CLI
+
+The recommended way to set up your CLI's config for long-term use is through the configuration file, located by default at `~/.dagster_cloud_cli/config`.
+
+### Setting up the configuration file
+
+Set up the config file:
+
+```shell
+dagster-cloud config setup
+```
+
+Select your authentication method. **Note**: Browser authentication is the easiest method to configure.
+
+
+BROWSER AUTHENTICATION
+
+The easiest way to set up is to authenticate through the browser.
+
+```shell
+$ dagster-cloud config setup
+? How would you like to authenticate the CLI? (Use arrow keys)
+ » Authenticate in browser
+ Authenticate using token
+Authorized for organization `hooli`
+
+? Default deployment: prod
+```
+
+When prompted, you can specify a default deployment. If specified, a deployment won't be required in subsequent `dagster-cloud` commands. The default deployment for a new Dagster+ organization is `prod`.
+
+
+
+
+TOKEN AUTHENTICATION
+
+Alternatively, you may authenticate using a user token. Refer to the [User tokens guide](/dagster-plus/deployment/management/tokens) for more info.
+
+```shell
+$ dagster-cloud config setup
+? How would you like to authenticate the CLI? (Use arrow keys)
+ Authenticate in browser
+ » Authenticate using token
+
+? Dagster+ organization: hooli
+? Dagster+ user token: *************************************
+? Default deployment: prod
+```
+
+When prompted, specify the following:
+
+- **Organization** - Your organization name as it appears in your Dagster+ URL. For example, if your Dagster+ instance is `https://hooli.dagster.cloud/`, this would be `hooli`.
+- **User token** - The user token.
+- **Default deployment** - **Optional**. A default deployment. If specified, a deployment won't be required in subsequent `dagster-cloud` commands. The default deployment for a new Dagster+ organization is `prod`.
+
+
+
+### Viewing and modifying the configuration file
+
+To view the contents of the CLI configuration file, run:
+
+```shell
+$ dagster-cloud config view
+
+default_deployment: prod
+organization: hooli
+user_token: '*******************************8214fe'
+```
+
+Specify the `--show-token` flag to show the full user token.
+
+To modify the existing config, re-run:
+
+```shell
+dagster-cloud config setup
+```
+
+## Toggling between deployments
+
+To quickly toggle between deployments, run:
+
+```shell
+dagster-cloud config set-deployment
+```
+
+## Getting help
+
+To view help options in the CLI:
+
+```shell
+dagster-cloud --help
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/deployments/deployment-settings-reference.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/deployments/deployment-settings-reference.md
new file mode 100644
index 0000000000000..4c4ccd17f05f8
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/deployments/deployment-settings-reference.md
@@ -0,0 +1,118 @@
+---
+title: "Full deployment settings"
+sidebar_position: 200
+---
+
+This reference describes the settings that can be configured for full deployments in Dagster+.
+
+Refer to the [Managing deployments in Dagster+ guide](managing-deployments) for information about configuring settings in the Dagster+ interface or using the `dagster-cloud` CLI.
+
+## Settings schema
+
+Settings are formatted using YAML. For example:
+
+```yaml
+run_queue:
+ max_concurrent_runs: 10
+ tag_concurrency_limits:
+ - key: "database"
+ value: "redshift"
+ limit: 5
+
+run_monitoring:
+ start_timeout_seconds: 1200
+ cancel_timeout_seconds: 1200
+ max_runtime_seconds: 7200
+
+run_retries:
+ max_retries: 0
+
+sso_default_role: EDITOR
+```
+
+## Settings
+
+For each deployment, you can configure settings for:
+
+- [Run queue](#run-queue-run_queue)
+- [Run monitoring](#run-monitoring-run_monitoring)
+- [Run retries](#run-retries-run_retries)
+- [SSO default role](#sso-default-role)
+- [Non-isolated runs](#non-isolated-runs)
+
+### Run queue (run_queue)
+
+The `run_queue` settings allow you to specify how many runs can execute concurrently in the deployment.
+
+```yaml
+run_queue:
+ max_concurrent_runs: 10
+ tag_concurrency_limits:
+ - key: "database"
+ value: "redshift"
+ limit: 5
+```
+
+| Property | Description |
+| --- | --- |
+| run_queue.max_concurrent_runs | The maximum number of runs that are allowed to be in progress at once. Set to 0 to stop any runs from launching. Negative values aren't permitted.
|
+| run_queue.tag_concurrency_limits | A list of limits applied to runs with particular tags.
**Defaults** - `[]`
Each list item may have the following properties:
`key`
`value`
If defined, the `limit` is applied only to the `key-value` pair.
If set to a dict with applyLimitPerUniqueValue: true, the `limit` is applied to the number of unique values for the `key`.
If set to a dict with `applyLimitPerUniqueValue: true`, the limit is applied to the number of unique values for the `key`.
`limit`
|
+
+### Run monitoring (run_monitoring)
+
+The `run_monitoring` settings allow you to define how long Dagster+ should wait for runs to start before making them as failed, or to terminate before marking them as canceled.
+
+```yaml
+run_monitoring:
+ start_timeout_seconds: 1200
+ cancel_timeout_seconds: 1200
+ max_runtime_seconds: 7200
+```
+
+| Property | Description |
+| --- | --- |
+| run_monitoring.start_timeout_seconds | The number of seconds that Dagster+ will wait after a run is launched for the process or container to start executing. After the timeout, the run will fail. This prevents runs from hanging in `STARTING` indefinitely when the process or container doesn't start.
**Default** - `1200` (20 minutes)
|
+| run_monitoring.cancel_timeout_seconds | The number of seconds that Dagster+ will wait after a run termination is initiated for the process or container to terminate. After the timeout, the run will move into a `CANCELED` state. This prevents runs from hanging in `CANCELING` indefinitely when the process or container doesn't terminate cleanly.
**Default** - `1200` (20 minutes)
|
+| run_monitoring.max_runtime_seconds | The number of seconds that Dagster+ will wait after a run is moved into a `STARTED` state for the run to complete. After the timeout, the run will be terminated and moved into a `FAILURE` state. This prevents runs from hanging in `STARTED` indefinitely if the process is hanging.
**Default** - `No limit`
|
+
+### Run retries (run_retries)
+
+The `run_retries` settings allow you to define how Dagster+ handles retrying failed runs in the deployment.
+
+```yaml
+run_retries:
+ max_retries: 0
+```
+
+| Property | Descripton |
+| --- | --- |
+| run_retries.max_retries | The maximum number of times Dagster+ should attempt to retry a failed run. Dagster+ will use the default if this setting is undefined.
**Default** - `0`
|
+| run_retries.retry_on_asset_or_op_failure | Whether to retry runs that failed due to assets or ops in the run failing. Set this to false if you only want to retry failures that occur due to the run worker crashing or unexpectedly terminating, and instead rely on op or asset-level retry policies to retry assert or op failures. Setting this field to false will only change retry behavior for runs on dagster version 1.6.7 or greater.
**Default** - `0`
|
+
+### SSO default role
+
+{/* dagster-plus/account/managing-users/managing-user-roles-permissions#user-permissions-reference */}
+The `sso_default_role` setting lets you configure the default role on the deployment which is granted to new users that log in via SSO. For more information on available roles, see the [Dagster+ permissions reference](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions).
+
+```yaml
+sso_default_role: EDITOR
+```
+
+| Property | Descripton |
+| --- | --- |
+| sso_default_role | If SAML SSO is enabled, this is the default role that will be assigned to Dagster+ users for this deployment. If SAML SSO is not enabled, this setting is ignored.
**Default** - `Viewer`
|
+
+### Non-isolated runs
+
+Configure [non-isolated runs](/dagster-plus/deployment/deployment-types/serverless/) on your deployment.
+
+```yaml
+non_isolated_runs:
+ enabled: True
+ max_concurrent_non_isolated_runs: 1
+```
+
+| Property | Descripton |
+| --- | --- |
+| enabled | If enabled, the `Isolate run environment` checkbox will appear in the Launchpad.
**Default** - `true`
|
+| max_concurrent_non_isolated_runs | A limit for how many non-isolated runs to launch at once. Once this limit is reached, the checkbox will be greyed out and all runs will be isolated. This helps to avoid running out of RAM on the code location server.
**Default** - `1`
|
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/deployments/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/deployments/index.md
new file mode 100644
index 0000000000000..9edb2d37ebea6
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/deployments/index.md
@@ -0,0 +1,15 @@
+---
+title: "Full deployments"
+sidebar_position: 50
+---
+
+In Dagster+, there are two types of deployments:
+
+- [**Branch deployments**](/dagster-plus/features/ci-cd/branch-deployments), which are temporary deployments built for testing purposes.
+- **Full deployments**, which are persistent, fully-featured deployments intended to perform actions on a recurring basis.
+
+This section focuses on **full deployments**, hereafter referred to as "deployments".
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/deployments/managing-deployments.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/deployments/managing-deployments.md
new file mode 100644
index 0000000000000..9af9730d95dfd
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/deployments/managing-deployments.md
@@ -0,0 +1,133 @@
+---
+title: "Managing full deployments in Dagster+"
+---
+
+Full deployments are standalone environments, allowing you to operate independent instances of Dagster with separately managed permissions.
+
+When a Dagster+ organization is created, a single deployment named `prod` will also be created. To create additional full deployments, you must sign up for a [Pro plan](https://dagster.io/pricing).
+
+Each full deployment can have one or multiple [code locations](/dagster-plus/deployment/code-locations).
+
+:::note Full deployments vs branch deployments
+
+In Dagster+, there are two types of deployments:
+
+- [**Branch deployments**](/dagster-plus/features/ci-cd/branch-deployments), which are temporary deployments built for testing purposes. We recommend using branch deployments to test your changes, even if you're able to create additional deployments. Branch deployments are available for all Dagster+ users, regardless of plan.
+- **Full deployments**, which are persistent, fully-featured deployments intended to perform actions on a recurring basis.
+
+This guide focuses on **full deployments**, hereafter referred to as **deployments**.
+
+:::
+
+## Viewing and switching deployments
+
+In Dagster+, you can view and switch between deployments using the **deployment switcher**:
+
+![The deployment switcher in Dagster+](/images/dagster-plus/full-deployments/deployment-switcher.png)
+
+To view all deployments, click **View all deployments**.
+
+## Creating deployments
+
+:::note
+
+[Organization Admin permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) are required to create deployments. Additionally, note that creating multiple deployments requires a [Pro plan](https://dagster.io/pricing).
+
+:::
+
+To create a deployment:
+
+1. Sign in to your Dagster+ account.
+2. Access the **Deployments** page using one of the following options:
+ - Click the **deployment switcher > View all deployments**.
+ - Click **your user icon > Organization Settings > Deployments**.
+3. Click the **+ New deployment** button.
+4. In the modal that displays, fill in the following:
+ - **Name** - Enter a name for the deployment.
+ - **Initial deployment permissions** - Select the permissions you want to use to create the deployment:
+ - **Empty permissions** - Creates the deployment with an empty set of permissions. **Note**: Only Organization Admins will be able to manage the deployment until other uses are granted Admin or Editor permissions.
+ - **Copy from** - Creates the deployment using permissions duplicated from an existing deployment.
+5. When finished, click **Create deployment**.
+
+## Deleting deployments
+
+:::note
+
+[Organization Admin permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) are required to delete deployments. Additionally, note that deleting a deployment also deletes all its associated data, including code locations, jobs, schedules, and sensors.
+
+:::
+
+To delete a deployment:
+
+1. Sign in to your Dagster+ account.
+2. Access the **Deployments** page using one of the following options:
+ - Click the **deployment switcher > View all deployments**.
+ - Click the **deployment switcher**, then the **gear icon** next to the deployment.
+ - Click **your user icon > Organization Settings > Deployments**.
+3. Click the **Delete** button next to the deployment you want to delete.
+4. When prompted, confirm the deletion.
+
+## Configuring deployment settings
+
+:::note
+
+[Editor permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) are required to modify deployment settings.
+
+:::
+
+Deployment settings can be configured in the Dagster+ interface or using the `dagster-cloud` CLI. Refer to the [deployment settings reference](/dagster-plus/deployment/management/deployments/deployment-settings-reference) for more info about individual settings.
+
+
+
+ To configure deployment settings in the Dagster+ UI:
+
+ 1. Sign in to your Dagster+ account.
+ 2. Access the **Deployments** page using one of the following:
+ - Click the **deployment switcher > View all deployments**.
+ - Click the **deployment switcher**, then the **gear icon** next to the deployment.
+ - Click **your user icon > Organization Settings > Deployments**.
+ 3. Click the **Settings** button next to the deployment you want to configure.
+ 4. In the window that displays, configure settings for the deployment.
+ 5. When finished, click **Save deployment settings**.
+
+
+
+:::note
+
+`dagster-cloud` 0.13.14 or later must be installed to run the CLI. Agent and/or job code doesn't need to be upgraded.
+
+:::
+
+Create a file with the settings you'd like to configure. For example:
+
+```yaml
+# my-settings.yaml
+
+run_queue:
+ max_concurrent_runs: 10
+ tag_concurrency_limits:
+ - key: "special-runs"
+ limit: 5
+
+run_monitoring:
+ start_timeout_seconds: 1200
+ cancel_timeout_seconds: 1200
+
+run_retries:
+ max_retries: 0
+```
+
+Use the CLI to upload the settings file:
+
+```shell
+dagster-cloud deployment settings set-from-file my-settings.yaml
+```
+
+This will replace all of your configured settings. Any that are not specified will resort to their default values. You also use the CLI to read your current settings, including the default values:
+
+```shell
+dagster-cloud deployment settings get
+```
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/agent-config.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/agent-config.md
new file mode 100644
index 0000000000000..82b1716ebbb7f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/agent-config.md
@@ -0,0 +1,245 @@
+---
+title: "Setting environment variables using agent config"
+sidebar_position: 300
+sidebar_label: "Set with agent config"
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+In this guide, we'll walk you through setting environment variables for a Dagster+ [Hybrid deployment](/dagster-plus/deployment/deployment-types/hybrid) using the Hybrid agent's configuration.
+
+There are two ways to set environment variables:
+
+- **On a per-code location basis**, which involves modifying the `dagster_cloud.yaml` file. **Note**: This approach is functionally the same as [setting environment variables using the Dagster+ UI](/dagster-plus/deployment/management/environment-variables/dagster-ui). Values will pass through Dagster+.
+- **For a full deployment and all the code locations it contains**. This approach makes variables available for all code locations in a full Dagster+ deployment. As values are pulled from the user cluster, values will bypass Dagster+ entirely.
+
+## Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- A Dagster+ account using [Hybrid deployment](/dagster-plus/deployment/deployment-types/hybrid/)
+- An existing [Hybrid agent](/dagster-plus/deployment/deployment-types/hybrid/#dagster-hybrid-agents)
+- **Editor**, **Admin**, or **Organization Admin** permissions in Dagster+
+
+## Setting environment variables for a code location
+
+:::note
+
+Setting environment variables require **Organization Admin**, **Admin**, or **Editor** permissions on Dagster+.
+
+If you're a Dagster **Editor** or **Admin**, you can only set environment variables for full deployments where you're an **Editor** or **Admin**.
+
+:::
+
+You can set environment variables for specific code locations by adding them to your agent's configuration in your project's [`dagster_cloud.yaml` file](/dagster-plus/deployment/code-locations/dagster-cloud-yaml.md). The `container_context` property in this file sets the variables in the agent's environment.
+
+This approach is functionally the same as [setting environment variables using the Dagster+ UI](/dagster-plus/deployment/management/environment-variables/dagster-ui).
+
+How `container_context` is configured depends on the agent type. Click the tab for your agent type to view instructions.
+
+
+
+
+Using the `container_context.ecs.env_vars` and `container_context.ecs.secrets` properties, you can configure environment variables and secrets for a specific code location.
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ ecs:
+ env_vars:
+ - DATABASE_NAME=testing
+ - DATABASE_PASSWORD
+ secrets:
+ - name: "MY_API_TOKEN"
+ valueFrom: "arn:aws:secretsmanager:us-east-1:123456789012:secret:FOO-AbCdEf:token::"
+ - name: "MY_PASSWORD"
+ valueFrom: "arn:aws:secretsmanager:us-east-1:123456789012:secret:FOO-AbCdEf:password::"
+ secrets_tags:
+ - "my_tag_name"
+```
+
+| Key | Description |
+|--------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| `container_context.ecs.env_vars` | A list of keys or key-value pairs. If a value is not specified, it pulls from the agent task. E.g., `FOO_ENV_VAR` = `foo_value`, `BAR_ENV_VAR` = agent task value. |
+| `container_context.ecs.secrets` | Individual secrets using the [ECS API structure](https://docs.aws.amazon.com/AmazonECS/latest/APIReference/API_Secret.html). |
+| `container_context.ecs.secrets_tags` | A list of tag names; secrets tagged with these in AWS Secrets Manager will be environment variables. The variable name is the secret name, the value is the secret's value. |
+
+After you've modified `dagster_cloud.yaml`, redeploy the code location in Dagster+ to apply the changes:
+
+!["Highlighted Redeploy option in the dropdown menu next to a code location in Dagster+"](/images/dagster-plus/deployment/code-locations/redeploy-code-location.png)
+
+
+
+
+Using the `container_context.docker.env_vars` property, you can include environment variables and secrets in the Docker container associated with a specific code location. For example:
+
+```yaml
+# dagster_cloud.yaml
+locations:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ docker:
+ env_vars:
+ - DATABASE_NAME
+ - DATABASE_USERNAME=hooli_testing
+```
+
+The `container_context.docker.env_vars` property is a list, where each item can be either `KEY` or `KEY=VALUE`. If only `KEY` is specified, the value will be pulled from the local environment.
+
+After you've modified `dagster_cloud.yaml`, redeploy the code location in Dagster+ to apply the changes:
+
+![Highlighted Redeploy option in the dropdown menu next to a code location in Dagster+](/images/dagster-plus/deployment/code-locations/redeploy-code-location.png)
+
+
+
+
+Using the `container_context.k8s.env_vars` and `container_context.k8s.env_secrets` properties, you can specify environment variables and secrets for a specific code location. For example:
+
+```yaml
+# dagster_cloud.yaml
+
+locations:
+ - location_name: cloud-examples
+ image: dagster/dagster-cloud-examples:latest
+ code_source:
+ package_name: dagster_cloud_examples
+ container_context:
+ k8s:
+ env_vars:
+ - database_name # value pulled from agent's environment
+ - database_username=hooli_testing
+ env_secrets:
+ - database_password
+```
+
+ | Key | Description |
+ |---------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+ | `env_vars` | A list of environment variable names to inject into the job, formatted as `KEY` or `KEY=VALUE`. If only `KEY` is specified, the value will be pulled from the current process. |
+ | `env_secrets` | A list of secret names, from which environment variables for a job are drawn using `envFrom`. For more information, see the [Kubernetes](https://kubernetes.io/docs/tasks/inject-data-application/distribute-credentials-secure/#configure-all-key-value-pairs-in-a-secret-as-container-environment-variables). |
+
+After you've modified `dagster_cloud.yaml`, redeploy the code location in Dagster+ to apply the changes:
+
+![Highlighted Redeploy option in the dropdown menu next to a code location in Dagster+](/images/dagster-plus/deployment/code-locations/redeploy-code-location.png)
+
+
+
+
+## Setting environment variables for full deployments
+
+:::note
+
+If you're a Dagster **Editor** or **Admin**, you can only set environment variables for full deployments where you're an **Editor** or **Admin**.
+
+:::
+
+Setting environment variables for a full deployment will make the variables available for all code locations in the full deployment. Using this approach will pull variable values from your user cluster, bypassing Dagster+ entirely.
+
+Click the tab for your agent type to view instructions.
+
+
+
+
+To make environment variables accessible to a full deployment with an Amazon ECS agent, you'll need to modify the agent's CloudFormation template as follows:
+
+1. Sign in to your AWS account.
+
+2. Navigate to **CloudFormation** and open the stack for the agent.
+
+3. Click **Update**.
+
+4. Click **Edit template in designer**.
+
+5. In the section that displays, click **View in Designer**. The AWS template designer will display.
+
+6. In the section displaying the template YAML, locate the `AgentTaskDefinition` section:
+
+ ![Highlighted AgentTaskDefinition section of the AWS ECS agent CloudFormation template in the AWS Console](/images/dagster-plus/deployment/environment-variables/aws-ecs-cloudformation-template.png)
+
+
+7. In the `user_code_launcher.config` portion of the `AgentTaskDefinition` section, add the environment variables as follows:
+
+ ```yaml
+ user_code_launcher:
+ module: dagster_cloud.workspace.ecs
+ class: EcsUserCodeLauncher
+ config:
+ cluster: ${ConfigCluster}
+ subnets: [${ConfigSubnet}]
+ service_discovery_namespace_id: ${ServiceDiscoveryNamespace}
+ execution_role_arn: ${TaskExecutionRole.Arn}
+ task_role_arn: ${AgentRole}
+ log_group: ${AgentLogGroup}
+ env_vars:
+ - SNOWFLAKE_USERNAME=dev
+ - SNOWFLAKE_PASSWORD ## pulled from agent environment
+ ' > $DAGSTER_HOME/dagster.yaml && cat $DAGSTER_HOME/dagster.yaml && dagster-cloud agent run"
+ ```
+
+8. When finished, click the **Create Stack** button:
+
+ ![Highlighted Create Stack button in the AWS Console](/images/dagster-plus/deployment/environment-variables/aws-ecs-save-template.png)
+
+9. You'll be redirected back to the **Update stack** wizard, where the new template will be populated. Click **Next**.
+
+10. Continue to click **Next** until you reach the **Review** page.
+
+11. Click **Submit** to update the stack.
+
+
+
+
+To make environment variables accessible to a full deployment with a Docker agent, you'll need to modify your project's `dagster.yaml` file.
+
+In the `user_code_launcher` section, add an `env_vars` property as follows:
+
+```yaml
+# dagster.yaml
+
+user_code_launcher:
+ module: dagster_cloud.workspace.docker
+ class: DockerUserCodeLauncher
+ config:
+ networks:
+ - dagster_cloud_agent
+ env_vars:
+ - SNOWFLAKE_PASSWORD # value pulled from agent's environment
+ - SNOWFLAKE_USERNAME=dev
+```
+
+In `env_vars`, specify the environment variables as keys (`SNOWFLAKE_PASSWORD`) or key-value pairs (`SNOWFLAKE_USERNAME=dev`). If only `KEY` is provided, the value will be pulled from the agent's environment.
+
+
+
+
+To make environment variables available to a full deployment with a Kubernetes agent, you'll need to modify and upgrade the Helm chart's `values.yaml`.
+
+1. In `values.yaml`, add or locate the `workspace` value.
+
+2. Add an `envVars` property as follows:
+
+ ```yaml
+ # values.yaml
+
+ workspace:
+ envVars:
+ - SNOWFLAKE_PASSWORD # value pulled from agent's environment
+ - SNOWFLAKE_USERNAME=dev
+ ```
+
+3. In `envVars`, specify the environment variables as keys (`SNOWFLAKE_PASSWORD`) or key-value pairs (`SNOWFLAKE_USERNAME=dev`). If only `KEY` is provided, the value will be pulled from the local (agent's) environment.
+
+4. Upgrade the Helm chart.
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/built-in.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/built-in.md
new file mode 100644
index 0000000000000..87ef437a5d6e2
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/built-in.md
@@ -0,0 +1,35 @@
+---
+title: "Built-in environment variables"
+sidebar_position: 100
+sidebar_label: "Built-in variables"
+---
+
+Dagster+ provides a set of built-in, automatically populated environment variables, such as the name of a deployment or details about a branch deployment commit, that can be used to modify behavior based on environment.
+
+### All deployment variables
+
+The following variables are available in every deployment of your Dagster+ instance.
+
+| Key | Value |
+|---|---|
+| `DAGSTER_CLOUD_DEPLOYMENT_NAME` | The name of the Dagster+ deployment.
**Example:** `prod`. |
+| `DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT` | `1` if the deployment is a branch deployment. |
+
+
+### Branch deployment variables
+
+The following environment variables are available only in a [branch deployment](/dagster-plus/features/ci-cd/branch-deployments/).
+
+For every commit made to a branch, the following environment variables are available:
+
+| Key | Value |
+|---|---|
+| `DAGSTER_CLOUD_GIT_SHA` | The SHA of the commit. |
+| `DAGSTER_CLOUD_GIT_TIMESTAMP` | The Unix timestamp in seconds when the commit occurred.
**Example:** `1724871941` |
+| `DAGSTER_CLOUD_GIT_AUTHOR_EMAIL` | The email of the git user who authored the commit. |
+| `DAGSTER_CLOUD_GIT_AUTHOR_NAME` | The name of the git user who authored the commit. |
+| `DAGSTER_CLOUD_GIT_MESSAGE` | The message associated with the commit. |
+| `DAGSTER_CLOUD_GIT_BRANCH` | The name of the branch associated with the commit. |
+| `DAGSTER_CLOUD_GIT_REPO` | The name of the repository associated with the commit. |
+| `DAGSTER_CLOUD_PULL_REQUEST_ID` | The ID of the pull request associated with the commit. |
+| `DAGSTER_CLOUD_PULL_REQUEST_STATUS` | The status of the pull request at the time of the commit.
**Possible values:** `OPEN`, `CLOSED`, and `MERGED`. |
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/dagster-ui.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/dagster-ui.md
new file mode 100644
index 0000000000000..086b6b4af3c85
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/dagster-ui.md
@@ -0,0 +1,115 @@
+---
+title: "Setting environment variables with the Dagster+ UI"
+sidebar_position: 200
+sidebar_label: "Set with Dagster+ UI"
+---
+
+Environment variable are key-value pairs that are set outside of your source code. Using environment variables lets you dynamically change the behavior of your application without modifying source code and securely configured secrets.
+
+Dagster supports several approaches for [accessing environment variable in your code](/guides/deploy/using-environment-variables-and-secrets). You can also set environment variables in several ways, but this guide will focus on the Dagster+ UI.
+
+
+ Prerequisites
+
+To configure environment variables in the Dagster+ UI , you'll need:
+
+- **Organization Admin**, **Admin**, or **Editor** permissions for your Dagster+ account
+- To be using Dagster version 1.0.17 or later
+
+
+
+## Overview
+
+### Storage and encryption
+
+To securely store environment variables defined using the Dagster+ UI, Dagster+ uses [Amazon Key Management Services (KMS)](https://docs.aws.amazon.com/kms/index.html) and [envelope encryption](https://docs.aws.amazon.com/kms/latest/developerguide/concepts.html#enveloping). Envelope encryption is a multi-layered approach to key encryption. Plaintext data is encrypted using a data key, and then the data under the data key is encrypted under another key.
+
+![Dagster+ encryption key hierarchy diagram](/images/dagster-plus/deployment/environment-variables/encryption-key-hierarchy.png)
+
+### Scope
+
+By default, new environment variables set in the Dagster+ UI default to all deployments and all code locations. When creating or modifying an environment variable, you'll be prompted to select the deployment(s) to which you want to scope the variable:
+
+- **Local:** - Variables with this scope will be included when downloading variables to a local `.env` file.
+- **Full deployment:** - Variables with this scope will be available to selected code locations in the full deployment.
+- **Branch deployments:** - Variables with this scope will be available to selected code locations in branch deployments.
+
+:::note
+
+Environment variables will be read-only for branch deployments viewed in Dagster+. Environment variables must be managed in the branch deployment's parent full deployment, which will usually be production.
+
+:::
+
+### Reserved variables
+
+[Built-in (system) Dagster+ environment variables](built-in) are reserved and therefore unavailable for use. You will see an error in Dagster+ if you use a built-in variable name.
+
+## Adding environment variables \{#add}
+
+Before you begin, use the deployment switcher to select the right deployment.
+
+1. Click the **+ Add environment variable** button.
+2. In the modal that displays, fill in the following:
+ - **Name** - Enter a name for the environment variable. This is how the variable will be referenced in your code.
+ - **Value** - Enter a value for the environment variable.
+ - **Deployment Scope** - select the deployment(s) where the variable should be accessible:
+ - **Full deployment** - The variable will be available to selected code locations in the full deployment.
+ - **Branch deployments** - The variable will be available to selected code locations in Branch Deployments.
+ - **Local** - If selected, the variable will be included when [exporting environment variables to a local `.env` file](#export).
+ - **Code Location Scope** - select the code location(s) where the variable should be accessible. At least one code location is required.
+
+![Create new environment variable dialog window in Dagster+](/images/dagster-plus/deployment/environment-variables/create-new-variable-in-ui.png)
+
+3. Click **Save**
+
+## Editing environment variables \{#edit}
+
+On the **Environment variables** page, edit an environment variable by clicking the **Edit** button in the **Actions** column.
+
+## Deleting environment variables \{#delete}
+
+On the **Environment variables** page, delete an environment variable by clicking the **Trash icon** in the **Actions** column.
+
+## Viewing environment variable values \{#view}
+
+On the **Environment variables** page, view an environment variable by clicking the **eye icon** in the **Value** column. To hide the value, click the **eye icon** again.
+
+:::note
+Viewing an environment variable only reveals the value to you. It doesn't show the value in plaintext to all users. If you navigate away from the environment variables page or reload the page, the value will be hidden again.
+:::
+
+## Exporting environment variables locally \{#export}
+
+1. On the **Environment variables** page, click the **arrow menu** to the right of the **+ Add environment variable** button.
+2. Click **Download local environment variables**.
+3. A file named `env.txt` will be downloaded.
+
+To use the downloaded environment variables for local Dagster development:
+
+1. Rename the downloaded `env.txt` file to `.env`.
+2. Move the file to the directory where you run `dagster dev` or `dagster-webserver`.
+3. Run `dagster dev`.
+
+If the environment variables were loaded successfully, you'll see a log message that begins with `Loaded environment variables from .env file`.
+
+## Setting environment-dependent variable values \{#environment-dependent-values}
+
+You can create multiple instances of the same environment variable key with different values, allowing you to provide different values to different deployment environments. For example, you may want to use different Snowflake credentials for your production deployment than in branch deployments.
+
+When you [add an environment variable](#add), you can select the deployment scope and code location scope for the environment variable. You can create multiple environment variables with different values and different scopes to customize the values in different deployment environments.
+
+For example, if you wanted to provide different Snowflake passwords for your production and branch deployments, you would make two environment variables with the same key:
+
+- For the **production** environment variable:
+ - Set the value as the production password, and
+ - Check only the **Full deployment** box
+- For the **branch deployment** environment variable:
+ - Set the value as the branch deployment password, and
+ - Check only the **Branch deployments** box
+
+![Example SNOWFLAKE_PASSWORD variables configured with different values based on deployment](/images/dagster-plus/deployment/environment-variables/same-variable-diff-scope.png)
+
+## Next steps
+
+- Learn how to [access environment variables in Dagster code](/guides/deploy/using-environment-variables-and-secrets#accessing-environment-variables)
+- Learn about the [built-in environment variables](built-in) provided by Dagster+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/index.md
new file mode 100644
index 0000000000000..11977fe54b57f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/environment-variables/index.md
@@ -0,0 +1,23 @@
+---
+title: Environment variables
+sidebar_position: 20
+---
+
+Environment variables, which are key-value pairs configured outside your source code, allow you to dynamically modify application behavior depending on environment.
+
+Using environment variables, you can define various configuration options for your Dagster application and securely set up secrets. For example, instead of hard-coding database credentials - which is bad practice and cumbersome for development - you can use environment variables to supply user details. This allows you to parameterize your pipeline without modifying code or insecurely storing sensitive data.
+
+There are two ways to declare and manage variables in Dagster+:
+
+* Through the [Dagster+ UI](dagster-ui)
+* With [agent configuration](agent-config).
+
+| | Dagster+ UI | Agent configuration |
+|-----------------------------|-------------|---------------------|
+| **Deployment type support** | [Serverless](/dagster-plus/deployment/deployment-types/serverless/), [Hybrid](/dagster-plus/deployment/deployment-types/hybrid/) | [Hybrid](/dagster-plus/deployment/deployment-types/hybrid/) |
+| **How it works** | Environment variables are managed in the Dagster+ UI. Values are pulled from storage and decrypted when your code is executed. | Environment variables are defined in the agent's configuration. Variables set at the code location level will pass through Dagster+, while those set at the deployment level bypass Dagster+ entirely. For more information, see "[Setting environment variables using agent config](agent-config)". |
+| **Requirements** |
Dagster code must use version 1.0.17 or later
If using [Hybrid](/dagster-plus/deployment/deployment-types/hybrid/), the agent must use Dagster version 1.0.17 or later
[Editor, Admin, or Organization Admin permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) in Dagster+. Note: Editors and Admins can only set environment variables for deployments where they're an Editor or Admin.
| Ability to modify your dagster.yaml and [dagster_cloud.yaml](/dagster-plus/deployment/code-locations/dagster-cloud-yaml) files |
+| **Limitations** |
Maximum of 1,000 variables per full deployment
Variables must be less than or equal to 4KB in size
Variable names:
Must be 512 characters or less in length
Must start with a letter or underscore
Must contain only letters, numbers, and underscores
May not be the same as [built-in (system) variables](built-in)
| Variable names:
Must start with a letter or underscore
Must contain only letters, numbers, and underscores
|
+| **Storage and encryption** | Uses Amazon Key Management Services (KMS) and envelope encryption. For more information, see "[Setting environment variables in the Dagster+ UI](dagster-ui#storage-and-encryption)". | Dependent on agent type. |
+| **Scope** | Scoped by deployment (full and branch) and optionally, code location. | Scoped by code location. Variables can be set for a full deployment (all code locations) or on a per-code location basis.|
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/index.md
new file mode 100644
index 0000000000000..144dd10871fe2
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/index.md
@@ -0,0 +1,8 @@
+---
+title: Deployment management
+sidebar_position: 30
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/managing-compute-logs-and-error-messages.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/managing-compute-logs-and-error-messages.md
new file mode 100644
index 0000000000000..db22055a62881
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/managing-compute-logs-and-error-messages.md
@@ -0,0 +1,102 @@
+---
+title: Managing compute logs and error messages
+sidebar_position: 200
+---
+
+import ThemedImage from '@theme/ThemedImage';
+
+:::note
+
+This guide is applicable to Dagster+.
+
+:::
+
+In this guide, we'll cover how to adjust where Dagster+ compute logs are stored and manage masking of error messages in the Dagster+ UI.
+
+By default, Dagster+ ingests [structured event logs and compute logs](/guides/monitor/logging/index.md#log-types) from runs and surfaces error messages from [code locations](/dagster-plus/deployment/code-locations/) in the UI.
+
+Depending on your organization's needs, you may want to retain these logs in your own infrastructure or mask error message contents.
+
+## Modifying compute log storage
+
+Dagster's compute logs are handled by the configured [`ComputeLogManager`](/api/python-api/internals#compute-log-manager). By default, Dagster+ utilizes the `CloudComputeLogManager` which stores logs in a Dagster+-managed Amazon S3 bucket, but you can customize this behavior to store logs in a destination of your choice.
+
+### Writing to your own S3 bucket
+
+If using the Kubernetes agent, you can instead forward logs to your own S3 bucket by using the [`S3ComputeLogManager`](/api/python-api/libraries/dagster-aws#dagster_aws.s3.S3ComputeLogManager).
+
+You can configure the `S3ComputeLogManager` in your [`dagster.yaml` file](/dagster-plus/deployment/management/settings/customizing-agent-settings):
+
+```yaml
+compute_logs:
+ module: dagster_aws.s3.compute_log_manager
+ class: S3ComputeLogManager
+ config:
+ show_url_only: true
+ bucket: your-compute-log-storage-bucket
+ region: your-bucket-region
+```
+
+If you are using Helm to deploy the Kubernetes agent, you can provide the following configuration in your `values.yaml` file:
+
+```yaml
+computeLogs:
+ enabled: true
+ custom:
+ module: dagster_aws.s3.compute_log_manager
+ class: S3ComputeLogManager
+ config:
+ show_url_only: true
+ bucket: your-compute-log-storage-bucket
+ region: your-bucket-region
+```
+
+### Disabling compute log upload
+
+If your organization has its own logging solution which ingests `stdout` and `stderr` from your compute environment, you may want to disable compute log upload entirely. You can do this with the .
+
+You can configure the `NoOpComputeLogManager` in your [`dagster.yaml` file](/dagster-plus/deployment/management/settings/customizing-agent-settings):
+
+```yaml
+compute_logs:
+ module: dagster.core.storage.noop_compute_log_manager
+ class: NoOpComputeLogManager
+```
+
+If you are using Helm to deploy the Kubernetes agent, use the `enabled` flag to disable compute log upload:
+
+```yaml
+computeLogs:
+ enabled: false
+```
+
+### Other compute log storage options
+
+For a full list of available compute log storage options, see "[Dagster instance configuration](/guides/deploy/dagster-instance-configuration#compute-log-storage)".
+
+## Masking error messages
+
+By default, Dagster+ surfaces error messages from your code locations in the UI, including when runs fail, sensors or schedules throw an exception, or code locations fail to load. You can mask these error messages in the case that their contents are sensitive.
+
+To mask error messages in a Dagster+ Deployment, set the environment variable `DAGSTER_REDACT_USER_CODE_ERRORS` equal to `1` using the [**Environment variables** page](/dagster-plus/deployment/management/environment-variables/) in the UI:
+
+
+
+Once set, error messages from your code locations will be masked in the UI. A unique error ID will be generated, which you can use to look up the error message in your own logs. This error ID will appear in place of the error message in UI dialogs or in a run's event logs.
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/managing-multiple-projects-and-teams.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/managing-multiple-projects-and-teams.md
new file mode 100644
index 0000000000000..d5c5d1818645d
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/managing-multiple-projects-and-teams.md
@@ -0,0 +1,273 @@
+---
+title: "Managing multiple projects and teams with Dagster+ Hybrid"
+description: "How to set up and manage multiple projects with Dagster+ Hybrid deployments."
+sidebar_position: 300
+---
+
+
+In this guide, we'll cover some strategies for managing multiple projects/code bases and teams in a Dagster+ account.
+
+## Separating code bases
+
+:::note
+
+In this section, repository refers to a version control system, such as Git or Mercurial.
+
+:::
+
+If you want to manage complexity or divide your work into areas of responsibility, consider isolating your code bases into multiple projects with:
+
+- Multiple directories in a single repository, or
+- Multiple repositories
+
+Refer to the following table for more information, including the pros and cons of each approach.
+
+| Approach | How it works | Pros | Cons |
+|----------|--------------|------|------|
+| **Multiple directories in a single repository** | You can use a single repository to manage multiple projects by placing each project in a separate directory. Depending on your VCS, you may be able to set code owners to restrict who can modify each project. |
Simple to implement
Facilitates code sharing between projects
|
All projects share the same CI/CD pipeline and cannot be deployed independently
Shared dependencies between projects may cause conflicts and require coordination between teams
|
+| **Multiple repositories** | For stronger isolation, you can use multiple repositories to manage multiple projects. |
Stronger isolation between projects and teams
Each project has its own CI/CD pipeline and be deployed independently
Dependencies between projects can be managed independently
| Code sharing between projects require additional coordination to publish and reuse packages between projects. |
+
+### Deployment configuration
+
+Whether you use a single repository or multiple, you can use a [`dagster_cloud.yaml` file](/dagster-plus/deployment/code-locations/dagster-cloud-yaml) to define the code locations to deploy. For each repository, follow the [steps appropriate to your CI/CD provider](/dagster-plus/features/ci-cd/configuring-ci-cd) and include only the code locations that are relevant to the repository in your CI/CD workflow.
+
+#### Example with GitHub CI/CD on Hybrid deployment
+
+1. **For each repository**, use the CI/CD workflow provided in [Dagster+ Hybrid quickstart repository](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/blob/main/.github/workflows/dagster-cloud-deploy.yml).
+
+2. **For each project in the repository**, configure a code location in the [`dagster_cloud.yaml` file](/dagster-plus/deployment/code-locations/dagster-cloud-yaml):
+
+ ```yaml
+ # dagster_cloud.yml
+
+ locations:
+ - location_name: project_a
+ code_source:
+ package_name: project_a
+ build:
+ # ...
+ - location_name: project_b
+ code_source:
+ package_name: project_b
+ build:
+ # ...
+ ```
+
+3. In the repository's `dagster-cloud-deploy.yml` file, modify the CI/CD workflow to deploy all code locations for the repository:
+
+ ```yaml
+ # .github/workflows/dagster-cloud-deploy.yml
+
+ jobs:
+ dagster-cloud-deploy:
+ # ...
+ steps:
+ - name: Update build session with image tag for "project_a" code location
+ id: ci-set-build-output-project-a
+ if: steps.prerun.outputs.result != 'skip'
+ uses: dagster-io/dagster-cloud-action/actions/utils/dagster-cloud-cli@v0.1
+ with:
+ command: "ci set-build-output --location-name=project_a --image-tag=$IMAGE_TAG"
+
+ - name: Update build session with image tag for "project_b" code location
+ id: ci-set-build-output-project-b
+ if: steps.prerun.outputs.result != 'skip'
+ uses: dagster-io/dagster-cloud-action/actions/utils/dagster-cloud-cli@v0.1
+ with:
+ command: "ci set-build-output --location-name=project_b --image-tag=$IMAGE_TAG"
+ # ...
+ ```
+
+## Isolating execution context between projects
+
+Separating execution context between projects can have several motivations:
+
+- Facilitating separation of duty between teams to prevent access to sensitive data
+- Differing compute environments and requirements, such as different architecture, cloud provider, etc.
+- Reducing impact on other projects. For example, a project with a large number of runs can impact the performance of other projects.
+
+In order from least to most isolated, there are three levels of isolation:
+
+- [Code location](#code-location-isolation)
+- [Agent](#agent-isolation)
+- [Deployment](#deployment-isolation)
+
+### Code location isolation
+
+If you have no specific requirements for isolation beyond the ability to deploy and run multiple projects, you can use a single agent and deployment to manage all your projects as individual code locations.
+
+![Diagram of isolation at the code location level](/images/dagster-plus/deployment/management/managing-deployments/isolation-level-code-locations.png)
+
+
+
+
+
+ Pros
+
+
Cons
+
+
+
+
+
+
+
+ Simplest and most cost-effective solution
+
+
User access control can be set at the code location level
+
Single glass pane to view all assets
+
+
+
+
+
+ No isolation between execution environments
+
+
+
+
+
+
+
+### Agent isolation
+
+:::note
+
+Agent queues are a Dagster+ Pro feature available on hybrid deployment.
+
+:::
+
+Using the [agent routing feature](/dagster-plus/deployment/deployment-types/hybrid/multiple#routing-requests-to-specific-agents), you can effectively isolate execution environments between projects by using a separate agent for each project.
+
+Motivations for utilizing this approach could include:
+
+- Different compute requirements, such as different cloud providers or architectures
+- Optimizing for locality or access, such as running the data processing closer or in environment with access to the storage locations
+
+![Diagram of isolation at the agent level](/images/dagster-plus/deployment/management/managing-deployments/isolation-level-agents.png)
+
+
+
+
+
+ Pros
+
+
Cons
+
+
+
+
+
+
+
+ Isolation between execution environments
+
+
User access control can be set at the code location level
+
Single glass pane to view all assets
+
+
+
Extra work to set up additional agents and agent queues
+
+
+
+
+### Deployment isolation
+
+:::note
+
+Multiple deployments are only available in Dagster+ Pro.
+
+:::
+
+Of the approaches outlined in this guide, multiple deployments are the most isolated solution. The typical motivation for this isolation level is to separate production and non-production environments. It may be considered to satisfy other organization specific requirements.
+
+![Diagram of isolation at the Dagster+ deployment level](/images/dagster-plus/deployment/management/managing-deployments/isolation-level-deployments.png)
+
+
+
+
+
+ Pros
+
+
Cons
+
+
+
+
+
+
+
+ Isolation between assets and execution environments
+
+
+ User access control can be set at the code location and deployment
+ level
+
+
+
+
+ No single glass pane to view all assets (requires switching between
+ multiple deployments in the UI)
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/rate-limits.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/rate-limits.md
new file mode 100644
index 0000000000000..9cccf2b46815e
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/rate-limits.md
@@ -0,0 +1,14 @@
+---
+title: Dagster+ rate limits
+sidebar_position: 100
+---
+
+Dagster+ enforces several rate limits to smoothly distribute the load. Deployments are limited to:
+
+- 40,000 user log events (e.g, `context.log.info`) per minute. This limit only applies to custom logs; system events like the ones that drive orchestration or materialize assets are not subject to this limit.
+- 35MB of events per minute. This limit applies to both custom events and system events.
+
+Rate-limited requests return a "429 - Too Many Requests" response. Dagster+ agents automatically retry these requests.
+
+{/* Switching from [Structured event logs](/concepts/logging#structured-event-logs) to [Raw compute logs](/concepts/logging#raw-compute-logs) or reducing your custom log volume can help you stay within these limits. */}
+Switching from [Structured event logs](/guides/monitor/logging/index.md#structured-event-logs) to [Raw compute logs](/guides/monitor/logging/index.md#raw-compute-logs) or reducing your custom log volume can help you stay within these limits.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/customizing-agent-settings.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/customizing-agent-settings.md
new file mode 100644
index 0000000000000..b46faa4ea732e
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/customizing-agent-settings.md
@@ -0,0 +1,139 @@
+---
+title: "Customizing Dagster+ agent settings in dagster.yaml"
+sidebar_position: 300
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+The Dagster+ Agent is a special variant of the Dagster instance used in Dagster Open Source and is configured through the same `dagster.yaml` file. You can customize your agent with these settings.
+
+:::note
+
+For [Kubernetes agents](/dagster-plus/deployment/deployment-types/hybrid/kubernetes/) deployed with the Dagster+ Helm chart, you'll need to refer to the Helm chart's config map for customizing the agent.
+
+:::
+
+## Enabling user code server TTL
+
+User code servers support a configurable time-to-live (TTL). The agent will spin down any user code servers that haven't served requests recently and will spin them back up the next time they're needed. Configuring TTL can save compute cost because user code servers will spend less time sitting idle.
+
+TTL is disabled by default for full deployments, and can be configured separately for full and [branch deployments](/dagster-plus/features/ci-cd/branch-deployments/setting-up-branch-deployments). TTL defaults to 24 hours for both full and branch deployments.
+
+To configure TTL:
+```yaml
+# dagster.yaml
+instance_class:
+ module: dagster_cloud.instance
+ class: DagsterCloudAgentInstance
+
+dagster_cloud_api:
+ agent_token:
+ env: DAGSTER_CLOUD_AGENT_TOKEN
+ deployment: prod
+
+user_code_launcher:
+ module: dagster_cloud.workspace.docker
+ class: DockerUserCodeLauncher
+ config:
+ server_ttl:
+ full_deployments:
+ enabled: true # Disabled by default for full deployments
+ ttl_seconds: 7200 # 2 hours
+ branch_deployments:
+ ttl_seconds: 3600 # 1 hour
+```
+
+## Streaming compute logs
+
+You can set up streaming compute logs by configuring the log upload interval (in seconds).
+
+```yaml
+# dagster.yaml
+instance_class:
+ module: dagster_cloud.instance
+ class: DagsterCloudAgentInstance
+
+dagster_cloud_api:
+ agent_token:
+ env: DAGSTER_CLOUD_AGENT_TOKEN
+ deployment: prod
+
+user_code_launcher:
+ module: dagster_cloud.workspace.docker
+ class: DockerUserCodeLauncher
+
+compute_logs:
+ module: dagster_cloud
+ class: CloudComputeLogManager
+ config:
+ upload_interval: 60
+```
+
+## Disabling compute logs
+
+You can disable forwarding compute logs to Dagster+ by configuring the `NoOpComputeLogManager` setting:
+
+```yaml
+# dagster.yaml
+instance_class:
+ module: dagster_cloud.instance
+ class: DagsterCloudAgentInstance
+
+dagster_cloud_api:
+ agent_token:
+ env: DAGSTER_CLOUD_AGENT_TOKEN
+ deployment: prod
+
+user_code_launcher:
+ module: dagster_cloud.workspace.docker
+ class: DockerUserCodeLauncher
+
+compute_logs:
+ module: dagster.core.storage.noop_compute_log_manager
+ class: NoOpComputeLogManager
+```
+
+## Writing compute logs to AWS S3
+
+{/* /api/python-api/libraries/dagster-aws#dagster_aws.s3.S3ComputeLogManager */}
+You can write compute logs to an AWS S3 bucket by configuring the module.
+
+You are also able to stream partial compute log files by configuring the log upload interval (in seconds) using the `upload_interval` parameter.
+
+Note: Dagster Labs will neither have nor use your AWS credentials. The Dagster+ UI will be able to show the URLs linking to the compute log files in your S3 bucket when you set the `show_url_only` parameter to `true`.
+
+```yaml
+# dagster.yaml
+instance_class:
+ module: dagster_cloud.instance
+ class: DagsterCloudAgentInstance
+
+dagster_cloud_api:
+ agent_token:
+ env: DAGSTER_CLOUD_AGENT_TOKEN
+ deployment: prod
+
+user_code_launcher:
+ module: dagster_cloud.workspace.docker
+ class: DockerUserCodeLauncher
+
+compute_logs:
+ module: dagster_aws.s3.compute_log_manager
+ class: S3ComputeLogManager
+ config:
+ bucket: "mycorp-dagster-compute-logs"
+ local_dir: "/tmp/cool"
+ prefix: "dagster-test-"
+ use_ssl: true
+ verify: true
+ verify_cert_path: "/path/to/cert/bundle.pem"
+ endpoint_url: "http://alternate-s3-host.io"
+ skip_empty_files: true
+ upload_interval: 30
+ upload_extra_args:
+ ServerSideEncryption: "AES256"
+ show_url_only: true
+ region: "us-west-1"
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/dagster-plus-settings.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/dagster-plus-settings.md
new file mode 100644
index 0000000000000..ac4d6cfc1af70
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/dagster-plus-settings.md
@@ -0,0 +1,6 @@
+---
+title: "Dagster+ settings"
+sidebar_position: 100
+unlisted: true
+---
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/hybrid-agent-settings.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/hybrid-agent-settings.md
new file mode 100644
index 0000000000000..51f792b6a4f5f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/hybrid-agent-settings.md
@@ -0,0 +1,8 @@
+---
+title: "Hybrid agent settings"
+sidebar_position: 400
+unlisted: true
+---
+
+{/* TODO not sure why this page exists, is this covered by "Customizing agent settings"? */}
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/index.md
new file mode 100644
index 0000000000000..b20c1332e2c58
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/settings/index.md
@@ -0,0 +1,8 @@
+---
+title: Settings
+sidebar_position: 10
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/tokens/agent-tokens.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/tokens/agent-tokens.md
new file mode 100644
index 0000000000000..8d7088a4d0894
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/tokens/agent-tokens.md
@@ -0,0 +1,52 @@
+---
+title: 'Managing agent tokens in Dagster+'
+sidebar_position: 200
+---
+
+:::note
+
+This guide is applicable to Dagster+.
+
+:::
+
+In this guide, we'll walk you through creating and revoking agent tokens in Dagster+.
+
+## Managing agent tokens
+:::note
+
+To manage agent tokens, you need to be an [Organization Admin](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions#user-permissions-reference) in Dagster+.
+
+:::
+
+Agent tokens are used to authenticate [Hybrid agents](/dagster-plus/deployment/deployment-types/hybrid/) with the Dagster+ Agents API.
+
+### Creating agent tokens
+
+1. Sign in to your Dagster+ account.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. Click the **Tokens** tab.
+4. Click **+ Create agent token**.
+
+After the token is created:
+
+- **To view a token**, click **Reveal token**. Clicking on the token value will copy it to the clipboard.
+- **To edit a token's description**, click the **pencil icon**.
+
+### Assigning agent token permissions
+
+1. Sign in to your Dagster+ account.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. Click the **Tokens** tab.
+4. Click **Edit** next to the agent token you'd like to change.
+
+The permissions dialog allows you to edit a token's ability to access certain deployments. By default, agent tokens have permission to access any deployment in the organization including branch deployments. This is called **Org Agent** and is set using the toggle in the top right of the dialog. To edit individual deployment permissions, **Org Agent** has to first be toggled off.
+
+### Revoking agent tokens
+
+To revoke a token:
+
+1. Sign in to your Dagster+ account.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. Click the **Tokens** tab.
+4. Click **Edit** next to the agent token you'd like to change.
+5. Click **Revoke** in the bottom left of the permissions dialog. When prompted, confirm to proceed with revoking the token.
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/tokens/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/tokens/index.md
new file mode 100644
index 0000000000000..408a48a81e1bf
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/tokens/index.md
@@ -0,0 +1,8 @@
+---
+title: Tokens
+sidebar_position: 30
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/tokens/user-tokens.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/tokens/user-tokens.md
new file mode 100644
index 0000000000000..ccb08b6b21b04
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/management/tokens/user-tokens.md
@@ -0,0 +1,38 @@
+---
+title: 'Managing user tokens in Dagster+'
+sidebar_position: 100
+---
+
+import ThemedImage from '@theme/ThemedImage';
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+In this guide, we'll walk you through creating user tokens in Dagster+.
+
+## Managing user tokens
+1. Sign in to your Dagster+ account.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. Click the **Tokens** tab.
+4. Click **+ Create user token**.
+
+After the token is created:
+
+- **To edit a token's description**, click the **pencil icon**.
+- **To view a token**, click **Reveal token**. Clicking on the token value will copy it to the clipboard.
+- **To revoke a token**, click **Revoke**.
+
+To manage tokens for another user, select the user from the **Manage tokens for** dropdown:
+
+
+
+:::note
+**Organization Admin** permissions are required to manage another user's tokens.
+:::
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/migration/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/migration/index.md
new file mode 100644
index 0000000000000..ea7c4a344ade3
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/migration/index.md
@@ -0,0 +1,8 @@
+---
+title: Migration
+sidebar_position: 50
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/migration/self-hosted-to-dagster-plus.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/migration/self-hosted-to-dagster-plus.md
new file mode 100644
index 0000000000000..e9ea6226a40ff
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/migration/self-hosted-to-dagster-plus.md
@@ -0,0 +1,6 @@
+---
+title: Migrating from self-hosted to Dagster+
+sidebar_label: Self-hosted to Dagster+
+sidebar_position: 200
+unlisted: true
+---
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/migration/serverless-to-hybrid.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/migration/serverless-to-hybrid.md
new file mode 100644
index 0000000000000..cf23e8648e11f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/deployment/migration/serverless-to-hybrid.md
@@ -0,0 +1,48 @@
+---
+title: Migrating from Serverless to Hybrid
+sidebar_label: Serverless to Hybrid
+sidebar_position: 100
+---
+
+After utilizing a Dagster+ [Serverless](/dagster-plus/deployment/deployment-types/serverless) deployment, you may decide to leverage your own infrastructure to execute your code. Transitioning to a Hybrid deployment requires only a few steps and can be done without any loss of execution history or metadata, allowing you to maintain continuity and control over your operations.
+
+:::warning
+Transitioning from Serverless to Hybrid requires some downtime, as your Dagster+ deployment won't have an agent to execute user code.
+:::
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- **Organization Admin** permissions in your Dagster+ account
+
+
+
+## Step 1: Deactivate your Serverless agent
+
+1. In the Dagster+ UI, navigate to the **Deployment > Agents** page.
+2. Click the drop down arrow on the right of the page and select **Switch to Hybrid**.
+
+![PRODUCT NOTE - this arrow drop down is pretty small and easy to confuse with the one in the row for the agent](/images/dagster-plus/deployment/switch-agent-to-hybrid.png)
+
+It may take a few minutes for the agent to deactivate and be removed from the list of agents.
+
+## Step 2: Create a Hybrid agent
+
+Next, you'll need to create a Hybrid agent to execute your code. Follow the setup instructions for the agent of your choice:
+
+- **[Amazon Web Services (AWS)](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs)**, which launches user code as Amazon Elastic Container Service (ECS) tasks.
+- **[Docker](/dagster-plus/deployment/deployment-types/hybrid/docker)**, which launches user code in Docker containers on your machine
+- **[Kubernetes](/dagster-plus/deployment/deployment-types/hybrid/kubernetes)**, which launches user code on a Kubernetes cluster
+- **[Local](/dagster-plus/deployment/deployment-types/hybrid/local)**, which launches user code in operating system subprocesses on your machine
+
+## Step 3: Confirm successful setup
+
+Once you've set up a Hybrid agent, navigate to the **Deployment > Agents** page in the UI. The new agent should display in the list with a `RUNNING` status:
+
+![Screenshot](/images/dagster-plus/deployment/running-agent.png)
+
+## Next steps
+
+- Learn about the configuration options for [dagster.yaml](/guides/deploy/dagster-yaml)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/alerts/configuring-an-alert-notification-service.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/alerts/configuring-an-alert-notification-service.md
new file mode 100644
index 0000000000000..13d55c43e1798
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/alerts/configuring-an-alert-notification-service.md
@@ -0,0 +1,54 @@
+---
+title: Configuring an alert notification service
+sidebar_position: 100
+---
+
+Dagster+ allows you to configure alerts to fire in response to a range of events. These alerts can be sent to a variety of different services, depending on your organization's needs.
+
+{/* TODO this needs content on the value prop of alerts -- save money, etc */}
+
+:::note
+You must have **Organization**, **Admin**, or **Editor** permissions on Dagster+ to configure an alert notification service.
+:::
+
+Before [creating alerts](creating-alerts), you'll need to configure a service to send alerts. Dagster+ currently supports sending alerts through email, Microsoft Teams, PagerDuty, and Slack.
+
+
+
+ No additional configuration is required to send emails from Dagster+.
+
+All alert emails will be sent by `"no-reply@dagster.cloud"` or `"no-reply@.dagster.cloud"`. Alerts can be configured to be sent to any number of emails.
+
+
+ Create an incoming webhook by following the [Microsoft Teams documentation](https://learn.microsoft.com/en-us/microsoftteams/platform/webhooks-and-connectors/how-to/add-incoming-webhook?tabs=newteams%2Cdotnet).
+
+This will provide you with a **webhook URL** which will be required when configuring alerts in the UI (after selecting "Microsoft Teams" as your Notification Service) or using the CLI (in the `notification_service` configuration).
+
+
+
+ :::note
+You will need sufficient permissions in PagerDuty to add or edit services.
+:::
+
+In PagerDuty, you can either:
+
+- [Create a new service](https://support.pagerduty.com/main/docs/services-and-integrations#create-a-service), and add Dagster+ as an integration, or
+- [Edit an existing service](https://support.pagerduty.com/main/docs/services-and-integrations#edit-service-settings) to include Dagster+ as an integration
+
+When configuring the integration, choose **Dagster+** as the integration type, and choose an integration name in the format `dagster-plus-{your_service_name}`.
+
+After adding your new integration, you will be taken to a screen containing an **Integration Key**. This value will be required when configuring alerts in the UI (after selecting "PagerDuty" as your Notification Service) or using the CLI (in the `notification_service` configuration).
+
+
+
+ :::note
+You will need sufficient permissions in Slack to add apps to your workspace.
+:::
+Navigate to **Deployment > Alerts** in the Dagster+ UI and click **Connect to Slack**. From there, you can complete the installation process.
+
+When setting up an alert, you can choose a Slack channel to send those alerts to. Make sure to invite the `@Dagster Cloud` bot to any channel that you'd like to receive an alert in.
+
+To disconnect Dagster+ from Slack, remove the Dagster Cloud app from your Slack workspace. For more information, see the [Slack documentation](https://slack.com/help/articles/360003125231-Remove-apps-and-custom-integrations-from-your-workspace#remove-an-app).
+
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/alerts/creating-alerts.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/alerts/creating-alerts.md
new file mode 100644
index 0000000000000..5b57ceee0774e
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/alerts/creating-alerts.md
@@ -0,0 +1,284 @@
+---
+title: Creating alerts in Dagster+
+sidebar_position: 200
+---
+
+You can create alerts in the Dagster+ UI or using the [`dagster-cloud` CLI](/dagster-plus/deployment/management/dagster-cloud-cli).
+
+{/* TODO link to dagster-cloud CLI tool doc */}
+
+:::note
+Before you create alerts, you must [configure an alert notification service](configuring-an-alert-notification-service).
+:::
+
+## Alerting when a run fails
+You can set up alerts to notify you when a run fails.
+
+By default, these alerts will target all runs in the deployment, but they can be scoped to runs with a specific tag.
+
+
+ 1. In the Dagster UI, click **Deployment**.
+2. Click the **Alerts** tab.
+3. Click **Add alert policy**.
+4. Select **Run alert** from the dropdown.
+
+5. Select **Job failure**.
+
+If desired, add **tags** in the format `{key}:{value}` to filter the runs that will be considered.
+
+
+
+ Execute the following command to sync the configured alert policy to your Dagster+ deployment.
+
+ ```bash
+ dagster-cloud deployment alert-policies sync -a /path/to/alert_policies.yaml
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Alerting when a run is taking too long to complete
+You can set up alerts to notify you whenever a run takes more than some threshold amount of time.
+
+ By default, these alerts will target all runs in the deployment, but they can be scoped to runs with a specific tag.
+
+
+ 1. In the Dagster UI, click **Deployment**.
+2. Click the **Alerts** tab.
+3. Click **Add alert policy**.
+4. Select **Run alert** from the dropdown.
+
+5. Select **Job running over** and how many hours to alert after.
+
+If desired, add **tags** in the format `{key}:{value}` to filter the runs that will be considered.
+
+
+
+ Execute the following command to sync the configured alert policy to your Dagster+ deployment.
+
+ ```bash
+ dagster-cloud deployment alert-policies sync -a /path/to/alert_policies.yaml
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Alerting when an asset fails to materialize
+You can set up alerts to notify you when an asset materialization attempt fails.
+
+By default, these alerts will target all assets in the deployment, but they can be scoped to a specific asset or group of assets.
+
+:::note
+
+If using a RetryPolicy, an alert will only be sent after all retries complete.
+
+:::
+
+
+
+ 1. In the Dagster UI, click **Deployment**.
+2. Click the **Alerts** tab.
+3. Click **Add alert policy**.
+4. Select **Asset alert** from the dropdown.
+
+5. Select **Failure** under the **Materializations** heading.
+
+If desired, select a **target** from the dropdown menu to scope this alert to a specific asset or group.
+
+
+
+ Execute the following command to sync the configured alert policy to your Dagster+ deployment.
+
+ ```bash
+ dagster-cloud deployment alert-policies sync -a /path/to/alert_policies.yaml
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Alerting when an asset check fails
+You can set up alerts to notify you when an asset check on an asset fails.
+
+By default, these alerts will target all assets in the deployment, but they can be scoped to checks on a specific asset or group of assets.
+
+
+ 1. In the Dagster UI, click **Deployment**.
+2. Click the **Alerts** tab.
+3. Click **Add alert policy**.
+4. Select **Asset alert** from the dropdown.
+
+5. Select **Failed (ERROR)** under the **Asset Checks** heading.
+
+If desired, select a **target** from the dropdown menu to scope this alert to a specific asset or group.
+
+
+
+ Execute the following command to sync the configured alert policy to your Dagster+ deployment.
+
+ ```bash
+ dagster-cloud deployment alert-policies sync -a /path/to/alert_policies.yaml
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Alerting when a schedule or sensor tick fails
+You can set up alerts to fire when any schedule or sensor tick across your entire deployment fails.
+
+Alerts are sent only when a schedule/sensor transitions from **success** to **failure**, so only the initial failure will trigger the alert.
+
+
+ 1. In the Dagster UI, click **Deployment**.
+2. Click the **Alerts** tab.
+3. Click **Add alert policy**.
+4. Select **Schedule/Sensor alert** from the dropdown.
+
+
+ Execute the following command to sync the configured alert policy to your Dagster+ deployment.
+
+ ```bash
+ dagster-cloud deployment alert-policies sync -a /path/to/alert_policies.yaml
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Alerting when a code location fails to load
+You can set up alerts to fire when any code location fails to load due to an error.
+
+
+ 1. In the Dagster UI, click **Deployment**.
+2. Click the **Alerts** tab.
+3. Click **Add alert policy**.
+4. Select **Code location error alert** from the dropdown.
+
+
+ Execute the following command to sync the configured alert policy to your Dagster+ deployment.
+
+ ```bash
+ dagster-cloud deployment alert-policies sync -a /path/to/alert_policies.yaml
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Alerting when a Hybrid agent becomes unavailable
+:::note
+This is only available for [Hybrid](/dagster-plus/deployment/deployment-types/hybrid/) deployments.
+:::
+
+You can set up alerts to fire if your Hybrid agent hasn't sent a heartbeat in the last 5 minutes.
+
+
+ 1. In the Dagster UI, click **Deployment**.
+2. Click the **Alerts** tab.
+3. Click **Add alert policy**.
+4. Select **Code location error alert** from the dropdown.
+
+
+ Execute the following command to sync the configured alert policy to your Dagster+ deployment.
+
+ ```bash
+ dagster-cloud deployment alert-policies sync -a /path/to/alert_policies.yaml
+ ```
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/alerts/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/alerts/index.md
new file mode 100644
index 0000000000000..d942794a2fd92
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/alerts/index.md
@@ -0,0 +1,8 @@
+---
+title: Alerts
+sidebar_position: 20
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/asset-catalog/catalog-views.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/asset-catalog/catalog-views.md
new file mode 100644
index 0000000000000..596ce9bfe7b5f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/asset-catalog/catalog-views.md
@@ -0,0 +1,65 @@
+---
+title: "Catalog views"
+sidebar_position: 100
+---
+
+Catalog views enable you to filter down your view of the Dagster Asset catalog in Dagster+, allowing you to toggle between sets of assets that you care about most.
+
+You can save catalog views for your own use or share them with your team. For example, you could create views that:
+
+- Filter assets based on ownership to only show those owned by your team
+- Filter assets based on the asset kind to give insight into the status of your ELT ingestion
+- Display assets with a "gold" medallion tag, showing only refined, high-quality data that analysts can use with confidence
+
+In this guide, you'll learn how to create, access, and share catalog views with others.
+
+:::note
+
+Catalog views require **Organization Admin**, **Admin**, or **Editor** permissions on Dagster+.
+
+:::
+
+## Creating catalog views
+
+To view the Dagster+ Asset catalog, use the **Catalog** button on the top navigation.
+
+In any Dagster+ catalog page, you can access the current catalog view, or create a new catalog view with the catalog view dropdown at the top left of the screen. By default, this button is labeled **All assets**, and has a globe icon.
+
+![Screenshot of the catalog view dropdown](/images/dagster-plus/features/asset-catalog/catalog-views.png)
+
+To create a new catalog view, you have two options:
+- [Create a new catalog view from scratch](#creating-a-new-catalog-view-from-scratch), from the catalog view menu.
+- [Create a new catalog view from your current set of filters](#creating-a-new-catalog-view-from-your-current-set-of-filters).
+
+### Creating a new catalog view from scratch
+
+1. Click the catalog view dropdown to open the catalog view menu. From here, click the **New** button.
+2. Give the view a name and optionally, a description and icon.
+3. Click **Add filters** to select filters to apply to the view. Filters can select a subset of assets based on their metadata, tags, kinds, owners, asset groups, or other properties.
+4. To make the view shareable, toggle the **Public view** switch.
+5. Click **Create view** to create the view.
+
+![Screenshot of new catalog view modal](/images/dagster-plus/features/asset-catalog/new-catalog-view.png)
+
+Give your view a name and optionally a description and icon. Next, you can select one or more filters to apply to your view by clicking the **Add filters** button. Filters can select a subset of assets based on their [metadata]/guides/build/assets/metadata-and-tags/), tags, kinds, owners, asset groups, or other properties.
+
+### Creating a new catalog view from your current set of filters
+
+When viewing the global asset lineage or asset list, you can create a new catalog view from your current set of filters.
+
+1. On these pages, select one or more asset filters.
+2. Click **Create new catalog view**, located near the top right of the page. This will open the catalog view creation dialog with your current filters pre-populated.
+3. Give the view a name and optionally, a description and icon.
+4. To make the view shareable, toggle the **Public view** switch.
+5. Click **Create view** to create the view.
+
+![Screenshot of creating catalog view from filters](/images/dagster-plus/features/asset-catalog/new-catalog-view-from-asset-page.png)
+
+## Editing, duplicating, or deleting catalog views
+
+1. Click the **catalog view** button to open the catalog view menu.
+2. Search for the view you want to edit, duplicate, or delete.
+3. Click the **three dot menu** to the right of the view to display available options.
+4. If modifying the view, note that any active filters will automatically be included in the set of changes. You can also change the view's name, description, icon, and sharing settings. 5. When finished, click **Save changes**.
+
+![Screenshot of editing catalog views](/images/dagster-plus/features/asset-catalog/edit-catalog-view.png)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/asset-catalog/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/asset-catalog/index.md
new file mode 100644
index 0000000000000..ab534f84e01b3
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/asset-catalog/index.md
@@ -0,0 +1,16 @@
+---
+title: "Asset catalog"
+sidebar_position: 40
+---
+
+The Dagster+ version of the asset catalog page displays assets broken out by compute kind, asset group, [code location](/dagster-plus/deployment/code-locations), [tags](/guides/build/assets/metadata-and-tags/tags), and owners, and more. On this page, you can:
+
+ - View all assets in your Dagster deployment
+ - View details about a specific asset by clicking on it
+ - Search assets by asset key, compute kind, asset group, code location, tags, owners, and more
+ - Access the global asset lineage
+ - Reload definitions
+
+To access the asset catalog, click **Catalog** in the top navigation.
+
+![The Asset Catalog page in the Dagster UI](/images/dagster-plus/features/asset-catalog/asset-catalog-cloud-pro.png)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/index.md
new file mode 100644
index 0000000000000..f8c1c0e7a8118
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/index.md
@@ -0,0 +1,8 @@
+---
+title: "Authentication and access control"
+sidebar_position: 30
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/audit-logs.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/audit-logs.md
new file mode 100644
index 0000000000000..fdb7677185aea
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/audit-logs.md
@@ -0,0 +1,65 @@
+---
+title: "Audit logs"
+sidebar_position: 300
+---
+
+The Dagster+ audit log enables Dagster+ Pro organizations to track and attribute changes to their Dagster deployment.
+
+For large organizations, tracking down when and by whom changes were made can be crucial for maintaining security and compliance. The audit log is also valuable
+ for tracking operational history, including sensor and schedule updates.
+
+This guide walks through how to access the audit log and details the interactions which are tracked in the audit log.
+
+
+Prerequisites
+- A Dagster+ Pro organization
+- An [Organization Admin](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) role in your Dagster+ organization
+
+
+## View audit logs
+
+To access the audit logs:
+
+1. Click your user icon at the top right corner of the page.
+2. Click **Organization settings**.
+3. Click the **Audit log** tab.
+
+:::warning
+
+Add screenshot
+
+:::
+
+Each entry in the audit log indicates when an action was taken, the user who performed the action, the action taken, and the deployment which the action affected. To view additional details for an action, click the **Show** button.
+
+## Filter the audit log
+
+The **Filter** button near the top left of the page can be used to filter the list of logs. You can filter to a combination of user, event type, affected deployment, or time frame.
+
+## Audit log entry types
+
+| Event type | Description | Additional details |
+|--------------------------------|---------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------|
+| Log in | A user logs in to the Dagster+ organization | |
+| Update sensor | A user toggles a sensor on or off | The sensor name, code location, and cursor |
+| Update schedule | A user toggles a schedule on or off | The schedule name, code location, and cursor |
+| Update alert policy | A user modifies an [alert policy](/dagster-plus/features/alerts/creating-alerts) | The new configuration for the alert policy |
+| Create deployment | A user creates a new deployment | Whether the deployment is a branch deployment |
+| Delete deployment | A user removes an existing deployment | Whether the deployment is a branch deployment |
+| Create user token | A user creates a new user token | |
+| Revoke user token | A user revokes an existing user token | |
+| Change user permissions | A user alters [permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) for another user | The permission grant and targeted deployment |
+| Create agent token | A user creates a new agent token | |
+| Revoke agent token | A user revokes an existing agent token | |
+| Update agent token permissions | A user alters [permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) for an agent token | The permission grant and targeted deployment |
+| Create secret | A user creates a new [environment variable](/dagster-plus/deployment/management/environment-variables/dagster-ui) | The created variable name |
+| Update secret | A user modifies an existing [environment variable](/dagster-plus/deployment/management/environment-variables/dagster-ui) | The previous and current variable names and whether the value was changed |
+| Delete secret | A user removes an [environment variable](/dagster-plus/deployment/management/environment-variables/dagster-ui) | The deleted variable name |
+| Update subscription | A user modifies the selected Dagster+ subscription for the organization | The previous and current plan types |
+
+## Programmatic access to audit logs
+
+Audit logs can be accessed programmatically over the Dagster+ GraphQL API. You can access a visual GraphiQL interface
+by navigating to `https://.dagster.cloud//graphql` in your browser. You can also query the API directly using the Python client.
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/index.md
new file mode 100644
index 0000000000000..5f6a44b46616a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/index.md
@@ -0,0 +1,8 @@
+---
+title: "Role-based access control"
+sidebar_position: 10
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/teams.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/teams.md
new file mode 100644
index 0000000000000..894a4a29f0db9
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/teams.md
@@ -0,0 +1,76 @@
+---
+title: Team management in Dagster+
+sidebar_label: "Team management"
+sidebar_position: 100
+---
+
+As part of [role-based access control (RBAC)](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions), Dagster+ supports the ability to assign users to teams. A team is a group of users with a set of default deployment, code location, and Branch Deployment user roles.
+
+
+ Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- A Dagster+ Pro plan
+- Dagster+ [Organization Admin permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions):
+ - In your organization, and
+ - For the deployments where you want to manage teams
+
+
+
+
+## Adding teams
+
+1. In the Dagster+ UI, click the **user menu (your icon) > Organization Settings**.
+2. Click the **Teams** tab.
+3. Click the **Create a team** button.
+4. In the window that displays, enter a name in the **Team name** field.
+5. Click **Create team**.
+
+After the team is created, you can [add team members](#adding-team-members) and [assign user roles to deployments](#managing-team-roles).
+
+## Adding team members
+
+Navigate to the **Organization Settings > Teams** tab and locate the team you want to add team members to. Then:
+
+1. Click the **Edit** button in the **Actions** column.
+2. In the **Members** tab, use the search bar to locate a user in your organization.
+3. Once located, click the user.
+4. Click **Add user to team**.
+5. Repeat as needed, clicking **Done** when finished.
+
+## Removing team members
+
+Navigate to the **Organization Settings > Teams** tab and locate the team you want to remove team members from. Then:
+
+1. Click the **Edit** button in the **Actions** column.
+2. In the **Members** tab, locate the user in the list of team members.
+3. Click **Remove from team**.
+4. Repeat as needed, clicking **Done** when finished.
+
+## Managing team roles
+
+Navigate to the **Organization Settings > Teams** tab and locate the team you want to manage roles for. Then:
+
+1. Click the **Edit** button in the **Actions** column.
+2. In the **Roles** tab, click the **Edit team role** button next to the deployment where you want to modify the team's role.
+3. In the window that displays, select the team role for the deployment. This [role](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) will be used as the default for this team for all code locations in the deployment.
+4. Click **Save**.
+5. To set permissions for individual [code locations](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) in a deployment:
+ 1. Click the toggle to the left of the deployment to open a list of code locations.
+ 2. Next to a code location, click **Edit team role**.
+ 3. Select the team role for the code location.
+ 4. Click **Save**.
+
+## Removing teams
+
+Navigate to the **Organization Settings > Teams** tab and locate the team you want to remove. Then:
+
+1. Click the **Edit** button in the **Actions** column.
+2. In the modal that displays, click the **Delete team** button.
+3. When prompted, confirm the deletion.
+
+## Next steps
+
+- Learn more about RBAC in [Understanding User Roles & Permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions)
+- Learn more about how to manage users in Dagster+ in [Understanding User Management in Dagster+](/dagster-plus/features/authentication-and-access-control/rbac/users)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions.md
new file mode 100644
index 0000000000000..71ef9a8102cf4
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions.md
@@ -0,0 +1,209 @@
+---
+title: User roles & permissions in Dagster+
+sidebar_label: 'User roles & permissions'
+sidebar_position: 200
+---
+
+Role-based access control (RBAC) enables you to grant specific permissions to users in your organization, ensuring that Dagster users have access to what they require in Dagster+, and no more.
+
+In this guide, we'll cover how RBAC works in Dagster+, how to assign roles to users, and the granular permissions for each user role.
+
+
+ Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- A Dagster+ account
+ - Additionally, in certain cases listed below, a Dagster+ Pro plan
+
+
+
+## Dagster+ user roles
+
+Dagster+ uses a hierarchical model for RBAC, meaning that the most permissive roles include permissions from the roles beneath them. The following user roles are currently supported, in order from the **most** permissive to the **least** permissive:
+
+- Organization Admin
+- Admin
+- Editor
+- Launcher (Pro plans only)
+- Viewer
+
+For example, the **Admin** user role includes permissions specific to this role and all permissions in the **Editor**, **Launcher**, and **Viewer** user roles. Refer to the [User permissions reference](#user-permissions-reference) for the full list of user permissions in Dagster+.
+
+### User role enforcement
+
+All user roles are enforced both in Dagster+ and the GraphQL API.
+
+### Teams
+
+Dagster+ Pro users can create teams of users and assign default permission sets. Refer to the [Managing teams in Dagster+](/dagster-plus/features/authentication-and-access-control/rbac/teams) guide for more info.
+
+## Assigning user and team roles
+
+With the exception of the **Organization Admin** role, user and team roles are set on a per-deployment basis.
+
+Organization Admins have access to the entire organization, including all [deployments](/dagster-plus/deployment/management/deployments/), [code locations](/dagster-plus/deployment/code-locations), and [Branch Deployments](dagster-plus/features/ci-cd/branch-deployments/index.md).
+
+| Level | Plan | Description |
+| ------------------ | --------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- |
+| Deployment | All plans | Defines the level of access for a given deployment. Roles set at this level will be the default role for the user or team for all code locations in the deployment.
Note: Granting access to a deployment grants a minimum of Viewer access to all code locations. Preventing access for specific code locations isn't currently supported. Additionally, having access to a deployment doesn't grant access to Branch Deployments - those permissions must be granted separately. |
+| Code location | Pro | Defines the level of access for a given code location in a deployment.
Dagster+ Pro users can [override the default deployment-level role for individual code locations](/dagster-plus/deployment/code-locations). For example, if the Deployment role is Launcher, you could override this role with a more permissive role, such as Editor or Admin.
For non-Pro users, users will have the same level of access for all code locations in a deployment. |
+| Branch deployments | All plans | Defines the level of access for all Branch Deployments in the code locations the user or team has access to. |
+
+### Applying role overrides
+
+As previously mentioned, you can define individual user roles for users in your organization.
+
+Dagster+ Pro users can also apply permission overrides to grant specific exceptions.
+
+Overrides may be used to apply a **more permissive** role. If, for example, the default role is **Admin** or **Organization Admin**, overrides will be disabled as these are the most permissive roles.
+
+#### Code locations
+
+To override a code location role for an individual user:
+
+1. Locate the user in the list of users.
+2. Click **Edit**.
+3. Click the toggle to the left of the deployment to open a list of code locations.
+4. Next to a code location, click **Edit user role**.
+5. Select the user role for the code location:
+ - TODO: add picture previously at "/images/dagster-cloud/user-token-management/code-location-override.png"
+6. Click **Save**.
+
+#### Team members
+
+Users in your organization can belong to one or more [teams](/dagster-plus/features/authentication-and-access-control/rbac/teams). When determining a user's level of access, Dagster+ will use the **most permissive** role assigned to the user between all of their team memberships and any individual role grants.
+
+For example, let's look at a user with the following roles for our `dev` deployment:
+
+- **Team 1**: Launcher
+- **Team 2**: Viewer
+- **Individual**: Viewer
+
+In this example, the user would have **Launcher** access to the `prod` deployment. This is because the Launcher role is more permissive than Viewer.
+
+The above also applies to code locations and Branch Deployment roles.
+
+#### Viewing overrides
+
+To view deployment-level overrides for a specific user, locate the user on the **Users** page and hover over a deployment:
+
+TODO: add picture previously at "/images/dagster-cloud/user-token-management/user-overrides-popup.png"
+
+If there are code location-level overrides, a small **N override(s)** link will display beneath the user's deployment role. Hover over it to display the list of overrides:
+
+TODO: add picture previously at "/images/dagster-cloud/user-token-management/code-location-override-popup.png"
+
+#### Removing overrides
+
+1. Locate the user in the list of users.
+2. Click **Edit**.
+3. To remove an override:
+ - **For a deployment**, click **Edit user role** next to the deployment.
+ - **For a code location**, click the toggle next to the deployment to display a list of code locations. Click **Edit user role** next to the code location.
+4. Click the **Remove override** button.
+5. Click **Save**.
+
+## User permissions reference
+
+### General
+
+| | Viewer | Launcher | Editor | Admin | Organization admin |
+| ------------------------------------------------------------------------ | ------ | -------- | ------ | ----- | ------------------------ |
+| View runs of [jobs](/guides/build/assets/asset-jobs) | ✅ | ✅ | ✅ | ✅ | ✅ |
+| Launch, re-execute, terminate, and delete runs of jobs | ❌ | ✅ | ✅ | ✅ | ✅ |
+| Start and stop [schedules](/guides/automate/schedules) | ❌ | ❌ | ✅ | ✅ | ✅ |
+| Start and stop [schedules](/guides/automate/sensors) | ❌ | ❌ | ✅ | ✅ | ✅ |
+| Wipe assets | ❌ | ❌ | ✅ | ✅ | ✅ |
+| Launch and cancel [schedules](/guides/automate/schedules) | ❌ | ✅ | ✅ | ✅ | ✅ |
+| Add dynamic partitions | ❌ | ❌ | ✅ | ✅ | ✅ |
+
+### Deployments
+
+Deployment settings are accessed in the UI by navigating to **user menu (your icon) > Organization Settings > Deployments**.
+
+| | Viewer | Launcher | Editor | Admin | Organization admin |
+|----------------------------------------------------------------------------------------------|-------|-----------|--------|-------|-------------------------------|
+| View [deployments](/dagster-plus/deployment/management/deployments/) | ✅ | ✅ | ✅ | ✅ | ✅ |
+| Modify [deployments](/dagster-plus/deployment/management/deployments/) settings | ❌ | ❌ | ✅ | ✅ | ✅ |
+| Create, edit, delete [environment variables](/dagster-plus/deployment/management/environment-variables) | ❌ | ❌ | ✅ | ✅ | ✅ |
+| View [environment variable](/dagster-plus/deployment/management/environment-variables) values | ❌ | ❌ | ✅ | ✅ | ✅ |
+| Export [environment variables](/dagster-plus/deployment/management/environment-variables) | ❌ | ❌ | ✅ | ✅ | ✅ |
+| Create and delete [deployments](/dagster-plus/deployment/management/deployments/) | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Create [Branch Deployments](dagster-plus/features/ci-cd/branch-deployments/index.md) | ❌ | ❌ | ✅ | ✅ | ✅ |
+
+### Code locations
+
+Code locations are accessed in the UI by navigating to **Deployment > Code locations**.
+
+| | Viewer | Launcher | Editor | Admin | Organization admin |
+| ------------------------------------------------------------------------------- | ------ | -------- | ------ | ----- | ------------------------ |
+| View [code locations](/dagster-plus/deployment/code-locations) | ✅ | ✅ | ✅ | ✅ | ✅ |
+| Create and remove [code locations](/dagster-plus/deployment/code-locations) | ❌ | ❌ | ✅ | ✅ | ✅ |
+| Reload [code locations](/dagster-plus/deployment/code-locations) and workspaces | ❌ | ❌ | ✅ | ✅ | ✅ |
+
+### Agent tokens
+
+Agent tokens are accessed in the UI by navigating to **user menu (your icon) > Organization Settings > Tokens**.
+
+| | Viewer | Launcher | Editor | Admin | Organization admin |
+| ----------------------------------------------------------- | ------ | -------- | ------ | ----- | ------------------------ |
+| View [agent tokens](/dagster-plus/deployment/management/tokens/agent-tokens) | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Create agent tokens | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Edit agent tokens | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Revoke agent tokens | ❌ | ❌ | ❌ | ❌ | ✅ |
+
+### User tokens
+
+User tokens are accessed in the UI by navigating to **user menu (your icon) > Organization Settings > Tokens**.
+
+| | Viewer | Launcher | Editor | Admin | Organization admin |
+| ---------------------------------------- | ------ | -------- | ------ | ----- | ------------------------ |
+| View and create own [user tokens](/dagster-plus/deployment/management/tokens/user-tokens) | ✅ | ✅ | ✅ | ✅ | ✅ |
+| List all user tokens | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Revoke all user tokens | ❌ | ❌ | ❌ | ❌ | ✅ |
+
+### Users
+
+User management is accessed in the UI by navigating to **user menu (your icon) > Organization Settings > Users**.
+
+| | Viewer | Launcher | Editor | Admin | Organization admin |
+| --------------------------------------------- | ------ | -------- | ------ | ----- | ------------------------ |
+| [View users](/dagster-plus/features/authentication-and-access-control/rbac/users) | ✅ | ✅ | ✅ | ✅ | ✅ |
+| Add users | ❌ | ❌ | ❌ | ✅ | ✅ |
+| Edit user roles | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Remove users | ❌ | ❌ | ❌ | ❌ | ✅ |
+
+### Teams
+
+Team management is accessed in the UI by navigating to **user menu (your icon) > Organization Settings > Teams**.
+
+**Note**: Admin users can modify teams only in deployments where they're an Admin.
+
+| | Viewer | Launcher | Editor | Admin | Organization admin |
+| --------------------------------------------- | ------ | -------- | ------ | ----- | ------------------------ |
+| [View teams](/dagster-plus/features/authentication-and-access-control/rbac/teams) | ✅ | ✅ | ✅ | ✅ | ✅ |
+| Modify team permissions | ❌ | ❌ | ❌ | ✅ | ✅ |
+| Create teams | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Re-name teams | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Add/remove team members | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Remove teams | ❌ | ❌ | ❌ | ❌ | ✅ |
+
+### Workspace administration
+
+| | Viewer | Launcher | Editor | Admin | Organization admin |
+| ------------------------------------------------------ | ------ | -------- | ------ | ----- | ------------------------ |
+| Manage [alerts](/dagster-plus/features/alerts) | ❌ | ❌ | ✅ | ✅ | ✅ |
+| Edit workspace | ❌ | ❌ | ✅ | ✅ | ✅ |
+{/* | [Administer SAML](/dagster-plus/features/authentication-and-access-control/sso/authentication) | ❌ | ❌ | ❌ | ❌ | ✅ | */}
+| Administer SAML | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Manage SCIM | ❌ | ❌ | ❌ | ❌ | ✅ |
+| View usage | ❌ | ❌ | ❌ | ❌ | ✅ |
+| Manage billing | ❌ | ❌ | ❌ | ❌ | ✅ |
+| View audit logs | ❌ | ❌ | ❌ | ❌ | ✅ |
+
+## Next steps
+
+- Learn more about how to manage users in Dagster+ in [Understanding User Management in Dagster+](/dagster-plus/features/authentication-and-access-control/rbac/users)
+- Learn more about how to manage teams in Dagster+ in [Understanding Team Management in Dagster+](/dagster-plus/features/authentication-and-access-control/rbac/teams)
+- Learn more about SCIM provisioning in [Understanding SCIM Provisioning](/dagster-plus/features/authentication-and-access-control/scim/index.md)
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/users.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/users.md
new file mode 100644
index 0000000000000..7157d25f40ddc
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/rbac/users.md
@@ -0,0 +1,92 @@
+---
+title: "Managing users in Dagster+"
+sidebar_label: "User management"
+sidebar_position: 400
+---
+
+Dagster+ allows you to grant specific permissions to your organization's users, ensuring that Dagster users have access only to what they require.
+
+In this guide, you'll learn how to manage users and their permissions using the Dagster+ UI.
+
+
+Prerequisites
+
+- A Dagster+ account
+- The required [Dagster+ permissions](/dagster-plus/features/authentication-and-access-control/rbac/):
+ - **Organization Admins** can add, manage, and remove users
+ - **Admins** can add users
+
+
+
+## Before you start
+
+- **If System for Cross-domain Identity Management specification (SCIM) provisioning is enabled,** you'll need to add new users in your identity provider (IdP). Adding users will be disabled in Dagster+.
+- **If using Google for Single sign-on (SSO)**, users must be added in Dagster+ before they can log in.
+- **If using an Identity Provider (IdP) like Okta for SSO**, users must be assigned to the Dagster app in the IdP to be able to log in to Dagster+. Refer to the [SSO setup guides](/dagster-plus/features/authentication-and-access-control/sso/) for setup instructions for each of our supported IdP solutions.
+
+By default, users will be granted Viewer permissions on each deployment. The default role can be adjusted by modifying the [`sso_default_role` deployment setting](/dagster-plus/deployment/management/deployments/deployment-settings-reference).
+
+## Adding users to Dagster+
+
+1. Sign in to your Dagster+ account.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. Click the **Users** tab.
+4. Click **Add new user.**
+5. In the **User email** field, enter the user's email address.
+6. Click **Add user**.
+
+After the user is created, they will be notified via email, and you can [add the user to teams](#teams) and [assign user roles for each deployment](#user-roles).
+
+![Screenshot of assigning roles to a user](/images/dagster-plus/features/authentication-and-access-control/adding-new-user.png)
+
+## Adding users to teams \{#teams}
+
+:::note
+Teams are a Dagster+ Pro feature.
+:::
+
+Teams are useful for centralizing permission sets for different types of users. Refer to [Managing teams](teams) for more information about creating and managing teams.
+
+![Screenshot of Managing teams page](/images/dagster-plus/features/authentication-and-access-control/mananging-teams.png)
+
+:::note
+When determining a user's level of access, Dagster+ will use the **most permissive** role assigned to the user between all of their team memberships and any individual role grants. Refer to [Managing user roles and permissions](user-roles-permissions) for more information.
+:::
+
+## Assigning user roles \{#user-roles}
+
+In the **Roles** section, you can assign a [user role](user-roles-permissions) for each deployment, granting them a set of permissions that controls their access to various features and functionalities within the platform.
+
+1. Next to a deployment, click **Edit user role**.
+2. Select the user role for the deployment. This [user role](user-roles-permissions) will be used as the default for all code locations in the deployment.
+3. Click **Save**.
+4. **Pro only**: To set permissions for individual [code locations](/dagster-plus/deployment/code-locations/) in a deployment:
+ 1. Click the toggle to the left of the deployment to open a list of code locations.
+ 2. Next to a code location, click **Edit user role**.
+ 3. Select the user role for the code location.
+ 4. Click **Save**.
+5. Repeat the previous steps for each deployment.
+6. **Optional**: To change the user's permissions for branch deployments:
+ 1. Next to **All branch deployments**, click **Edit user role**.
+ 2. Select the user role to use for all branch deployments.
+ 3. Click **Save**.
+7. Click **Done**.
+
+## Removing users
+
+Removing a user removes them from the organization. **Note**: If using a SAML-based SSO solution like Okta, you'll also need to remove the user from the IdP. Removing the user in Dagster+ doesn't remove them from the IdP.
+
+1. Sign in to your Dagster+ account.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. Click the **Users** tab.
+4. Locate the user in the user list.
+5. Click **Edit**.
+6. Click **Remove user**.
+7. When prompted, confirm the removal.
+
+## Next steps
+
+- Learn more about role-based access control (RBAC) in [Understanding User Roles & Permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions)
+- Learn more about how to manage teams in Dagster+ in [Understanding Team Management in Dagster+](/dagster-plus/features/authentication-and-access-control/rbac/teams)
+- Learn more about SCIM provisioning in [SCIM Provisioning](/dagster-plus/features/authentication-and-access-control/scim)
+- Learn more about authentication in the [SSO documentation](/dagster-plus/features/authentication-and-access-control/sso/)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/azure-active-directory.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/azure-active-directory.md
new file mode 100644
index 0000000000000..a27c033a49b64
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/azure-active-directory.md
@@ -0,0 +1,6 @@
+---
+title: 'Azure Active Directory'
+sidebar_position: 300
+---
+
+See the [Microsoft Dagster Cloud provisioning tutorial](https://learn.microsoft.com/en-us/azure/active-directory/saas-apps/dagster-cloud-provisioning-tutorial).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/enabling-scim-provisioning.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/enabling-scim-provisioning.md
new file mode 100644
index 0000000000000..d219d80a04cb7
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/enabling-scim-provisioning.md
@@ -0,0 +1,26 @@
+---
+title: Enabling SCIM provisioning
+sidebar_position: 100
+---
+
+## Prerequisites
+
+To use SCIM provisioning, you'll need:
+
+- A Dagster+ Pro plan
+- [An IdP for which Dagster+ supports SSO and SCIM provisioning](#supported-identity-providers)
+- Permissions in your IdP that allow you to configure SSO and SCIM provisioning
+
+## Supported Identity Providers
+
+Dagster+ currently supports SCIM provisioning for the following Identity Providers (IdP):
+
+- [Okta](okta-scim)
+- [Microsoft Azure AD](https://learn.microsoft.com/en-us/azure/active-directory/saas-apps/dagster-cloud-provisioning-tutorial)
+
+Use the setup guide for your IdP to get started.
+
+## Related
+
+- [Managing user roles and permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions)
+- [Managing teams](/dagster-plus/features/authentication-and-access-control/rbac/teams)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/index.md
new file mode 100644
index 0000000000000..5f66a900f51c7
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/index.md
@@ -0,0 +1,34 @@
+---
+title: "SCIM provisioning"
+sidebar_position: 30
+---
+
+SCIM provisioning eases the burden of manually provisioning users across your cloud applications. When enabled, you can automatically sync user information from your IdP to Dagster+ and back again, ensuring user data is always up-to-date.
+
+For a detailed look at SCIM provisioning, [check out this blog post](https://www.strongdm.com/blog/scim-provisioning).
+
+## Managing users
+
+When SCIM is enabled in Dagster+, a few things about user management will change:
+
+- **New users must be added in the IdP.** The ability to add new users will be disabled in Dagster+ while SCIM is enabled.
+- **Only 'unsynced' users can be removed in Dagster+.** 'Synced' users will have an icon indicating they're externally managed by the IdP, while unsynced users will not. For example, the first two users in the following image are synced, while the last isn't:
+
+![Highlighted unsynced user in the Dagster+ UI](/images/dagster-plus/features/authentication-and-access-control/scim-unsynced-user.png)
+
+ You might see unsynced users in Dagster+ when:
+
+ - **Users exist in Dagster+, but not in the IdP.** In this case, create matching users in the IdP and then provision them. This will link the IdP users to the Dagster+ users.
+ - **Users are assigned to the Dagster+ IdP app before provisioning is enabled.** In this case, you'll need to provision the users in the IdP to link them to the Dagster+ users.
+
+If you choose to disable SCIM provisioning in Dagster+, users and teams will remain as-is at the time SCIM is disabled.
+
+## Managing teams
+
+In addition to the above user management changes, there are a few things to keep in mind when managing user groups, otherwise known as Dagster+ [teams](/dagster-plus/features/authentication-and-access-control/rbac/teams).
+
+User groups in your IdP can be mapped to Dagster+ teams, allowing you to centralize the management of user groups and memberships. When SCIM is enabled:
+
+- **Teams can still be managed in Dagster+.** You can choose to map and sync these teams to the IdP or administer them solely in Dagster+. Synced groups should be managed only in the IdP, or changes made in Dagster+ may be overwritten when a sync is triggered from the IdP.
+- **If a group exists only in the IdP** and is synced to Dagster+, you'll be prompted to either create a new Dagster+ team with the same name or create a link between the IdP group and an existing team in Dagster+.
+- **If a group exists only in Dagster+**, the group will display in the IdP as an 'external' group with no members. In this case, you can either create a new group in the IdP and link it to an existing Dagster+ team, or choose to manage the team only in Dagster+.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/okta-scim.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/okta-scim.md
new file mode 100644
index 0000000000000..1763736820be5
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/scim/okta-scim.md
@@ -0,0 +1,139 @@
+---
+title: 'Okta SCIM provisioning'
+sidebar_position: 200
+---
+
+The [System for Cross-domain Identity Management specification](https://scim.cloud/) (SCIM) is a standard designed to manage user identity information. When enabled in Dagster+, SCIM allows you to efficiently and easily manage users in your Identity Provider (IdP) - in this case, Okta - and sync their information to Dagster+.
+
+In this guide, we'll walk you through configuring [Okta SCIM provisioning](https://developer.okta.com/docs/concepts/scim/) for Dagster+.
+
+## About this feature
+
+
+
+
+### Supported features
+
+With Dagster+'s Okta SCIM provisioning feature, you can:
+
+- **Create users**. Users that are assigned to the Dagster+ application in the IdP will be automatically added to your Dagster+ organization.
+- **Update user attributes.** Updating a user's name or email address in the IdP will automatically sync the change to your user list in Dagster+.
+- **Remove users.** Deactivating or unassigning a user from the Dagster+ application in the IdP will remove them from the Dagster+ organization
+{/* - **Push user groups.** Groups and their members in the IdP can be pushed to Dagster+ as [Teams](/dagster-plus/account/managing-users/managing-teams). */}
+- **Push user groups.** Groups and their members in the IdP can be pushed to Dagster+ as
+ [Teams](/dagster-plus/features/authentication-and-access-control/rbac/teams).
+
+Refer to [Okta's SCIM documentation](https://developer.okta.com/docs/concepts/scim/) for more information about Okta's SCIM offering.
+
+
+
+
+### Limitations
+
+Dagster+ currently supports the following attributes for SCIM syncing:
+
+- `user.firstName`
+- `user.lastName`
+- `user.email`, which must match the user's username in Okta
+- `user.displayName`
+
+
+
+
+## Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+{/* - **To have set up Okta SSO for Dagster+.** Refer to the [Okta SSO setup guide](/dagster-plus/account/authentication/okta/saml-sso) for more info. */}
+- **To have set up Okta SSO for Dagster+.** Refer to the [Okta SSO setup guide](//dagster-plus/features/authentication-and-access-control/sso/okta-sso) for more info.
+- **Permissions in Okta that allow you to configure applications.**
+- **The following in Dagster+:**
+ - A Pro plan
+ - [Organization Admin permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) in your organization
+
+## Step 1: Enable SCIM provisioning in Dagster+
+
+1. Sign in to your Dagster+ account.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. Click the **Provisioning** tab.
+4. If SCIM provisioning isn't enabled, click the **Enable SCIM provisioning** button to enable it.
+5. Click **Create SCIM token** to create an API token. This token will be used to authenticate requests from Okta to Dagster+.
+
+Keep the API token handy - you'll need it in the next step.
+
+## Step 2: Enable SCIM provisioning in Okta
+
+1. Sign in to your Okta Admin Dashboard.
+
+2. Using the sidebar, click **Applications > Applications**.
+
+3. Click the Dagster+ app. **Note**: If you haven't set up SSO for Okta, [follow this guide](/dagster-plus/features/authentication-and-access-control/sso/okta-sso) to do so before continuing.
+
+4. Click the **Sign On** tab and complete the following:
+
+ 1. Click **Edit**.
+
+ 2. In the **Advanced Sign-on Settings** section, enter the name of your organization in the **Organization** field.
+
+ 3. In the **Credential Details** section, set the **Application username format** field to **Email**:
+
+
+ ![Configured Sign On tab of Dagster+ Okta application](/images/dagster-plus/features/authentication-and-access-control/okta/scim-sign-on-tab.png)
+
+ 4. Click **Save**.
+
+5. Click the **Provisioning** tab and complete the following:
+
+ 1. Click **Configure API Integration**.
+
+ 2. Check the **Enable API integration** checkbox that displays.
+
+ 3. In the **API Token** field, paste the Dagster+ API token you generated in [Step 1](#step-1-enable-scim-provisioning-in-dagster):
+
+ ![Configured Provisioning tab of Dagster+ Okta application](/images/dagster-plus/features/authentication-and-access-control/okta/provisioning-tab.png)
+
+ 4. Click **Test API Credentials** to verify that your organization and API token work correctly.
+
+ 5. When finished, click **Save**.
+
+## Step 3: Enable user syncing in Okta
+
+After you confirm that your API credentials work in the Dagster+ Okta application, you can enable user syncing:
+
+1. In the Dagster+ Okta app, click the **Provisioning** tab.
+
+2. In the **Settings** panel, click **To App**.
+
+3. Click **Edit**.
+
+4. Next to **Create Users**, check the **Enable** checkbox:
+
+ ![Highlighted Create users setting and default username setting in Okta](/images/dagster-plus/features/authentication-and-access-control/okta/provisioning-to-app-create-users.png)
+
+ **Note**: The default username used to create accounts must be set to **Email** or user provisioning may not work correctly.
+
+5. Optionally, check **Enable** next to **Update User Attributes** and **Deactivate Users** to enable these features.
+
+6. When finished, click **Save**.
+
+## Step 4: Enable group syncing in Okta
+
+{/*
+:::note
+ This step is required only if you want to sync Okta user groups to Dagster+ as [Teams](/dagster-plus/account/managing-users/managing-teams).
+:::
+*/}
+:::note
+ This step is required only if you want to sync Okta user groups to Dagster+ as [Teams](/dagster-plus/features/authentication-and-access-control/rbac/teams).
+:::
+
+{/* When **Push groups** is enabled in Okta, you can sync user groups from Okta to Dagster+ as [Teams](/dagster-plus/account/managing-users/managing-teams). Refer to the [Okta documentation](https://help.okta.com/oie/en-us/Content/Topics/users-groups-profiles/usgp-enable-group-push.htm) for setup instructions. */}
+When **Push groups** is enabled in Okta, you can sync user groups from Okta to Dagster+ as [Teams](/dagster-plus/features/authentication-and-access-control/rbac/teams). Refer to the [Okta documentation](https://help.okta.com/oie/en-us/Content/Topics/users-groups-profiles/usgp-enable-group-push.htm) for setup instructions.
+
+## Next steps
+
+That's it! Once Okta successfully syncs users to Dagster+, synced users will have a 'synced' icon next to them in the Dagster+ users page:
+
+ ![Synced/external user icon next to user in Dagster+ user list](/images/dagster-plus/features/authentication-and-access-control/dagster-cloud-external-user.png)
+
+Refer to the [Enabling SCIM provisioning guide](/dagster-plus/features/authentication-and-access-control/scim/enabling-scim-provisioning) for more info about how user and team management works when SCIM provisioning is enabled.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/azure-ad-sso.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/azure-ad-sso.md
new file mode 100644
index 0000000000000..487d6c326a2ef
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/azure-ad-sso.md
@@ -0,0 +1,95 @@
+---
+title: Setting up Azure Active Directory SSO for Dagster+
+sidebar_label: 'Azure Active Directory SSO'
+sidebar_position: 200
+---
+
+In this guide, you'll configure Azure Active Directory (AD) to use single sign-on (SSO) with your Dagster+ organization.
+
+
+ Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- **An existing Azure AD account**
+- **To install the [`dagster-cloud` CLI](/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring)**
+- **The following in Dagster+:**
+ - A Pro plan
+ - [Access to a user token](/dagster-plus/deployment/management/tokens/user-tokens)
+ - [Organization Admin permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) in your organization
+
+
+
+## Step 1: add the Dagster+ app in Azure AD \{#dagster-app}
+
+In this step, you'll add the Dagster+ app to your list of managed SaaS apps in Azure AD.
+
+1. Sign in to the Azure portal.
+2. On the left navigation pane, click the **Azure Active Directory** service.
+3. Navigate to **Enterprise Applications** and then **All Applications**.
+4. Click **New application**.
+5. In the **Add from the gallery** section, type **Dagster+** in the search box.
+6. Select **Dagster+** from the results panel and then add the app. Wait a few seconds while the app is added to your tenant.
+
+## Step 2: configure SSO in Azure AD \{#configure-sso}
+
+In this step, you'll configure and enable SSO for Azure AD in your Azure portal.
+
+1. On the **Dagster+** application integration page, locate the **Manage** section and select **single sign-on**.
+2. On the **Select a single sign-on method** page, select **SAML**.
+3. On the **Set up single sign-on with SAML** page, click the pencil icon for **Basic SAML Configuration** to edit the settings.
+
+ ![Settings Dropdown](/images/dagster-plus/features/authentication-and-access-control/azure/set-up-single-sign-on.png)
+4. In the **Basic SAML Configuration** section, fill in the **Identifier** and **Reply URL** fields as follows:
+
+ Copy and paste the following URL, replacing `` with your Dagster+ organization name:
+
+ ```
+ https://.dagster.cloud/auth/saml/consume
+ ```
+
+5. Click **Set additional URLs**.
+6. In the **Sign-on URL** field, copy and paste the URL you entered in the **Identifier** and **Reply URL** fields.
+7. Next, you'll configure the SAML assertions. In addition to the default attributes, Dagster+ requires the following:
+
+ - `FirstName` - `user.givenname`
+ - `LastName` - `user.surname`
+ - `Email` - `user.userprincipalname`
+
+ Add these attribute mappings to the SAML assertion.
+8. On the **Set up single sign-on with SAML** page:
+ 1. Locate the **SAML Signing Certificate** section.
+ 2. Next to **Federation Metadata XML**, click **Download**:
+
+ ![Download SAML Certificate](/images/dagster-plus/features/authentication-and-access-control/azure/download.png)
+
+ When prompted, save the SAML metadata file to your computer.
+
+## Step 3: upload the SAML metadata to Dagster+ \{#upload-saml}
+
+After you've downloaded the SAML metadata file, upload it to Dagster+ using the `dagster-cloud` CLI:
+
+```shell
+dagster-cloud organization settings saml upload-identity-provider-metadata \
+ --api-token= \
+ --url https://.dagster.cloud
+```
+
+## Step 4: create a test user \{#test-user}
+
+In this section, you'll create a test user in the Azure portal.
+
+1. From the left pane in the Azure portal, click **Azure Active Directory**.
+2. Click **Users > All users**.
+3. Click **New user** at the top of the screen.
+4. In **User** properties, fill in the following fields:
+ - **Name**: Enter `B.Simon`.
+ - **User name**: Enter `B.Simon@contoso.com`.
+ - Select the **Show password** checkbox and write down the value displayed in the **Password** box.
+5. Click **Create**.
+
+import TestSSO from '../../../../partials/\_TestSSO.md';
+
+
+
+Click **Test this application** in the Azure portal. If successful, you'll be automatically signed into your Dagster+ organization.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/google-workspace-sso.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/google-workspace-sso.md
new file mode 100644
index 0000000000000..7c205694b49ed
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/google-workspace-sso.md
@@ -0,0 +1,109 @@
+---
+title: Setting up Google Workspace SSO for Dagster+
+sidebar_label: 'Google Workspace SSO'
+sidebar_position: 300
+---
+
+In this guide, you'll configure Google Workspace to use single sign-on (SSO) with your Dagster+ organization.
+
+
+ Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- **The following in Google**:
+ - An existing Google account
+ - [Workspace Admin permissions](https://support.google.com/a/answer/6365252?hl=en&ref_topic=4388346)
+- **To install the [`dagster-cloud` CLI](/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring)**
+- **The following in Dagster+:**
+ - A Pro plan
+ - [Access to a user token](/dagster-plus/deployment/management/tokens/user-tokens)
+ - [Organization Admin permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) in your organization
+
+
+
+## Step 1: Add the Dagster+ app in Google Workspace \{#dagster-app}
+
+1. Navigate to your [Google Admin Console](https://admin.google.com).
+2. Using the sidebar, navigate to **Apps > Web and mobile apps**:
+
+ ![Google Workspace Sidebar](/images/dagster-plus/features/authentication-and-access-control/google-workspace/sidebar.png)
+
+3. On the **Web and mobile apps** page, click **Add App > Add custom SAML app**:
+ ![Add App](/images/dagster-plus/features/authentication-and-access-control/google-workspace/add-app.png)
+ This opens a new page for adding app details.
+
+## Step 2: Configure SSO in Google Workspace \{#configure-sso}
+
+1. On the **App details** page:
+ 1. Fill in the **App name** field.
+ 2. Fill in the **Description** field.
+
+ The page should look similar to the following:
+
+ ![Application Details](/images/dagster-plus/features/authentication-and-access-control/google-workspace/application-details.png)
+
+ 3. Click **Continue**.
+
+2. On the **Google Identity Provider details** page, click **Continue**. No action is required for this page.
+3. On the **Service provider details** page:
+ 1. In the **ACS URL** and **Entity ID** fields:
+
+ Copy and paste the following URL, replacing `` with your Dagster+ organization name:
+
+ ```
+ https://.dagster.cloud/auth/saml/consume
+ ```
+
+ 2. Check the **Signed Response** box. The page should look similar to the image below. In this example, the organization's name is `hooli` and the Dagster+ domain is `https://hooli.dagster.cloud`:
+
+ ![Service Provider Details](/images/dagster-plus/features/authentication-and-access-control/google-workspace/service-provider-details.png)
+
+ 3. When finished, click **Continue**.
+4. On the **Attributes** page:
+ 1. Click **Add mapping** to add and configure the following attributes:
+
+ - **Basic Information > First Name** - `FirstName`
+ - **Basic Information > Last Name** - `LastName`
+ - **Basic Information > Email** - `Email`
+
+ The page should look like the following image:
+
+ ![Attribute Mapping](/images/dagster-plus/features/authentication-and-access-control/google-workspace/attribute-mapping.png)
+
+ 2. Click **Finish**.
+
+## Step 3: Upload the SAML metadata to Dagster+ \{#upload-saml}
+
+Next, you'll save and upload the application's SAML metadata to Dagster+. This will enable single sign-on.
+
+1. In your Google Workspace, open the Dagster+ application you added in [Step 2](#configure-sso).
+2. Click **Download metadata**:
+
+ ![SAML Metadata](/images/dagster-plus/features/authentication-and-access-control/google-workspace/saml-metadata.png)
+
+3. In the modal that displays, click **Download metadata** to start the download. Save the file to your computer.
+4. After you've downloaded the SAML metadata file, upload it to Dagster+ using the `dagster-cloud` CLI:
+
+ ```shell
+ dagster-cloud organization settings saml upload-identity-provider-metadata \
+ --api-token= \
+ --url https://.dagster.cloud
+ ```
+
+## Step 4: Grant access to users \{#grant-access}
+
+In this step, you'll assign users in your Google Workspace to the Dagster+ application. This allows members of the workspace to log in to Dagster+ using their credentials when the single sign-on flow is initiated.
+
+1. In the Google Workspace Dagster+ application, click **User access**.
+2. Select an organizational unit.
+3. Click **ON for everyone**.
+4. Click **Save**.
+
+ ![Assign New Login](/images/dagster-plus/features/authentication-and-access-control/google-workspace/new-login.png)
+
+import TestSSO from '../../../../partials/\_TestSSO.md';
+
+
+
+In the Google Workspace portal, click the **Dagster+ icon**. If successful, you'll be automatically signed into your Dagster+ organization.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/index.md
new file mode 100644
index 0000000000000..e5711aad31dd8
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/index.md
@@ -0,0 +1,8 @@
+---
+title: "Single sign-on"
+sidebar_position: 20
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/okta-sso.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/okta-sso.md
new file mode 100644
index 0000000000000..74d163b37ad64
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/okta-sso.md
@@ -0,0 +1,93 @@
+---
+title: Setting up Okta SSO for Dagster+
+sidebar_label: 'Okta SSO'
+sidebar_position: 400
+---
+
+In this guide, you'll configure Okta to use single sign-on (SSO) with your Dagster+ organization.
+
+
+ Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- **An existing Okta account**
+- **To install the [`dagster-cloud` CLI](/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring)**
+- **The following in Dagster+:**
+ - A Pro plan
+ - [Access to a user token](/dagster-plus/deployment/management/tokens/user-tokens)
+ - [Organization Admin permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) in your organization
+
+
+
+
+## Step 1: Add the Dagster+ app in Okta \{#dagster-app}
+
+1. Sign in to your Okta Admin Dashboard.
+2. Using the sidebar, click **Applications > Applications**.
+3. On the **Applications** page, click **Browse App Catalog**.
+4. On the **Browse App Integration Catalog** page, search for `Dagster Cloud`.
+5. Add and save the application.
+
+
+## Step 2: Configure SSO in Okta \{#configure-sso}
+
+1. In Okta, open the Dagster Cloud application and navigate to its **Sign On Settings**.
+2. Scroll down to the **Advanced Sign-on settings** section.
+3. In the **Organization** field, enter your Dagster+ organization name. This is used to route the SAML response to the correct Dagster+ subdomain.
+
+ For example, your organization name is `hooli` and your Dagster+ domain is `https://hooli.dagster.cloud`. To configure this correctly, you'd enter `hooli` into the **Organization** field:
+
+ ![Okta Subdomain Configuration](/images/dagster-plus/features/authentication-and-access-control/okta/subdomain-configuration.png)
+
+4. When finished, click **Done**.
+
+
+## Step 3: Upload the SAML metadata to Dagster+ \{#upload-saml}
+
+Next, you'll save and upload the application's SAML metadata to Dagster+. This will enable single sign-on.
+
+1. In the **Sign On Settings**, navigate to the **SAML Signing Certificates** section.
+2. Click the **Actions** button of the **Active** certificate.
+3. Click **View IdP metadata**:
+
+ ![Okta IdP metadata options](/images/dagster-plus/features/authentication-and-access-control/okta/save-identity-provider-metadata.png)
+
+ This will open a new page in your browser with the IdP metadata in XML format.
+
+4. Right-click the page and use **Save As** or **Save Page As**:
+
+ ![Save IdP metadata as XML](/images/dagster-plus/features/authentication-and-access-control/okta/save-xml-as-page.png)
+
+ In Chrome and Edge, the file will be downloaded as an XML file. In Firefox, choose **Save Page As > Save as type**, then select **All files**.
+
+ :::note
+ Copying and pasting the metadata can cause formatting issues that will prevent successful setup. Saving the page directly from the browser will avoid this.
+ :::
+
+5. After you've downloaded the metadata file, upload it to Dagster+ using the `dagster-cloud` CLI:
+
+ ```shell
+ dagster-cloud organization settings saml upload-identity-provider-metadata \
+ --api-token= \
+ --url https://.dagster.cloud
+ ```
+
+
+## Step 4: Grant access to users \{#grant-access}
+
+Next, you'll assign users to the Dagster+ application in Okta. This will allow them to log in using their Okta credentials when the single sign-on flow is initiated.
+
+1. In the Dagster+ application, navigate to **Assignments**.
+2. Click **Assign > Assign to People**.
+3. For each user you want to have access to Dagster+, click **Assign** then **Save and Go Back**.
+
+import TestSSO from '../../../../partials/\_TestSSO.md';
+
+
+
+In the Okta **Applications** page, click the **Dagster+** icon:
+
+![Okta idP Login](/images/dagster-plus/features/authentication-and-access-control/okta/idp-login.png)
+
+If successful, you'll be automatically signed into your Dagster+ organization.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/onelogin-sso.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/onelogin-sso.md
new file mode 100644
index 0000000000000..25a5f75bedbdd
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/onelogin-sso.md
@@ -0,0 +1,89 @@
+---
+title: 'OneLogin SSO'
+sidebar_position: 500
+---
+
+In this guide, you'll configure OneLogin to use single sign-on (SSO) with your Dagster+ organization.
+
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- **The following in OneLogin:**
+ - An existing OneLogin account
+ - Admin permissions
+- **To install the [`dagster-cloud` CLI](/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring)**
+- **The following in Dagster+:**
+ - A Pro plan
+ - [Access to a user token](/dagster-plus/deployment/management/tokens/user-tokens)
+ - [Organization Admin permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) in your organization
+
+
+
+
+## Step 1: Add the Dagster+ app in OneLogin \{#dagster-app}
+
+1. Sign into your OneLogin portal.
+2. Navigate to **Administration > Applications**.
+3. On the **Applications** page, click **Add App**.
+4. On the **Find Applications** page, search for `Dagster+`:
+
+ ![Find Applications in OneLogin](/images/dagster-plus/features/authentication-and-access-control/onelogin/find-applications.png)
+
+5. Add and save the application.
+
+
+
+## Step 2: Configure SSO in OneLogin \{#configure-sso}
+
+1. In OneLogin, open the application and navigate to its **Configuration**.
+2. In the **Dagster+ organisation name** field, enter your Dagster+ organization name. This is used to route the SAML response to the correct Dagster+ subdomain.
+
+ For example, your organization name is `hooli` and your Dagster+ domain is `https://hooli.dagster.cloud`. To configure this correctly, you'd enter `hooli` into the **Subdomain** field.
+3. When finished, click **Done**.
+
+
+
+## Step 3: Upload the SAML metadata to Dagster+ \{#upload-saml}
+
+Next, you'll save and upload the application's SAML metadata to Dagster+. This will enable single sign-on.
+
+1. In OneLogin, open the Dagster+ application.
+2. Navigate to **More Actions > SAML Metadata**.
+3. When prompted, save the file to your computer.
+4. After you've downloaded the SAML metadata file, upload it to Dagster+ using the `dagster-cloud` CLI:
+
+ ```shell
+ dagster-cloud organization settings saml upload-identity-provider-metadata \
+ --api-token= \
+ --url https://.dagster.cloud
+ ```
+
+
+## Step 4: Grant access to users \{#grant-access}
+
+Next, you'll assign users to the Dagster+ application in OneLogin. This will allow them to log in using their OneLogin credentials with the sign in flow is initiated.
+
+1. In Okta, navigate to **Users**.
+2. Select a user.
+3. On the user's page, click **Applications**.
+4. Assign the user to Dagster+. In the following image, the user `Test D'Test` has been assigned to Dagster+:
+
+ ![Screenshot of Assign New Login in OneLogin](/images/dagster-plus/features/authentication-and-access-control/onelogin/new-login.png)
+
+5. Click **Continue**.
+6. Click **Save User.**
+7. Repeat steps 2-6 for every user you want to access Dagster+.
+
+
+import TestSSO from '../../../../partials/\_TestSSO.md';
+
+
+
+In the OneLogin portal, click the Dagster+ icon:
+
+![Screenshot of the Dagster+ icon in OneLogin](/images/dagster-plus/features/authentication-and-access-control/onelogin/onelogin-dagster.png)
+
+If successful, you'll be automatically signed into your Dagster+ organization.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/pingone-sso.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/pingone-sso.md
new file mode 100644
index 0000000000000..633b87b141799
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/authentication-and-access-control/sso/pingone-sso.md
@@ -0,0 +1,119 @@
+---
+title: Setting up PingOne SSO for Dagster+
+sidebar_label: 'PingOne SSO'
+sidebar_position: 600
+---
+
+In this guide, you'll configure PingOne to use single sign-on (SSO) with your Dagster+ organization.
+
+
+ Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- **The following in PingOne:**
+ - An existing PingOne account
+ - Organization admin permissions
+- **To install the [`dagster-cloud` CLI](/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring)**
+- **The following in Dagster+:**
+ - A Pro plan
+ - [Access to a user token](/dagster-plus/deployment/management/tokens/user-tokens)
+ - [Organization Admin permissions](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions) in your organization
+
+
+
+## Step 1: Add the Dagster+ app in PingOne \{#dagster-app}
+
+1. Sign into your PingOne Console.
+2. Using the sidebar, click **Connections > Applications**.
+
+ ![PingOne Sidebar](/images/dagster-plus/features/authentication-and-access-control/pingone/sidebar.png)
+
+3. On the **Applications** page, add an application.
+4. In **Select an application type**, click **Web app**.
+5. Click **SAML > Configure**:
+
+ ![Add App](/images/dagster-plus/features/authentication-and-access-control/pingone/add-app.png)
+
+## Step 2: Configure SSO in PingOne \{#configure-sso}
+
+1. In the **Create App Profile** page:
+
+ 1. Add an application name, description, and icon:
+
+ ![Application Details](/images/dagster-plus/features/authentication-and-access-control/pingone/application-details.png)
+
+ 2. When finished, click **Save and Continue.**
+
+2. In the **Configure SAML** page:
+
+ 1. Fill in the following:
+
+ - **ACS URLs** and **Entity ID**: Copy and paste the following URL, replacing `` with your Dagster+ organization name:
+
+ ```
+ https://.dagster.cloud/auth/saml/consume
+ ```
+
+ - **Assertion Validity Duration**: Type `60`.
+ In the following example, the organization's name is `hooli` and the Dagster+ domain is `https://hooli.dagster.cloud`:
+
+ ![Service Provider Details](/images/dagster-plus/features/authentication-and-access-control/pingone/service-provider-details.png)
+
+ 2. When finished, click **Save and Continue.**
+
+3. In the **Map Attributes** page:
+
+ 1. Configure the following attributes:
+
+ | Application attribute | Outgoing value |
+ | --------------------- | -------------- |
+ | Email | Email Address |
+ | FirstName | Given Name |
+ | LastName | Family Name |
+
+ The page should look similar to the following:
+
+ ![Attribute Mapping](/images/dagster-plus/features/authentication-and-access-control/pingone/attribute-mapping.png)
+
+ 2. When finished, click **Save and Continue.**
+
+## Step 3: Upload the SAML metadata to Dagster+ \{#upload-saml}
+
+Next, you'll save and upload the application's SAML metadata to Dagster+. This will enable single sign-on.
+
+1. In PingOne, open the Dagster+ application.
+2. Click the **Configuration** tab.
+3. In the **Connection Details** section, click **Download Metadata**:
+
+ ![SAML Metadata](/images/dagster-plus/features/authentication-and-access-control/pingone/saml-metadata.png)
+
+4. When prompted, save the file to your computer.
+5. After you've downloaded the SAML metadata file, upload it to Dagster+ using the `dagster-cloud` CLI:
+
+ ```shell
+ dagster-cloud organization settings saml upload-identity-provider-metadata \
+ --api-token= \
+ --url https://.dagster.cloud
+ ```
+
+## Step 4: Grant access to users \{#grant-access}
+
+Next, you'll assign users to the Dagster+ application in PingOne. This will allow them to log in using their PingOne credentials when the single sign-on flow is initiated.
+
+1. In the Dagster+ application, click the **Access** tab.
+2. Click the **pencil icon** to edit the **Group membership policy**:
+
+ ![Assign New Login](/images/dagster-plus/features/authentication-and-access-control/pingone/new-login.png)
+
+3. Edit the policy as needed to grant users access to the application.
+
+import TestSSO from '../../../../partials/\_TestSSO.md';
+
+
+
+In the PingOne application portal, click the **Dagster+** icon:
+
+![Identity Provider Login](/images/dagster-plus/features/authentication-and-access-control/pingone/identity-provider-login.png)
+
+If successful, you'll be automatically signed in to your Dagster+ organization.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/change-tracking.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/change-tracking.md
new file mode 100644
index 0000000000000..803e1aba6f659
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/change-tracking.md
@@ -0,0 +1,211 @@
+---
+title: "Change tracking in branch deployments"
+sidebar_position: 200
+unlisted: true
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+Branch Deployments Change Tracking makes it easier for you and your team to identify how changes in a pull request will impact data assets. By the end of this guide, you'll understand how Change Tracking works and what types of asset changes can be detected.
+
+## How it works
+
+Branch Deployments compare asset definitions in the branch deployment against the asset definitions in the main deployment. The UI will then mark changed assets accordingly. For example, if the pull request associated with the branch deployment adds a new asset, the UI will display a label indicating the addition.
+
+You can also apply filters to show only new and changed assets in the UI. This makes it easy to understand which assets will be impacted by the changes in the pull request associated with the branch deployment.
+
+{/* **Note:** The default main deployment is `prod`. To configure a different deployment as the main deployment, [create a branch deployment using the dagster-cloud CLI](/dagster-plus/managing-deployments/branch-deployments/using-branch-deployments) and specify it using the optional `--base-deployment-name` parameter. */}
+**Note:** The default main deployment is `prod`. To configure a different deployment as the main deployment, [create a branch deployment using the dagster-cloud CLI](using-branch-deployments-with-the-cli) and specify it using the optional `--base-deployment-name` parameter.
+
+## Supported change types
+
+Change Tracking can detect the following changes to assets:
+
+- [New assets](#new-assets)
+- [Code versions](#code-versions)
+- [Upstream dependencies](#upstream-dependencies)
+- [Partitions definitions](#partitions-definitions)
+- [Tags](#tags)
+- [Metadata](#metadata)
+
+### New assets
+
+If an asset is new in the branch deployment, the asset will have a **New in branch** label in the asset graph:
+
+![Change tracking new](/images/dagster-plus/deployment/management/managing-deployments/change-tracking-new.png)
+
+### Code versions
+
+If using the `code_version` argument on the asset decorator, Change Tracking can detect when this value changes.
+
+Some Dagster integrations, like `dagster-dbt`, automatically compute code versions for you. For more information on code versions, refer to the [Code versioning guide](/guides/build/assets/asset-versioning-and-caching).
+
+
+
+
+In this example, the `customers` asset has a **Changed in branch** label indicating its `code_version` has been changed.
+
+Click the **Asset definition** tab to see the code change that created this label.
+
+![Change tracking code version](/images/dagster-plus/deployment/management/managing-deployments/change-tracking-code-version.png)
+
+
+
+
+**In the main branch**, we have a `customers` asset with a code version of `v1`:
+
+```python file=/dagster_cloud/branch_deployments/change_tracking_code_version.py startafter=start_main_deployment endbefore=end_main_deployment dedent=4
+@asset(code_version="v1")
+def customers(): ...
+```
+
+**In the pull request**, `customers` is modified to change the code version to `v2`:
+
+```python file=/dagster_cloud/branch_deployments/change_tracking_code_version.py startafter=start_branch_deployment endbefore=end_branch_deployment dedent=4
+@asset(code_version="v2")
+def customers(): ...
+```
+
+
+
+
+### Upstream dependencies
+
+Change Tracking can detect when an asset's upstream dependencies have changed, whether they've been added or removed.
+
+**Note**: If an asset is marked as having changed dependencies, it means that the defining its upstream dependencies have changed. It doesn't mean that an upstream dependency has new data.
+
+
+
+
+In this example, the `returns` asset has a **Changed in branch** label indicating it has changed dependencies.
+
+Click the **Asset definition** tab to see the code change that created this label.
+
+![Change tracking dependencies](/images/dagster-plus/deployment/management/managing-deployments/change-tracking-dependencies.png)
+
+```python file=/dagster_cloud/branch_deployments/change_tracking_dependencies.py startafter=start_branch_deployment endbefore=end_branch_deployment dedent=4
+@asset(deps=[orders, customers])
+def returns(): ...
+```
+
+
+
+
+### Partitions definitions
+
+Change Tracking can detect if an asset's has been changed, whether it's been added, removed, or updated.
+
+
+
+
+In this example, the `weekly_orders` asset has a **Changed in branch** label indicating a changed partitions definition.
+
+Click the **Asset definition** tab to see the code change that created this label.
+
+![Change tracking partitions](/images/dagster-plus/deployment/management/managing-deployments/change-tracking-partitions.png)
+
+
+
+
+**In the main branch**, we have a `weekly_orders` asset:
+
+```python file=/dagster_cloud/branch_deployments/change_tracking_partitions_definition.py startafter=start_main_deployment endbefore=end_main_deployment dedent=4
+@asset(partitions_def=WeeklyPartitionsDefinition(start_date="2024-01-01"))
+def weekly_orders(): ...
+```
+
+**In the pull request**, we updated the to start one year earlier:
+
+```python file=/dagster_cloud/branch_deployments/change_tracking_partitions_definition.py startafter=start_branch_deployment endbefore=end_branch_deployment dedent=4
+@asset(partitions_def=WeeklyPartitionsDefinition(start_date="2023-01-01"))
+def weekly_orders(): ...
+```
+
+
+
+
+### Tags
+
+{/* Change Tracking can detect when an [asset's tags](/concepts/metadata-tags/tags) have changed, whether they've been added, modified, or removed. */}
+Change Tracking can detect when an [asset's tags](/guides/build/assets/metadata-and-tags/tags) have changed, whether they've been added, modified, or removed.
+
+
+
+
+In this example, the `fruits_in_stock` asset has a **Changed in branch** label indicating it has changed tags.
+
+Click the **Asset definition** tab to see the code change that created this label.
+
+![Change tracking tags](/images/dagster-plus/deployment/management/managing-deployments/change-tracking-tags.png)
+
+
+
+
+**In the main branch**, we have a `fruits_in_stock` asset:
+
+```python file=/dagster_cloud/branch_deployments/change_tracking_tags.py startafter=start_main_deployment endbefore=end_main_deployment dedent=4
+@asset(tags={"section": "produce"})
+def fruits_in_stock(): ...
+```
+
+**In the pull request**, we added the `type: perishable` tag to `fruits_in_stock`:
+
+```python file=/dagster_cloud/branch_deployments/change_tracking_tags.py startafter=start_branch_deployment endbefore=end_branch_deployment dedent=4
+@asset(tags={"section": "produce", "type": "perishable"})
+def fruits_in_stock(): ...
+```
+
+
+
+
+### Metadata
+
+{/* Change Tracking can detect when an [asset's definition metadata](/concepts/metadata-tags/asset-metadata#attaching-definition-metadata) has changed, whether it's been added, modified, or removed. */}
+Change Tracking can detect when an [asset's definition metadata](/guides/build/assets/metadata-and-tags/) has changed, whether it's been added, modified, or removed.
+
+
+
+
+In this example, the `produtcs` asset has a **Changed in branch** label indicating it has changed metadata.
+
+Click the **Asset definition** tab to see the code change that created this label.
+/images/dagster-plus/management/managing-deployments/change-tracking-metadata.png)
+
+
+
+
+**In the main branch**, we have a `products` asset:
+
+```python file=/dagster_cloud/branch_deployments/change_tracking_metadata.py startafter=start_main_deployment endbefore=end_main_deployment dedent=4
+@asset(metadata={"expected_columns": ["sku", "price", "supplier"]})
+def products(): ...
+```
+
+**In the pull request**, we update the value of the `expected_columns` metadata on `products`:
+
+```python file=/dagster_cloud/branch_deployments/change_tracking_metadata.py startafter=start_branch_deployment endbefore=end_branch_deployment dedent=4
+@asset(metadata={"expected_columns": ["sku", "price", "supplier", "backstock"]})
+def products(): ...
+```
+
+
+
+
+## Related
+
+{/*
+
+
+
+
+
+*/}
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/index.md
new file mode 100644
index 0000000000000..39e2db6935879
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/index.md
@@ -0,0 +1,8 @@
+---
+title: Branch deployments
+sidebar_position: 20
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/setting-up-branch-deployments.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/setting-up-branch-deployments.md
new file mode 100644
index 0000000000000..e10c1ed6cc986
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/setting-up-branch-deployments.md
@@ -0,0 +1,446 @@
+---
+title: "Setting up branch deployments"
+sidebar_position: 100
+---
+
+In this guide, we'll walk you through setting up Branch Deployments for a code location. Once you're finished, any time a PR is created or updated in your repository, it will automatically create or update an associated branch deployment in Dagster+.
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- **Organization Admin** permissions in Dagster+
+- The ability to run a new agent in your infrastructure (only if you are using a **Hybrid deployment**)
+
+
+
+## Step 1: Choose a method
+
+Choose a method for setting up branch deployments:
+
+
+
+
+You can set up GitHub to automatically create branch deployments for new PRs, using GitHub Actions.
+
+Using this approach to branch deployments may be a good fit if:
+
+- You use **GitHub** for version control
+- You want Dagster to fully automate Branch Deployments
+
+This approach is simplified if you use the [GitHub integration](/integrations/libraries/github) to import your project into Dagster+.
+
+
+
+
+You can set up GitLab to automatically create branch deployments for new PRs, using GitLab's CI/CD workflow.
+
+Using this approach to branch deployments may be a good fit if:
+
+- You use **GitLab** for version control
+- You want Dagster to fully automate Branch Deployments
+
+This approach is simplified if you use the [GitLab integration](/todo) to import your project into Dagster+.
+
+
+
+
+You can manually execute dagster-cloud CLI commands to deploy and manage branch deployments.
+
+Using this approach to branch deployments may be a good fit if:
+
+- You don't use GitHub or GitLab for version control
+- You use an alternative CI platform
+- You want full control over Branch Deployment configuration
+
+This is a more advanced option than the other methods.
+
+
+
+
+## Step 2: Generate a Dagster+ agent token
+
+In this step, you'll generate a token for the Dagster+ agent. The Dagster+ agent will use this to authenticate to the agent API.
+
+1. Sign in to your Dagster+ instance.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. In the **Organization Settings** page, click the **Tokens** tab.
+4. Click the **Create agent token** button.
+5. After the token has been created, click **Reveal token**.
+
+Keep the token somewhere handy - you'll need it to complete the setup.
+
+## Step 3: Create and configure an agent
+
+:::note
+If using [Serverless deployment](/dagster-plus/deployment/deployment-types/serverless), this step can be skipped.
+:::
+
+While you can use your existing production agent, we recommend creating a dedicated branch deployment agent. This ensures that your production instance isn't negatively impacted by the workload associated with branch deployments.
+
+
+
+
+ 1. **Deploy an ECS agent to serve your branch deployments**. Follow the [ECS agent](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/new-vpc) setup guide, making sure to set the **Enable Branch Deployments** parameter if using the CloudFormation template. If you are running an existing agent, follow the [upgrade guide](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/existing-vpc) to ensure your template is up-to-date. Then, turn on the **Enable Branch Deployments** parameter.
+
+ 2. **Create a private [Amazon Elastic Registry (ECR) repository](https://console.aws.amazon.com/ecr/repositories).** Refer to the [AWS ECR documentation](https://docs.aws.amazon.com/AmazonECR/latest/userguide/repository-create.html) for instructions.
+
+ After the repository has been created, navigate back to the list of [ECR repositories](https://console.aws.amazon.com/ecr/repositories).
+
+ In the list, locate the repository and its **URI**:
+
+ ![Show this in the UI](/images/dagster-plus/features/branch-deployments/aws-ecr-uri.png)
+
+ Keep this around, as you'll need it in a later step.
+
+ 3. [**Create an IAM user.**](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_users_create.html) This user must:
+
+ - Have push access to the ECR repository, and
+ - Have programmatic access to AWS using an [access key](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_credentials_access-keys.html)
+
+ After the user is created, save the **Access key ID** and **Secret access key** values shown on the confirmation page:
+
+ ![Show this in the UI](/images/dagster-plus/features/branch-deployments/aws-iam-user-keys.png)
+
+
+
+
+ 1. Set up a new Docker agent. Refer to the [Docker agent setup guide](/dagster-plus/deployment/deployment-types/hybrid/docker) for instructions.
+ 2. After the agent is set up, modify the `dagster.yaml` file as follows:
+
+ - Set the `dagster_cloud_api.branch_deployments` field to `true`
+ - Remove any `deployment` field(s)
+
+ For example:
+
+
+
+
+
+
+ 1. Set up a new Kubernetes agent. Refer to the [Kubernetes agent setup guide](/dagster-plus/deployment/deployment-types/hybrid/kubernetes) for instructions.
+
+ 2. After the agent is set up, modify your Helm values file to include the following:
+
+
+
+
+
+
+## Step 4: Set up branch deployments
+
+
+
+
+### Step 4.1: Add GitHub CI/CD script to your project
+:::note
+If you used the GitHub app to configure you're repository, this step isn't required. [Skip ahead to Step 4.5](#step-45-verify-github-action-runs)
+:::
+
+Copy the following files to your project, and **replace** all references to `quickstart-etl` with the name of your project:
+
+- [`dagster_cloud.yaml`](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/blob/main/dagster_cloud.yaml)
+- [`.github/workflows/dagster-cloud-deploy.yml`](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/blob/main/.github/workflows/dagster-cloud-deploy.yml) (for **Hybrid** deployments)
+- [`.github/workflows/branch_deployments.yml`](https://github.com/dagster-io/dagster-cloud-serverless-quickstart/blob/main/.github/workflows/branch_deployments.yml) (for **Serverless** deployments)
+
+In the next step, you'll modify these files to work with your Dagster+ setup.
+
+### Step 4.2: Add the agent registry to dagster_cloud.yaml
+
+:::note
+If you used the GitHub app to configure you're repository, this step isn't required. [Skip ahead to Step 4.5](#step-45-verify-github-action-runs)
+:::
+
+In the `dagster_cloud.yaml` file, replace `build.registry` with the registry used by the [agent you created in Step 2](#step-2-generate-a-dagster-agent-token).
+
+For example:
+
+
+
+### Step 4.3: Configure GitHub Action secrets
+
+:::note
+If you used the GitHub app to configure you're repository, this step isn't required. [Skip ahead to Step 4.5](#step-45-verify-github-action-runs)
+:::
+
+1. In your GitHub repository, click the **Settings** tab.
+2. In the **Security** section of the sidebar, click **Secrets > Actions**.
+3. Click **New repository secret**.
+4. In the **Name** field, enter the name of the secret. For example, `DAGSTER_CLOUD_API_TOKEN`
+5. In the **Value** field, paste the value of the secret.
+6. Click **Add secret**.
+
+Repeat steps 3-6 for each of the secrets required for the registry used by the agent you created in Step 2. See below for more details:
+
+
+
+
+
+- `DAGSTER_CLOUD_API_TOKEN` - The Dagster+ agent token you created in [Step 2](#step-2-generate-a-dagster-agent-token)
+- `DAGSTER_CLOUD_URL` - Your Dagster+ base URL (`https://my_org.dagster.cloud`)
+- `DOCKERHUB_USERNAME` - Your DockerHub username
+- `DOCKERHUB_TOKEN` - A DockerHub [access token](https://docs.docker.com/docker-hub/access-tokens/#create-an-access-token)
+
+
+
+
+
+- `DAGSTER_CLOUD_API_TOKEN` - The Dagster+ agent token you created in [Step 2](#step-2-generate-a-dagster-agent-token)
+- `DAGSTER_CLOUD_URL` - Your Dagster+ base URL (`https://my_org.dagster.cloud`)
+- `AWS_ACCESS_KEY` - The **Access key ID** of the AWS IAM user you created in [Step 3](#step-3-create-and-configure-an-agent)
+- `AWS_SECRET_ACCESS_KEY` - The **Secret access key** of the AWS IAM user you created in [Step 3](#step-3-create-and-configure-an-agent)
+- `AWS_REGION` - The AWS region where your ECR registry is located
+
+
+
+
+
+- `DAGSTER_CLOUD_API_TOKEN` - The Dagster+ agent token you created in [Step 2](#step-2-generate-a-dagster-agent-token)
+- `DAGSTER_CLOUD_URL` - Your Dagster+ base URL (`https://my_org.dagster.cloud`)
+- `GCR_JSON_KEY` - Your GCR JSON credentials
+
+
+
+
+
+### Step 4.4: Configure GitHub Action
+
+:::note
+If you used the GitHub app to configure you're repository, this step isn't required. [Skip ahead to Step 4.5](#step-45-verify-github-action-runs)
+:::
+
+In this step, you'll update the GitHub workflow files in your repository to set up Docker registry access.
+
+In the `.github/workflows/dagster-cloud-deploy.yml` file, un-comment the `step` associated with your registry. For example, for an Amazon ECR registry, you'd un-comment the following portion of the workflow file:
+
+```yaml
+jobs:
+ dagster-cloud-deploy:
+ steps:
+ - name: Configure AWS credentials
+ uses: aws-actions/configure-aws-credentials@v1
+ with:
+ aws-access-key-id: ${{ secrets.AWS_ACCESS_KEY_ID }}
+ aws-secret-access-key: ${{ secrets.AWS_SECRET_ACCESS_KEY }}
+ aws-region: ${{ secrets.AWS_REGION }}
+```
+
+Save and commit the file to your repository.
+
+### Step 4.5: Verify GitHub action runs
+
+The last step is to verify that the GitHub Action runs successfully.
+
+1. In the repository, click the **Actions** tab.
+2. In the list of workflows, locate the latest branch deployment run. For example:
+
+![Show this in the UI](/images/dagster-plus/features/branch-deployments/github-verify-run.png)
+
+
+
+
+### Step 4.1: add GitLab CI/CD script to your project
+
+:::note
+If you used the GitLab app to configure you're repository, this step isn't required. [Skip ahead to Step 4.5](#step-45-verify-gitlab-pipeline-runs)
+:::
+
+Copy the following files to your project, and **replace** all references to `quickstart-etl` with the name of your project:
+
+- [`dagster_cloud.yaml`](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/blob/main/dagster_cloud.yaml)
+- [`.gitlab-ci.yml`](https://github.com/dagster-io/dagster-cloud-action/blob/main/gitlab/hybrid-ci.yml) (for **Hybrid** deployments)
+- [`.gitlab-ci.yml`](https://github.com/dagster-io/dagster-cloud-action/blob/main/gitlab/serverless-ci.yml) (for **Serverless** deployments)
+
+In the next step, you'll modify these files to work with your Dagster+ setup.
+
+### Step 4.2: add the agent registry to dagster_cloud.yaml
+
+:::note
+If you used the GitLab app to configure you're repository, this step isn't required. [Skip ahead to Step 4.5](#step-45-verify-gitlab-pipeline-runs)
+:::
+
+
+In the `dagster_cloud.yaml` file, replace `build.registry` with the registry used by the [agent you created in Step 2](#step-2-generate-a-dagster-agent-token).
+
+For example:
+
+
+
+### Step 4.3: configure GitLab CI/CD variables
+
+:::note
+If you used the GitLab app to configure you're repository, this step isn't required. [Skip ahead to Step 4.5](#step-45-verify-gitlab-pipeline-runs)
+:::
+
+
+1. In your project, click the **Settings** tab.
+2. In the **CI/CD** section of the sidebar, expand **Variables**.
+3. Click **Add variable**.
+4. In the **Key** field, enter the name of the variable. For example, `DAGSTER_CLOUD_API_TOKEN`
+5. In the **Value** field, paste the value of the variable.
+6. Click **Add variable**.
+
+Repeat steps 3-6 for each of the secrets required for your registry type:
+
+
+
+
+
+- `DAGSTER_CLOUD_API_TOKEN` - The Dagster+ agent token you created in [Step 2](#step-2-generate-a-dagster-agent-token)
+- `DAGSTER_CLOUD_URL` - Your Dagster+ base URL (`https://my_org.dagster.cloud`)
+
+
+
+
+
+- `DAGSTER_CLOUD_API_TOKEN` - The Dagster+ agent token you created in [Step 2](#step-2-generate-a-dagster-agent-token)
+- `DAGSTER_CLOUD_URL` - Your Dagster+ base URL (`https://my_org.dagster.cloud`)
+- `DOCKERHUB_USERNAME` - Your DockerHub username
+- `DOCKERHUB_TOKEN` - A DockerHub [access token](https://docs.docker.com/docker-hub/access-tokens/#create-an-access-token)
+
+
+
+
+
+- `DAGSTER_CLOUD_API_TOKEN` - The Dagster+ agent token you created in [Step 2](#step-2-generate-a-dagster-agent-token)
+- `DAGSTER_CLOUD_URL` - Your Dagster+ base URL (`https://my_org.dagster.cloud`)
+- `AWS_ACCESS_KEY` - The **Access key ID** of the AWS IAM user you created in [Step 3](#step-3-create-and-configure-an-agent)
+- `AWS_SECRET_ACCESS_KEY` - The **Secret access key** of the AWS IAM user you created in [Step 3](#step-3-create-and-configure-an-agent)
+- `AWS_REGION` - The AWS region where your ECR registry is located
+
+
+
+
+
+- `DAGSTER_CLOUD_API_TOKEN` - The Dagster+ agent token you created in [Step 2](#step-2-generate-a-dagster-agent-token)
+- `DAGSTER_CLOUD_URL` - Your Dagster+ base URL (`https://my_org.dagster.cloud`)
+- `GCR_JSON_KEY` - Your GCR JSON credentials
+
+
+
+
+
+### Step 4.4: configure GitLab CI/CD script
+
+:::note
+If you used the GitLab app to configure you're repository, this step isn't required. [Skip ahead to Step 4.5](#step-45-verify-gitlab-pipeline-runs)
+:::
+
+In this step, you'll update the GitLab CI/CD config to set up Docker registry access.
+
+In the `.gitlab-ci.yml` file, un-comment the `step` associated with your registry. For example, for the GitLab container registry, you'd un-comment the following portion of the `.gitlab-ci.yml` file:
+
+```yaml
+build-image:
+ ...
+ before_script:
+ # For GitLab Container Registry
+ - echo $CI_JOB_TOKEN | docker login --username $CI_REGISTRY_USER --password-stdin $REGISTRY_URL
+```
+
+Save and commit the files to the project.
+
+### Step 4.5: verify GitLab pipeline runs
+
+The last step is to verify that the GitLab pipeline runs successfully.
+
+1. On the project page, click the **CI/CD** tab.
+2. In the list of pipelines, locate the latest branch deployment run. For example:
+
+![Show this in the UI](/images/dagster-plus/features/branch-deployments/gitlab-verify-run.png)
+
+
+
+
+Whenever the state of your branch is updated, Dagster+ expects the following steps to occur:
+
+1. A new image containing your code and requirements is built on the branch. Refer to [Managing code locations](/dagster-plus/deployment/code-locations/) to learn more.
+
+2. The new image is pushed to a Docker registry accessible to your agent.
+
+The details of how this is accomplished depend on your specific CI/CD solution.
+
+:::note
+
+The following examples assume the registry URL and image tag are stored in the `LOCATION_REGISTRY_URL` and `IMAGE_TAG` environment variables.
+
+:::
+
+### Step 4.1 Create a branch deployment associated with the branch
+
+Execute the following command within your CI/CD process:
+
+ ```shell
+ BRANCH_DEPLOYMENT_NAME=$(
+ dagster-cloud branch-deployment create-or-update \
+ --organization $ORGANIZATION_NAME \
+ --api-token $DAGSTER_CLOUD_API_TOKEN \ # Agent token from Step 1
+ --git-repo-name $REPOSITORY_NAME \ # Git repository name
+ --branch-name $BRANCH_NAME \ # Git branch name
+ --commit-hash $COMMIT_SHA \ # Latest commit SHA on the branch
+ --timestamp $TIMESTAMP # UTC unixtime timestamp of the latest commit
+ )
+ ```
+
+One or more additional parameters can optionally be supplied to the `create-or-update` command to enhance the Branch Deployments UI in Dagster+:
+
+```shell
+BRANCH_DEPLOYMENT_NAME=$(
+ dagster-cloud branch-deployment create-or-update \
+ --organization $ORGANIZATION_NAME \
+ --api-token $DAGSTER_CLOUD_API_TOKEN \
+ --git-repo-name $REPOSITORY_NAME \
+ --branch-name $BRANCH_NAME \
+ --commit-hash $COMMIT_SHA \
+ --timestamp $TIMESTAMP
+ --code-review-url $PR_URL \ # URL to review the given changes, e.g.
+ # Pull Request or Merge Request
+ --code-review-id $INPUT_PR \ # Alphanumeric ID for the given set of changes
+ --pull-request-status $PR_STATUS \ # A status, one of `OPEN`, `CLOSED`,
+ # or `MERGED`, that describes the set of changes
+ --commit-message $MESSAGE \ # The message associated with the latest commit
+ --author-name $NAME \ # A display name for the latest commit's author
+ --author-email $EMAIL \ # An email for the latest commit's author
+ --author-avatar-url $AVATAR_URL # An avatar URL for the latest commit's author
+ --base-deployment-name $BASE_DEPLOYMENT_NAME # The main deployment that will be compared against. Default is 'prod'
+)
+```
+
+If the command is being executed from the context of the git repository, you can alternatively pull this metadata from the repository itself:
+
+```shell
+BRANCH_DEPLOYMENT_NAME=$(
+ dagster-cloud branch-deployment create-or-update \
+ --organization $ORGANIZATION_NAME \
+ --api-token $DAGSTER_CLOUD_API_TOKEN \
+ --git-repo-name $REPOSITORY_NAME \
+ --branch-name $BRANCH_NAME \
+ --read-git-state # Equivalent to passing --commit-hash, --timestamp
+ # --commit-message, --author-name, --author-email
+)
+```
+
+### Step 4.2 Deploy your code to the branch deployment
+
+Execute the following command within your CI/CD process:
+
+```shell
+dagster-cloud deployment add-location \
+ --organization $ORGANIZATION_NAME \
+ --deployment $BRANCH_DEPLOYMENT_NAME \
+ --api-token $DAGSTER_CLOUD_API_TOKEN \
+ --location-file $LOCATION_FILE \
+ --location-name $LOCATION_NAME \
+ --image "${LOCATION_REGISTRY_URL}:${IMAGE_TAG}" \
+ --commit-hash "${COMMIT_SHA}" \
+ --git-url "${GIT_URL}"
+```
+
+
+
+
+## Next steps
+
+- Learn more about [Branch Deployments](/dagster-plus/features/ci-cd/branch-deployments/index.md)
+- Learn how to [Track changes on a Branch Deployment](/dagster-plus/features/ci-cd/branch-deployments/change-tracking)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/testing.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/testing.md
new file mode 100644
index 0000000000000..2f243201e3a83
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/testing.md
@@ -0,0 +1,476 @@
+---
+title: "Testing against production with branch deployments"
+sidebar_position: 400
+---
+
+:::note
+This guide is applicable to Dagster+.
+:::
+
+This guide details a workflow to test Dagster code in your cloud environment without impacting your production data. To highlight this functionality, we'll leverage Dagster+ branch deployments and a Snowflake database to:
+
+- Execute code on a feature branch directly on Dagster+
+- Read and write to a unique per-branch clone of our Snowflake data
+
+With these tools, we can merge changes with confidence in the impact on our data platform and with the assurance that our code will execute as intended.
+
+Here's an overview of the main concepts we'll be using:
+
+- [Assets](/guides/build/assets/) - We'll define three assets that each persist a table to Snowflake.
+- [Ops](/guides/build/ops) - We'll define two ops that query Snowflake: the first will clone a database, and the second will drop database clones.
+- [Graphs](/guides/build/graphs) - We'll build graphs that define the order our ops should run.
+- [Jobs](/guides/build/assets/asset-jobs) - We'll define jobs by binding our graphs to resources.
+- [Resources](/guides/build/external-resources/) - We'll use the to swap in different Snowflake connections to our jobs depending on environment.
+- [I/O managers](/guides/build/io-managers/) - We'll use a Snowflake I/O manager to persist asset outputs to Snowflake.
+
+## Prerequisites
+
+:::note
+
+This guide is an extension of the [Transitioning data pipelines from development to production](/guides/deploy/dev-to-prod) guide, illustrating a workflow for staging deployments. We'll use the examples from this guide to build a workflow atop Dagster+'s branch deployment feature.
+
+:::
+
+To complete the steps in this guide, you'll need:
+
+- A Dagster+ account
+- An existing Branch Deployments setup that uses [GitHub actions](/dagster-plus/features/ci-cd/configuring-ci-cd#github) or [Gitlab CI/CD](/dagster-plus/features/ci-cd/configuring-ci-cd#non-github). Your setup should contain a Dagster project set up for branch deployments containing:
+ - Either a GitHub actions workflow file (e.g. `.github/workflows/branch-deployments.yaml`) or a Gitlab CI/CD file (e.g. `.gitlab-ci.yml`)
+ - Dockerfile that installs your Dagster project
+- User permissions in Dagster+ that allow you to [access Branch Deployments](/dagster-plus/features/authentication-and-access-control/rbac/user-roles-permissions)
+
+## Overview
+
+We have a `PRODUCTION` Snowflake database with a schema named `HACKER_NEWS`. In our production cloud environment, we'd like to write tables to Snowflake containing subsets of Hacker News data. These tables will be:
+
+- `ITEMS` - A table containing the entire dataset
+- `COMMENTS` - A table containing data about comments
+- `STORIES` - A table containing data about stories
+
+To set up a branch deployment workflow to construct and test these tables, we will:
+
+1. Define these tables as [assets](/guides/build/assets/).
+2. Configure our assets to write to Snowflake using a different connection (credentials and database name) for two environments: production and branch deployment.
+3. Write a job that will clone the production database upon each branch deployment launch. Each clone will be named `PRODUCTION_CLONE_`, where `` is the pull request ID of the branch. Then we'll create a branch deployment and test our Hacker News assets against our newly cloned database.
+4. Write a job that will delete the corresponding database clone upon closing the feature branch.
+
+## Step 1: Create our assets
+
+In production, we want to write three tables to Snowflake: `ITEMS`, `COMMENTS`, and `STORIES`. We can define these tables as assets as follows:
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/assets.py startafter=start_assets endbefore=end_assets
+# assets.py
+import pandas as pd
+import requests
+
+from dagster import Config, asset
+
+
+class ItemsConfig(Config):
+ base_item_id: int
+
+
+@asset(
+ io_manager_key="snowflake_io_manager",
+)
+def items(config: ItemsConfig) -> pd.DataFrame:
+ """Items from the Hacker News API: each is a story or a comment on a story."""
+ rows = []
+ max_id = requests.get(
+ "https://hacker-news.firebaseio.com/v0/maxitem.json", timeout=5
+ ).json()
+ # Hacker News API is 1-indexed, so adjust range by 1
+ for item_id in range(max_id - config.base_item_id + 1, max_id + 1):
+ item_url = f"https://hacker-news.firebaseio.com/v0/item/{item_id}.json"
+ rows.append(requests.get(item_url, timeout=5).json())
+
+ # ITEM_FIELD_NAMES is a list of the column names in the Hacker News dataset
+ result = pd.DataFrame(rows, columns=ITEM_FIELD_NAMES).drop_duplicates(subset=["id"])
+ result.rename(columns={"by": "user_id"}, inplace=True)
+ return result
+
+
+@asset(
+ io_manager_key="snowflake_io_manager",
+)
+def comments(items: pd.DataFrame) -> pd.DataFrame:
+ """Comments from the Hacker News API."""
+ return items[items["type"] == "comment"]
+
+
+@asset(
+ io_manager_key="snowflake_io_manager",
+)
+def stories(items: pd.DataFrame) -> pd.DataFrame:
+ """Stories from the Hacker News API."""
+ return items[items["type"] == "story"]
+```
+
+As you can see, our assets use an [I/O manager](/guides/build/io-managers/) named `snowflake_io_manager`. Using I/O managers and other resources allow us to swap out implementations per environment without modifying our business logic.
+
+## Step 2: Configure our assets for each environment
+
+At runtime, we'd like to determine which environment our code is running in: branch deployment, or production. This information dictates how our code should execute, specifically with which credentials and with which database.
+
+To ensure we can't accidentally write to production from within our branch deployment, we'll use a different set of credentials from production and write to our database clone.
+
+Dagster automatically sets certain [environment variables](/dagster-plus/deployment/management/environment-variables/built-in) containing deployment metadata, allowing us to read these environment variables to discern between deployments. We can access the `DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT` environment variable to determine the currently executing environment.
+
+Because we want to configure our assets to write to Snowflake using a different set of credentials and database in each environment, we'll configure a separate I/O manager for each environment:
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/branch_deployments/repository_v1.py startafter=start_repository endbefore=end_repository
+# definitions.py
+from dagster import Definitions
+
+from ..assets import comments, items, stories
+
+snowflake_config = {
+ "account": "abc1234.us-east-1",
+ "user": "system@company.com",
+ "password": {"env": "SYSTEM_SNOWFLAKE_PASSWORD"},
+ "schema": "HACKER_NEWS",
+}
+
+resources = {
+ "branch": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ **snowflake_config,
+ database=f"PRODUCTION_CLONE_{os.getenv('DAGSTER_CLOUD_PULL_REQUEST_ID')}",
+ ),
+ },
+ "prod": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ **snowflake_config,
+ database="PRODUCTION",
+ ),
+ },
+}
+
+
+def get_current_env():
+ is_branch_depl = os.getenv("DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT") == "1"
+ assert is_branch_depl is not None # env var must be set
+ return "branch" if is_branch_depl else "prod"
+
+
+defs = Definitions(
+ assets=[items, comments, stories], resources=resources[get_current_env()]
+)
+```
+
+Refer to the [Dagster+ environment variables documentation](/dagster-plus/deployment/management/environment-variables/) for more info about available environment variables.
+
+## Step 3: Create jobs to manage database cloning per branch deployment
+
+We'll first need to define a job that clones our `PRODUCTION` database for each branch deployment. Later, in our GitHub actions workflow, we can trigger this job to run upon each redeploy. Each clone will be named `PRODUCTION_CLONE_` with `` representing the pull request ID, ensuring each branch deployment has a unique clone. This job will drop a database clone if it exists and then reclone from production, ensuring each redeployment has a fresh clone of `PRODUCTION`:
+
+:::note
+
+ Why use ops and jobs instead of assets? We'll be writing
+ops to clone the production database for each branch deployment and drop the
+clone once the branch is merged. In this case, we chose to use ops since we
+are primarily interested in the task that's being performed: cloning or
+dropping the database. Additionally, we don't need asset-specific features for
+these tasks, like viewing them in the Global Asset Graph.
+
+:::
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/branch_deployments/clone_and_drop_db.py startafter=start_clone_db endbefore=end_clone_db
+from dagster_snowflake import SnowflakeResource
+
+from dagster import In, Nothing, graph, op
+
+
+@op
+def drop_database_clone(snowflake: SnowflakeResource):
+ with snowflake.get_connection() as conn:
+ cur = conn.cursor()
+ cur.execute(
+ "DROP DATABASE IF EXISTS"
+ f" PRODUCTION_CLONE_{os.environ['DAGSTER_CLOUD_PULL_REQUEST_ID']}"
+ )
+
+
+@op(ins={"start": In(Nothing)})
+def clone_production_database(snowflake: SnowflakeResource):
+ with snowflake.get_connection() as conn:
+ cur = conn.cursor()
+ cur.execute(
+ "CREATE DATABASE"
+ f" PRODUCTION_CLONE_{os.environ['DAGSTER_CLOUD_PULL_REQUEST_ID']} CLONE"
+ ' "PRODUCTION"'
+ )
+
+
+@graph
+def clone_prod():
+ clone_production_database(start=drop_database_clone())
+
+
+@graph
+def drop_prod_clone():
+ drop_database_clone()
+```
+
+We've defined `drop_database_clone` and `clone_production_database` to utilize the . The Snowflake resource will use the same configuration as the Snowflake I/O manager to generate a connection to Snowflake. However, while our I/O manager writes outputs to Snowflake, the Snowflake resource executes queries against Snowflake.
+
+We now need to define resources that configure our jobs to the current environment. We can modify the resource mapping by environment as follows:
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/branch_deployments/repository_v2.py startafter=start_resources endbefore=end_resources
+resources = {
+ "branch": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ **snowflake_config,
+ database=f"PRODUCTION_CLONE_{os.getenv('DAGSTER_CLOUD_PULL_REQUEST_ID')}",
+ ),
+ "snowflake": SnowflakeResource(
+ **snowflake_config,
+ database=f"PRODUCTION_CLONE_{os.getenv('DAGSTER_CLOUD_PULL_REQUEST_ID')}",
+ ),
+ },
+ "prod": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ **snowflake_config,
+ database="PRODUCTION",
+ ),
+ "snowflake": SnowflakeResource(**snowflake_config, database="PRODUCTION"),
+ },
+}
+```
+
+Then, we can add the `clone_prod` and `drop_prod_clone` jobs that now use the appropriate resource to the environment and add them to our definitions:
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/branch_deployments/repository_v2.py startafter=start_repository endbefore=end_repository
+branch_deployment_jobs = [
+ clone_prod.to_job(),
+ drop_prod_clone.to_job(),
+]
+defs = Definitions(
+ assets=[items, comments, stories],
+ resources=resources[get_current_env()],
+ jobs=(
+ branch_deployment_jobs
+ if os.getenv("DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT") == "1"
+ else []
+ ),
+)
+```
+
+## Step 4: Create our database clone upon opening a branch
+
+
+
+
+The `branch_deployments.yml` file located in `.github/workflows/branch_deployments.yml` defines a `dagster_cloud_build_push` job with a series of steps that launch a branch deployment. Because we want to queue a run of `clone_prod` within each deployment after it launches, we'll add an additional step at the end `dagster_cloud_build_push`. This job is triggered on multiple pull request events: `opened`, `synchronize`, `reopen`, and `closed`. This means that upon future pushes to the branch, we'll trigger a run of `clone_prod`. The `if` condition below ensures that `clone_prod` will not run if the pull request is closed:
+
+{/* TODO convert to */}
+```yaml file=/guides/dagster/development_to_production/branch_deployments/clone_prod.yaml
+# .github/workflows/branch_deployments.yml
+
+name: Dagster Branch Deployments
+ on:
+ pull_request:
+ types: [opened, synchronize, reopened, closed]
+ env:
+ DAGSTER_CLOUD_URL: ${{ secrets.DAGSTER_CLOUD_URL }}
+
+ jobs:
+ dagster_cloud_build_push:
+ runs-on: ubuntu-latest
+ name: Dagster Branch Deployments
+ strategy:
+ ...
+ steps:
+ # Existing steps here
+ ...
+ - name: Clone Snowflake schema upon launch
+ if: github.event.action != 'closed'
+ uses: dagster-io/dagster-cloud-action/actions/utils/run@v0.1
+ with:
+ location_name: ${{ matrix.location.name }}
+ deployment: ${{ steps.deploy.outputs.deployment }}
+ job_name: clone_prod
+ env:
+ DAGSTER_CLOUD_URL: ${{ secrets.DAGSTER_CLOUD_URL }}
+ DAGSTER_CLOUD_API_TOKEN: ${{ secrets.DAGSTER_CLOUD_API_TOKEN }}
+```
+
+Opening a pull request for our current branch will automatically kick off a branch deployment. After the deployment launches, we can confirm that the `clone_prod` job has run:
+
+![Instance overview](/images/guides/development_to_production/branch_deployments/instance_overview.png)
+
+Alternatively, the logs for the branch deployment workflow can be found in the **Actions** tab on the GitHub pull request.
+
+We can also view our database in Snowflake to confirm that a clone exists for each branch deployment. When we materialize our assets within our branch deployment, we'll now be writing to our clone of `PRODUCTION`. Within Snowflake, we can run queries against this clone to confirm the validity of our data:
+
+![Instance overview](/images/guides/development_to_production/branch_deployments/snowflake.png)
+
+
+
+
+The `.gitlab-ci.yaml` script contains a `deploy` job that defines a series of steps that launch a branch deployment. Because we want to queue a run of `clone_prod` within each deployment after it launches, we'll add an additional step at the end of `deploy`. This job is triggered on when a merge request is created or updated. This means that upon future pushes to the branch, we'll trigger a run of `clone_prod`.
+
+```yaml file=/guides/dagster/development_to_production/branch_deployments/clone_prod.gitlab-ci.yml
+# .gitlab-ci.yml
+
+stages:
+ - setup
+ - build
+ - deploy
+
+workflow:
+ rules:
+ - if: $CI_COMMIT_BRANCH == $CI_DEFAULT_BRANCH
+ - if: $CI_PIPELINE_SOURCE == 'merge_request_event'
+
+parse-workspace:
+ ...
+
+build-image:
+ ...
+
+deploy-docker:
+ ...
+
+deploy-docker-branch:
+ stage: deploy
+ rules:
+ - if: $CI_PIPELINE_SOURCE == 'merge_request_event'
+ dependencies:
+ - build-image
+ - parse-workspace
+ image: ghcr.io/dagster-io/dagster-cloud-action:0.1.23
+ script:
+ # Existing steps here
+ ...
+
+ # Add a step to launch the job cloning the prod db
+ - dagster-plus job launch
+ --url "$DAGSTER_CLOUD_URL/$DEPLOYMENT_NAME"
+ --api-token "$DAGSTER_CLOUD_API_TOKEN"
+ --location "location_name_containing_clone_prod_job"
+ --job clone_prod
+ environment:
+ name: branch/$CI_COMMIT_REF_NAME
+ on_stop: close_branch
+
+close_branch:
+ ...
+```
+
+Opening a merge request for our current branch will automatically kick off a branch deployment. After the deployment launches, we can confirm that the `clone_prod` job has run:
+
+![Instance overview](/images/guides/development_to_production/branch_deployments/instance_overview.png)
+
+We can also view our database in Snowflake to confirm that a clone exists for each branch deployment. When we materialize our assets within our branch deployment, we'll now be writing to our clone of `PRODUCTION`. Within Snowflake, we can run queries against this clone to confirm the validity of our data:
+
+![Instance overview](/images/guides/development_to_production/branch_deployments/snowflake.png)
+
+
+
+
+
+## Step 5: Delete our database clone upon closing a branch
+
+
+
+
+Finally, we can add a step to our `branch_deployments.yml` file that queues a run of our `drop_prod_clone` job:
+
+{/* TODO convert to */}
+```yaml file=/guides/dagster/development_to_production/branch_deployments/drop_db_clone.yaml
+# .github/workflows/branch_deployments.yml
+
+name: Dagster Branch Deployments
+ on:
+ pull_request:
+ types: [opened, synchronize, reopened, closed]
+ env:
+ DAGSTER_CLOUD_URL: ${{ secrets.DAGSTER_CLOUD_URL }}
+
+ jobs:
+ dagster_cloud_build_push:
+ runs-on: ubuntu-latest
+ name: Dagster Branch Deployments
+ strategy:
+ ...
+ steps:
+ # Existing steps here
+ ...
+ - name: Clone Snowflake schema upon launch
+ ...
+ - name: Delete schema clone upon PR close
+ if: github.event.action == 'closed'
+ uses: dagster-io/dagster-cloud-action/actions/utils/run@v0.1
+ with:
+ location_name: ${{ matrix.location.name }}
+ deployment: ${{ steps.deploy.outputs.deployment }}
+ job_name: drop_prod_clone
+ env:
+ DAGSTER_CLOUD_URL: ${{ secrets.DAGSTER_CLOUD_URL }}
+ DAGSTER_CLOUD_API_TOKEN: ${{ secrets.DAGSTER_CLOUD_API_TOKEN }}
+```
+
+
+
+
+Finally, we can add a step to our `.gitlab-ci.yml` file that queues a run of our `drop_prod_clone` job:
+
+{/* TODO convert to */}
+```yaml file=/guides/dagster/development_to_production/branch_deployments/drop_db_clone.gitlab-ci.yml
+# .gitlab-ci.yml
+
+stages:
+ - setup
+ - build
+ - deploy
+
+workflow:
+ rules:
+ - if: $CI_COMMIT_BRANCH == $CI_DEFAULT_BRANCH
+ - if: $CI_PIPELINE_SOURCE == 'merge_request_event'
+
+parse-workspace:
+ ...
+
+build-image:
+ ...
+
+deploy-docker:
+ ...
+
+deploy-docker-branch:
+ ...
+
+close_branch:
+ stage: deploy
+ image: ghcr.io/dagster-io/dagster-cloud-action:0.1.23
+ when: manual
+ only:
+ - merge_requests
+ script:
+ # Existing steps here
+ ...
+
+ # Add a step to launch the job dropping the cloned db
+ - dagster-plus job launch
+ --url "$DAGSTER_CLOUD_URL/$DEPLOYMENT_NAME"
+ --api-token "$DAGSTER_CLOUD_API_TOKEN"
+ --location "location_name_containing_drop_prod_clone_job"
+ --job drop_prod_clone
+ environment:
+ name: branch/$CI_COMMIT_REF_NAME
+ action: stop
+```
+
+
+
+
+After merging our branch, viewing our Snowflake database will confirm that our branch deployment step has successfully deleted our database clone.
+
+We've now built an elegant workflow that enables future branch deployments to automatically have access to their own clones of our production database that are cleaned up upon merge!
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/using-branch-deployments-with-the-cli.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/using-branch-deployments-with-the-cli.md
new file mode 100644
index 0000000000000..cf4ff44f2e124
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/branch-deployments/using-branch-deployments-with-the-cli.md
@@ -0,0 +1,213 @@
+---
+title: "Using branch deployments with the dagster-cloud CLI"
+sidebar_position: 300
+---
+
+:::note
+
+This guide is applicable to Dagster+.
+
+:::
+
+In this article, we'll walk you through setting up branch deployments with a general continuous integration (CI) or git platform, using the [`dagster-cloud` CLI](/dagster-plus/deployment/management/dagster-cloud-cli/).
+
+Using this approach to branch deployments may be a good fit if:
+
+- You don't use GitHub for version control
+- You use an alternate CI platform
+- You want full control over Branch Deployment configuration
+
+**If you use GitHub for version control or want Dagster to automate branch deployments**, see "[Setting up branch deployments](setting-up-branch-deployments)" and choose the GitHub tab.
+
+## Prerequisites
+
+Using Branch Deployments requires setting up two components: the Branch Deployment agent and CI platform. You'll need:
+
+- **Organization Admin** permissions in Dagster+
+- To install the [`dagster-cloud` CLI](/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring)
+- The ability to configure your CI platform
+- The ability to run a new agent in your infrastructure. This isn't required if you're using [Serverless deployment](/dagster-plus/deployment/deployment-types/serverless/).
+
+## Step 1: Generate a Dagster+ agent token
+
+In this step, you'll generate a token for the Dagster+ agent. The Dagster+ agent will use this to authenticate to the agent API.
+
+1. Sign in to your Dagster+ instance.
+2. Click the **user menu (your icon) > Organization Settings**.
+3. In the **Organization Settings** page, click the **Tokens** tab.
+3. Click the **+ Create agent token** button.
+4. After the token has been created, click **Reveal token**.
+5. Copy the token somewhere safe, since you'll need it to complete the setup.
+
+## Step 2: Create and configure an agent
+
+:::note
+
+If you're using Serverless deployment, this step isn't required. Skip to [step 3](#step-3-create-a-branch-deployment-using-the-dagster-cloud-cli)
+
+:::
+
+While you can use your existing production agent, we recommend creating a dedicated branch deployment agent. This ensures that your production instance isn't negatively impacted by the workload associated with branch deployments.
+
+Using the tabs, select your agent type to view instructions.
+
+
+
+
+1. **Deploy an ECS agent to serve your branch deployments**. Follow the [ECS agent](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/) setup guide, making sure to set the **Enable Branch Deployments** parameter if using the CloudFormation template. If you are running an existing agent, follow the [upgrade guide](/dagster-plus/deployment/deployment-types/hybrid/amazon-ecs/upgrading-cloudformation) to ensure your template is up-to-date. Then, turn on the **Enable Branch Deployments** parameter.
+
+2. **Create a private [Amazon Elastic Registry (ECR) repository](https://console.aws.amazon.com/ecr/repositories).** Refer to the [AWS ECR documentation](https://docs.aws.amazon.com/AmazonECR/latest/userguide/repository-create.html) for instructions.
+
+ After the repository has been created, navigate back to the list of [ECR repositories](https://console.aws.amazon.com/ecr/repositories).
+
+ In the list, locate the repository and its **URI**:
+
+ ![Highlighted repository URI in the AWS ECR console](/images/dagster-plus/features/branch-deployments/aws-ecr-uri.png)
+
+3. [**Create an IAM user.**](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_users_create.html) This user must:
+
+ - Have push access to the ECR repository, and
+ - Have programmatic access to AWS using an [access key](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_credentials_access-keys.html)
+
+ After the user is created, save the **Access key ID** and **Secret access key** values shown on the confirmation page:
+
+ ![Highlighted repository URI in the AWS ECR console](/images/dagster-plus/features/branch-deployments/aws-iam-user-keys.png)
+
+
+
+
+1. Set up a new Docker agent. Refer to the [Docker agent setup guide](/dagster-plus/deployment/deployment-types/hybrid/docker/) for instructions.
+2. After the agent is set up, modify the `dagster.yaml` file as follows:
+
+ - Set the `dagster_cloud_api.branch_deployments` field to `true`
+ - Remove any `deployment` field(s)
+
+ For example:
+
+ ```yaml
+ # dagster.yaml
+
+ instance_class:
+ module: dagster_cloud.instance
+ class: DagsterCloudAgentInstance
+
+ dagster_cloud_api:
+ agent_token:
+ branch_deployments: true ## true enables branch deployments
+
+ user_code_launcher:
+ module: dagster_cloud.workspace.docker
+ class: DockerUserCodeLauncher
+ config:
+ networks:
+ - dagster_cloud_agent
+ server_ttl:
+ enabled: true
+ ttl_seconds: 7200 #2 hours
+ ```
+
+
+
+
+1. Set up a new Kubernetes agent. Refer to the [Kubernetes agent setup guide](/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration) for instructions.
+
+2. After the agent is set up, modify your Helm values file to include the following:
+
+ ```yaml
+ dagsterCloud:
+ branchDeployments: true
+ ---
+ workspace:
+ serverTTL:
+ enabled: true
+ ttlSeconds: 7200
+ ```
+
+
+
+
+## Step 3: Create a branch deployment using the dagster-cloud CLI
+
+Next, you'll create a branch deployment using the `dagster-cloud` CLI. When the state of a branch or merge request is updated, Dagster+ first expects these steps to occur:
+
+1. A new image containing your code and requirements is built on the branch. Refer to the [Dagster code requirements guide](/dagster-plus/deployment/code-locations/).
+
+2. The new image is pushed to a Docker registry accessible to your agent. **Note**: The following examples assume the registry URL and image tag are stored in the `LOCATION_REGISTRY_URL` and `IMAGE_TAG` environment variables, respectively.
+
+After the above has occurred:
+
+1. Use the `dagster-cloud` CLI to create a branch deployment associated with the branch, as follows:
+
+ ```shell
+ BRANCH_DEPLOYMENT_NAME=$(
+ dagster-cloud branch-deployment create-or-update \
+ --organization $ORGANIZATION_NAME \
+ --api-token $DAGSTER_CLOUD_API_TOKEN \ # Agent token from Step 1
+ --git-repo-name $REPOSITORY_NAME \ # Git repository name
+ --branch-name $BRANCH_NAME \ # Git branch name
+ --commit-hash $COMMIT_SHA \ # Latest commit SHA on the branch
+ --timestamp $TIMESTAMP # UTC unixtime timestamp of the latest commit
+ )
+ ```
+
+ One or more additional parameters can optionally be supplied to the `create-or-update` command to enhance the Branch Deployments UI in Dagster+:
+
+ ```shell
+ BRANCH_DEPLOYMENT_NAME=$(
+ dagster-cloud branch-deployment create-or-update \
+ --organization $ORGANIZATION_NAME \
+ --api-token $DAGSTER_CLOUD_API_TOKEN \
+ --git-repo-name $REPOSITORY_NAME \
+ --branch-name $BRANCH_NAME \
+ --commit-hash $COMMIT_SHA \
+ --timestamp $TIMESTAMP
+ --code-review-url $PR_URL \ # URL to review the given changes, e.g.
+ # Pull Request or Merge Request
+ --code-review-id $INPUT_PR \ # Alphanumeric ID for the given set of changes
+ --pull-request-status $PR_STATUS \ # A status, one of `OPEN`, `CLOSED`,
+ # or `MERGED`, that describes the set of changes
+ --commit-message $MESSAGE \ # The message associated with the latest commit
+ --author-name $NAME \ # A display name for the latest commit's author
+ --author-email $EMAIL \ # An email for the latest commit's author
+ --author-avatar-url $AVATAR_URL # An avatar URL for the latest commit's author
+ --base-deployment-name $BASE_DEPLOYMENT_NAME # The main deployment that will be compared against. Default is 'prod'
+ )
+ ```
+
+ If the command is being executed from the context of the git repository, you can alternatively pull this metadata from the repository itself:
+
+ ```shell
+ BRANCH_DEPLOYMENT_NAME=$(
+ dagster-cloud branch-deployment create-or-update \
+ --organization $ORGANIZATION_NAME \
+ --api-token $DAGSTER_CLOUD_API_TOKEN \
+ --git-repo-name $REPOSITORY_NAME \
+ --branch-name $BRANCH_NAME \
+ --read-git-state # Equivalent to passing --commit-hash, --timestamp
+ # --commit-message, --author-name, --author-email
+ )
+ ```
+
+2. Deploy the code to the branch deployment:
+
+ ```shell
+ dagster-cloud deployment add-location \
+ --organization $ORGANIZATION_NAME \
+ --deployment $BRANCH_DEPLOYMENT_NAME \
+ --api-token $DAGSTER_CLOUD_API_TOKEN \
+ --location-file $LOCATION_FILE \
+ --location-name $LOCATION_NAME \
+ --image "${LOCATION_REGISTRY_URL}:${IMAGE_TAG}" \
+ --commit-hash "${COMMIT_SHA}" \
+ --git-url "${GIT_URL}"
+ ```
+
+ Refer to the [Code location guide](/dagster-plus/deployment/code-locations) for more info on how a location's details are specified.
+
+## Step 4: Access the branch deployment
+
+Now that Branch Deployments are configured, you can check out the preview in Dagster+.
+
+Click the **deployment switcher** to view your workspace's branch deployments:
+
+![Highlighted branch deployments in the Dagster+'s deployment switcher](/images/dagster-plus/features/branch-deployments/dagster-ui-deployment-switcher.png)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/ci-cd-file-reference.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/ci-cd-file-reference.md
new file mode 100644
index 0000000000000..df933c4be9e31
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/ci-cd-file-reference.md
@@ -0,0 +1,26 @@
+---
+title: CI/CD file reference
+---
+
+:::note
+This reference is applicable to Dagster+.
+:::
+
+When you import a project into Dagster+ from GitHub or Gitlab, a few `.yml` files will be added to the repository. These files are essential as they manage the deployments in Dagster+.
+
+## branch_deployments.yml
+
+This file defines the steps required to use [branch deployments](/dagster-plus/features/ci-cd/branch-deployments/). It is required for branch deployments.
+
+:::note
+
+If you are using a [Hybrid deployment](/dagster-plus/deployment/deployment-types/hybrid/), you must manually add `branch_deployments.yml` to the repository.
+
+:::
+
+## deploy.yml
+
+This file is required for Dagster+. It defines the steps required to deploy a project in Dagster+, including running checks, checking out the project directory, and deploying the project. Additionally, note the following:
+
+* **If you are using a [Hybrid deployment](/dagster-plus/deployment/deployment-types/hybrid/)**, you must manually add `deploy.yml` to the repository.
+* **If you are using [dbt](/integrations/libraries/dbt)**, you may need to take additional steps to deploy your project. For more information, see "[Using dbt with Dagster+](/integrations/libraries/dbt/using-dbt-with-dagster-plus/).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/configuring-ci-cd.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/configuring-ci-cd.md
new file mode 100644
index 0000000000000..cc7b81ff44da9
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/configuring-ci-cd.md
@@ -0,0 +1,100 @@
+---
+title: "Configuring CI/CD for your project"
+---
+
+:::note
+
+This article only applies to [Dagster+ Hybrid deployments](/dagster-plus/deployment/deployment-types/hybrid/).
+
+:::
+
+You can configure CI/CD for your project using GitHub or a non-GitHub CI/CD provider.
+
+* If you use [GitHub](#github) as a CI/CD provider, you can use our GitHub Actions workflow to set up CI/CD for your project.
+* If you use a [non-GitHub CI/CD provider](#non-github), you can configure CI/CD using the `dagster-cloud CLI`.
+
+## GitHub
+
+To set up continuous integration using GitHub Actions, you can the Dagster+ Hybrid Quickstart template, which is a template with everything you need to get started using Hybrid deployment in Dagster+, or you can use your own code.
+
+* **If using the template:** Clone the [template repository](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart).
+* **If using your own code:** Copy the [GitHub workflow file](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/tree/main/.github/workflows) from the template repository and add it to your repository.
+
+### Configure the GitHub workflow YAML file
+
+The GitHub workflow deploys your code to Dagster+ using these steps:
+
+* **Initialize:** Your code is checked out and `dagster_cloud.yaml` file is validated.
+* **Docker image push:** A Docker image is built from your code and uploaded to your container registry.
+* **Deploy to Dagster+** The code locations in Dagster+ are updated to use the new Docker image.
+
+To configure the workflow:
+
+1. In the repository, set the `DAGSTER_CLOUD_API_TOKEN` GitHub action secret to the Dagster+ agent token. See "[Managing agent tokens in Dagster+](/dagster-plus/deployment/management/tokens/agent-tokens)". For more information on GitHub Action Secrets, see the [GitHub documentation](https://docs.github.com/en/actions/security-guides/encrypted-secrets#creating-encrypted-secrets-for-a-repository).
+2. In your [`dagster-cloud-deploy.yml`](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/blob/main/.github/workflows/dagster-cloud-deploy.yml), set the DAGSTER_CLOUD_ORGANIZATION environment variable to your Dagster+ organization name.
+3. In your `dagster-cloud-deploy.yml`, uncomment the step that is relevant to your Docker container registry. For example, if using DockerHub, uncomment the DockerHub step. Make sure you have set up the relevant secrets for building and uploading your Docker images.
+
+After you make the above changes and commit the workflow file, the CI process should be triggered to deploy your GitHub repository to Dagster+.
+
+During the deployment, the agent will attempt to load your code and update the metadata in Dagster+. When that has finished, you should see the GitHub Action complete successfully, and also be able to see the code location under the **Deployment** tag in Dagster+.
+
+## Non-GitHub CI/CD provider \{#non-github}
+
+If you are using a non-GitHub CI/CD provider, your system should use the `dagster-cloud ci` command to deploy code locations to Dagster+.
+
+1. Set the build environment variables. Note that all variables are required:
+ - `DAGSTER_CLOUD_ORGANIZATION`: The name of your organization in Dagster+.
+ - `DAGSTER_CLOUD_API_TOKEN`: A Dagster+ API token. **Note:** This is a sensitive value and should be stored as a CI/CD secret if possible.
+ - `DAGSTER_BUILD_STATEDIR`: A path to a blank or non-existent temporary directory on the build machine that will be used to store local state during the build.
+2. Run the configuration check:
+ ```
+ dagster-cloud ci check --project-dir=.
+ ```
+ This is an optional step but useful to validate the contents of your dagster_cloud.yaml and connection to Dagster+.
+3. Initialize the build session:
+ ```
+ dagster-cloud ci init --project-dir=.
+ ```
+ This reads the dagster_cloud.yaml configuration and initializes the DAGSTER_BUILD_STATEDIR.
+4. Build and upload Docker images for your code locations.
+
+ The Docker image should contain a Python environment with `dagster`, `dagster-cloud`, and your code. For reference, see the [example Dockerfile](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/blob/main/Dockerfile) in our template repository. The example uses `pip install .` to install the code including the dependencies specified in [`setup.py`](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/blob/main/setup.py).
+
+ It is a good idea to use a unique image tag for each Docker build. You can build one image per code location or a shared image for multiple code locations. As an example image tag, you can use the git commit SHA:
+
+ ```
+ export IMAGE_TAG=`git log --format=format:%H -n 1`
+ ```
+
+ Use this tag to build and upload your Docker image, for example:
+
+ ```
+ docker build . -t ghcr.io/org/dagster-cloud-image:$IMAGE_TAG
+ docker push ghcr.io/org/dagster-cloud-image:$IMAGE_TAG
+ ```
+
+ The upload step is specific to your Docker container registry and will require authentication. The only requirement is that the registry you upload to must match the registry specified in `dagster_cloud.yaml`.
+
+5. Update the build session with the Docker image tag. For each code location you want to deploy, run the following command passing the `IMAGE_TAG` used in the previous step:
+
+ ```
+ dagster-cloud ci set-build-output --location-name=code-location-a --image-tag=IMAGE_TAG
+ ```
+
+ This command does not deploy the code location but just updates the local state in `DAGSTER_BUILD_STATEDIR`.
+
+6. Deploy to Dagster+:
+
+ ```
+ dagster-cloud ci deploy
+ ```
+
+ This command updates the code locations in Dagster+. Once this finishes successfully, you should be able to see the code locations under the **Deployments** tab in Dagster+.
+
+:::note
+
+Creating branch deployments using the CLI requires some additional steps. For more information, see "[Using branch deployments with the dagster-cloud CLI](/dagster-plus/features/ci-cd/branch-deployments/using-branch-deployments-with-the-cli).
+:::
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/index.md
new file mode 100644
index 0000000000000..033e5a3f59389
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/ci-cd/index.md
@@ -0,0 +1,8 @@
+---
+title: "CI/CD (branch deployments)"
+sidebar_position: 50
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/asset-metadata.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/asset-metadata.md
new file mode 100644
index 0000000000000..096daf4c2436d
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/asset-metadata.md
@@ -0,0 +1,52 @@
+---
+title: "Integrate asset metadata into Dagster+ Insights"
+sidebar_label: "Integrate asset metadata"
+sidebar_position: 100
+---
+
+Out of the box, Dagster+ Insights gives you visibility into a variety of common metrics across your data platform.
+By creating custom metrics from asset metadata, you can use Insights to perform historical aggregation on any
+data your assets can emit.
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need a Dagster+ account on the Pro plan.
+
+
+
+## Step 1: Emit numeric metadata on your assets at runtime
+
+You'll need one or more assets that emit the same metadata key at run time. Insights metrics
+are most valuable when you have multiple assets that emit the same kind of metadata, such as
+such as the number of rows processed or the size of a file uploaded to object storage.
+
+Follow [the metadata guide](/guides/build/assets/metadata-and-tags/index.md#runtime-metadata) to add numeric metadata
+to your asset materializations.
+
+## Step 2: Enable viewing your metadata in Dagster+ Insights
+
+Once your assets are emitting numeric metadata values, you'll be able to enable viewing them in the Insights UI.
+
+To add your metadata key to the list of metrics shown in Insights, click **Edit** in the sidebar next to the **User provided metrics** header:
+
+![Viewing the Insights tab in the Dagster+ UI](/images/dagster-plus/features/insights/insights-settings.png)
+
+In the dialog that appears, use the eye indicator to show or hide metrics in Insights. Selected metrics will be visible in both the Insights sidebar and on individual asset pages.
+
+:::note
+
+It may take up to 24 hours for Insights to ingest new metadata. If this is a newly added metadata key and
+it isn't showing up in the list of metrics that can be displayed, try again in a few hours.
+
+:::
+
+## Step 3: Customize how your metric is displayed in Insights
+
+You can also change a metric's icon, display name, and description by clicking the **pencil icon** next to the metric
+in the **Edit** dialog.
+
+If the metric you're tracking is directly associated with a cost, you can input the cost per unit in the **Estimate costs** tab. Insights will
+use this to show an estimated cost alongside any aggregations of that metric.
+
+![Cost editor dialog](/images/dagster-plus/features/insights/customized-metrics.png)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/export-metrics.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/export-metrics.md
new file mode 100644
index 0000000000000..0dab34a996e12
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/export-metrics.md
@@ -0,0 +1,151 @@
+---
+title: "Export metrics from Dagster+ Insights"
+sidebar_label: "Export metrics"
+sidebar_position: 200
+---
+
+Using a GraphQL API endpoint, you can export [Dagster+ Insights](/dagster-plus/features/insights/) metrics from your Dagster+ instance.
+
+Refer to the [Built-in Insights metrics](/dagster-plus/features/insights/index.md#built-in-metrics) for a list of available metrics.
+
+## Prerequisites
+
+To complete the steps in this guide, you'll need:
+
+- A Dagster+ account
+- Access to the [Dagster+ Insights feature](/dagster-plus/features/insights)
+- A Dagster+ [user token](/dagster-plus/deployment/management/tokens/user-tokens)
+- Your deployment-scoped Dagster+ deployment URL. For example: `dagster-university.dagster.cloud/prod`
+
+## Before you start
+
+Before you start, note that:
+
+- Metrics are currently computed once per day
+- We don't recommend frequently querying over large time ranges that may download a large amount of data. After an initial data load, we recommend loading data daily for the most recent week or less.
+
+## Using the API
+
+In this example, we're using the [GraphQL Python Client](/guides/operate/graphql/graphql-client) to export the Dagster credits metric for all assets for September 2023:
+
+```python
+from datetime import datetime
+from dagster_graphql import DagsterGraphQLClient
+
+ASSET_METRICS_QUERY = """
+query AssetMetrics($metricName: String, $after: Float, $before: Float) {
+ reportingMetricsByAsset(
+ metricsSelector: {
+ metricName: $metricName
+ after: $after
+ before: $before
+ sortAggregationFunction: SUM
+ granularity: DAILY
+ }
+ ) {
+ __typename
+ ... on ReportingMetrics {
+ metrics {
+ values
+ entity {
+ ... on ReportingAsset {
+ assetKey {
+ path
+ }
+ }
+ }
+ }
+ }
+ }
+}
+
+"""
+
+
+def get_client():
+ url = "YOUR_ORG.dagster.cloud/prod" # Your deployment-scoped url
+ user_token = "YOUR_TOKEN" # A token generated from Organization Settings > Tokens
+ return DagsterGraphQLClient(url, headers={"Dagster-Cloud-Api-Token": user_token})
+
+
+if __name__ == "__main__":
+ client = get_client()
+ result = client._execute(
+ ASSET_METRICS_QUERY,
+ {
+ "metricName": "__dagster_dagster_credits",
+ "after": datetime(2023, 9, 1).timestamp(),
+ "before": datetime(2023, 10, 1).timestamp(),
+ },
+ )
+
+ for asset_series in result["reportingMetricsByAsset"]["metrics"]:
+ print("Asset key:", asset_series["entity"]["assetKey"]["path"])
+ print("Daily values:", asset_series["values"])
+
+```
+
+To use this example yourself, replace the values of `url` and `user_token` in this function:
+
+```python
+def get_client():
+ url = "YOUR_ORG.dagster.cloud/prod" # Your deployment-scoped url
+ user_token = "YOUR_TOKEN" # A token generated from Organization Settings > Tokens
+ return DagsterGraphQLClient(url, headers={"Dagster-Cloud-Api-Token": user_token})
+```
+
+Refer to the [Reference section](#reference) for more info about the endpoints available in the GraphQL API.
+
+## Reference
+
+For the full GraphQL API reference:
+
+1. Navigate to `YOUR_ORG.dagster.cloud/prod/graphql`, replacing `YOUR_ORG` with your organization name. For example: `https://dagster-university.dagster.cloud/prod/graphql`
+2. Click the **Schema** tab.
+
+### Available top-level queries
+
+```graphql
+reportingMetricsByJob(
+ metricsFilter: JobReportingMetricsFilter
+ metricsSelector: ReportingMetricsSelector!
+): ReportingMetricsOrError!
+
+reportingMetricsByAsset(
+ metricsFilter: AssetReportingMetricsFilter
+ metricsSelector: ReportingMetricsSelector!
+): ReportingMetricsOrError!
+
+reportingMetricsByAssetGroup(
+ metricsFilter: AssetGroupReportingMetricsFilter
+ metricsSelector: ReportingMetricsSelector!
+): ReportingMetricsOrError!
+```
+
+### Specifying metrics and time granularity
+
+Use `metricsSelector` to specify the metric name and time granularity:
+
+```graphql
+input ReportingMetricsSelector {
+ after: Float # timestamp
+ before: Float # timestamp
+ metricName: String # see below for valid values
+ granularity: ReportingMetricsGranularity
+}
+
+enum ReportingMetricsGranularity {
+ DAILY
+ WEEKLY
+ MONTHLY
+}
+
+# The valid metric names are:
+# "__dagster_dagster_credits"
+# "__dagster_execution_time_ms"
+# "__dagster_materializations"
+# "__dagster_step_failures"
+# "__dagster_step_retries"
+# "__dagster_asset_check_errors"
+# "__dagster_asset_check_warnings"
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/google-bigquery.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/google-bigquery.md
new file mode 100644
index 0000000000000..26c88c2cb58fd
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/google-bigquery.md
@@ -0,0 +1,76 @@
+---
+title: "Track Google BigQuery usage with Dagster+ Insights"
+sidebar_label: "Google BigQuery"
+sidebar_position: 400
+---
+
+Dagster allows you to track external metrics, such as BigQuery usage in the Insights UI. Out of the box integrations are provided to capture query runtime and billed usage, and associate them with the relevant assets or jobs.
+
+:::note
+The BigQuery cost metric is based off of the bytes billed for queries executed with Dagster, based on a unit price of $6.25 per TiB.
+:::
+
+## Requirements
+
+To use these features, you will need:
+
+- A Dagster+ account on the **Pro** plan
+- Access to the [Dagster+ Insights feature](/dagster-plus/features/insights)
+- BigQuery credentials which have access to the **`INFORMATION_SCHEMA.JOBS`** table, such as a BigQuery Resource viewer role.
+ - For more information, see the [BigQuery Documentation](https://cloud.google.com/bigquery/docs/information-schema-jobs)
+- The following packages installed:
+
+```bash
+pip install dagster dagster-cloud
+```
+
+## Limitations
+
+- Up to two million individual data points may be added to Insights, per month
+- External metrics data will be retained for 120 days
+- Insights data may take up to 24 hours to appear in the UI
+
+## Tracking usage with the BigQueryResource
+
+The `dagster-cloud` package provides an `InsightsBigQueryResource`, which is a drop-in replacement for the `BigQueryResource` provided by `dagster-gcp`.
+
+This resource will emit BigQuery usage metrics to the Dagster+ Insights API whenever it makes a query.
+
+To enable this behavior, replace usage of `BigQueryResource` with `InsightsBigQueryResource`.
+
+
+
+
+
+
+
+
+
+
+## Tracking usage with dagster-dbt
+
+If you use `dagster-dbt` to manage a dbt project that targets Google BigQuery, you can emit usage metrics to the Dagster+ API with the `DbtCliResource`.
+
+First, add a `.with_insights()` call to your `dbt.cli()` command(s).
+
+
+
+
+
+
+
+
+
+
+Then, add the following to your `dbt_project.yml`:
+
+
+
+
+
+
+
+
+
+
+This adds a comment to each query, which is used by Dagster+ to attribute cost metrics to the correct assets.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/index.md
new file mode 100644
index 0000000000000..6abb2bb09f4df
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/index.md
@@ -0,0 +1,71 @@
+---
+title: Dagster+ Insights
+sidebar_label: 'Insights'
+description: 'Visiblity into historical usage, cost, and metadata.'
+sidebar_position: 10
+---
+
+Using Dagster+ Insights, you can gain visibility into historical usage and cost metrics such as Dagster+ run duration, credit usage, and failures. You can also define your own custom metrics, such as the number of rows processed by an asset.
+
+Visualizations are built into the Dagster+ UI, allowing you to explore metrics from Dagster and external systems, such as Google BigQuery, in one place.
+
+### With Insights, you can
+
+- [Explore usage trends in your Dagster pipelines](#explore-dagsters-built-in-metrics)
+- [Integrate additional metrics](#integrate-metrics), like data warehouse cost or your own custom metadata
+- [Export metrics](#export-metrics) from Dagster+
+- [Create alerts](/dagster-plus/features/alerts) based off of Insights metrics TODO: write this alerts section
+
+
+ Prerequisites
+
+To use Insights, you'll need a Dagster+ account.
+
+
+
+## Explore Dagster's built-in metrics
+
+To access Insights, click **Insights** in the top navigation bar in the UI:
+
+![Viewing the Insights tab in the Dagster+ UI](/images/dagster-plus/features/insights/insights-tab.png)
+
+The left navigation panel on this page contains a list of available metrics. For each metric, the daily, weekly, or monthly aggregated values are displayed in the graph.
+
+Use the tabs above the charts to view metrics for **Assets**, **Asset groups**, **Jobs**, and **Deployments**.
+
+These metrics are updated on a daily basis. Refer to the [Built-in metrics](#built-in-metrics) section for more information about what Dagster provides out of the box.
+
+## Working with Insights metrics \{#insights-metrics}
+
+### Data retention
+
+How long historical Insights data is retained depends on your Dagster+ plan:
+
+- **Dagster+ Pro** - 120 days
+- **All other plans** - 30 days
+
+### Built-in metrics
+
+| Metric | Description |
+| -------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ |
+| Dagster credits | The Dagster credit cost associated with computing this object. Dagster credits are charged for every step that's run, and for every asset that's materialized. For more information, [refer to the pricing FAQ](https://dagster.io/pricing#faq). |
+| Compute duration | The time spent computing steps. For jobs that run steps in parallel, the compute duration may be longer than the wall clock time it takes for the run to complete. |
+| Materializations | The number of asset materializations associated with computing this object. |
+| Observations | The number of [asset observations](/guides/build/assets/metadata-and-tags/asset-observations) associated with computing this object. |
+| Step failures | The number of times steps failed when computing this object. **Note**: Steps that retry and succeed aren't included in this metric. |
+| Step retries | The number of times steps were retried when computing this object. |
+| Asset check warnings | The number of [asset checks](/guides/test/asset-checks) that produced warnings. |
+| Asset check errors | The number of [asset checks](/guides/test/asset-checks) that produced errors. |
+| Retry compute | The time spent computing steps, including time spent retrying failed steps. For jobs that run steps in parallel, the compute duration may be longer than the wall clock time it takes for the run to complete. |
+
+### Integrate other metrics \{#integrate-metrics}
+
+Users on the Pro plan can integration other metrics into Insights, such as asset materialization metadata or Snowflake credits. Insights supports the following additional metrics:
+
+- **Asset materialization metadata.** Refer to the [Using asset metadata with Dagster+ Insights guide](/dagster-plus/features/insights/asset-metadata) for more info.
+- [**Google BigQuery usage**](/dagster-plus/features/insights/google-bigquery) generated by either queries made to BigQuery resources or using dbt to materialize tables
+- [**Snowflake usage**](/dagster-plus/features/insights/snowflake) generated by either queries made to Snowflake resources or using dbt to materialize tables
+
+### Export metrics
+
+Metrics in Dagster+ Insights can be exported using a GraphQL API endpoint. Refer to the [Exporting Insights metrics from Dagster+ guide](/dagster-plus/features/insights/export-metrics) for details.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/snowflake.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/snowflake.md
new file mode 100644
index 0000000000000..7f19521ec4e39
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/features/insights/snowflake.md
@@ -0,0 +1,84 @@
+---
+title: "Track Snowflake usage with Dagster+ Insights"
+sidebar_label: "Snowflake"
+sidebar_position: 300
+---
+
+Dagster allows you to track external metrics, such as Snowflake usage, in the Insights UI. Out of the box integrations are provided to capture query runtime and billed usage, and associate them with the relevant assets or jobs.
+
+## Requirements
+
+To use these features, you will need:
+
+- A Dagster+ account on the **Pro** plan
+- Access to the [Dagster+ Insights feature](/dagster-plus/features/insights)
+- Snowflake credentials which have access to the **`snowflake.account_usage.query_history`**.
+ - For more information, see the [Snowflake Documentation](https://docs.snowflake.com/en/sql-reference/account-usage#enabling-the-snowflake-database-usage-for-other-roles)
+- The following packages installed:
+
+```bash
+pip install dagster dagster-cloud dagster-snowflake
+```
+
+## Limitations
+
+- Up to two million individual data points may be added to Insights, per month
+- External metrics data will be retained for 120 days
+- Insights data may take up to 24 hours to appear in the UI
+
+## Tracking usage with the SnowflakeResource
+
+The `dagster-cloud` package provides an `InsightsSnowflakeResource`, which is a drop-in replacement for the `SnowflakeResource` provided by `dagster-snowflake`.
+
+This resource will emit Snowflake usage metrics to the Dagster+ Insights API whenever it makes a query.
+
+To enable this behavior, replace usage of `SnowflakeResource` with `InsightsSnowflakeResource`, and add Snowflake-specific insights definitions to your code using `create_snowflake_insights_asset_and_schedule`.
+
+These additional definitions are required because Snowflake usage information is only available after a delay. These definitions automatically handle running a computation on a schedule to ingest Snowflake usage information from the previous hour.
+
+:::note
+Only use `create_snowflake_insights_asset_and_schedule` in a single code location per deployment, as this will handle ingesting usage data from your entire deployment.
+:::
+
+
+
+
+
+
+
+
+
+
+## Tracking usage with dagster-dbt
+
+If you use `dagster-dbt` to manage a dbt project that targets Snowflake, you can emit usage metrics to the Dagster+ API with the `DbtCliResource`.
+
+First, add a `.with_insights()` call to your `dbt.cli()` command(s), and add Snowflake-specific insights definitions to your code using `create_snowflake_insights_asset_and_schedule`.
+
+These additional definitions are required because Snowflake usage information is only available after a delay. These definitions automatically handle running a computation on a schedule to ingest Snowflake usage information from the previous hour.
+
+:::note
+Only use `create_snowflake_insights_asset_and_schedule` in a single code location per deployment, as this will handle ingesting usage data from your entire deployment.
+:::
+
+
+
+
+
+
+
+
+
+
+Then, add the following to your `dbt_project.yml`:
+
+
+
+
+
+
+
+ This adds a comment to each query, which is used by Dagster+ to attribute cost metrics to the correct assets.
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/getting-started.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/getting-started.md
new file mode 100644
index 0000000000000..73ca4216301a3
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/getting-started.md
@@ -0,0 +1,36 @@
+---
+title: "Getting started with Dagster+"
+---
+
+To get started with Dagster+, you will need to create a Dagster+ organization and choose your deployment type (Serverless or Hybrid).
+
+## Create a Dagster+ organization
+
+First, [create a Dagster+ organization](https://dagster.plus/signup). You can sign up with:
+- a Google email address
+- a GitHub account
+- a one-time email link (ideal if you are using a corporate email). You can set up SSO after completing these steps.
+
+## Choose your deployment type
+
+### Dagster+ Serverless
+
+[Dagster+ Serverless](/dagster-plus/deployment/deployment-types/serverless) is the easiest way to get started and is a good options if you have limited DevOps support. In Dagster+ Serverless, your Dagster code is executed in Dagster+, so you will need to be comfortable sharing credentials with Dagster+ for the tools you want to orchestrate.
+
+To get started with Dagster+ Serverless, follow the Dagster+ onboarding to add a new project. You will be guided through the steps to create a Git repository with your Dagster code and set up the necessary CI/CD actions to deploy that repository to Dagster+.
+
+:::tip
+
+If you don't have any Dagster code, you can select an example project or import an existing dbt project.
+
+:::
+
+### Dagster+ Hybrid
+
+[Dagster+ Hybrid](/dagster-plus/deployment/deployment-types/hybrid) is a good choice if you want to orchestrate tools without giving Dagster+ direct access to your systems. Dagster+ Hybrid requires more DevOps support.
+
+To get started with Dagster+ Hybrid, follow the steps in the [Dagster+ Hybrid documentation](/dagster-plus/deployment/deployment-types/hybrid/) to install a Dagster+ Hybrid agent and set up CI/CD.
+
+## Next steps
+
+Your Dagster+ account is automatically enrolled in a trial. You can [pick your plan type and enter your billing information](/dagster-plus/deployment/management/settings/dagster-plus-settings), or [contact the Dagster team](https://dagster.io/contact) if you need support or want to evaluate the Dagster+ Pro plan.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/index.md
new file mode 100644
index 0000000000000..290fc6d4c7407
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/dagster-plus/index.md
@@ -0,0 +1,34 @@
+---
+title: "About Dagster+"
+sidebar_position: 1
+---
+
+Dagster+ is a managed orchestration platform built on top of Dagster's open source engine.
+
+Dagster+ is built to be the most performant, reliable, and cost effective way for data engineering teams to run Dagster in production. Dagster+ is also great for students, researchers, or individuals who want to explore Dagster with minimal overhead.
+
+Dagster+ comes in two flavors: a fully [Serverless](/dagster-plus/deployment/deployment-types/serverless) offering and a [Hybrid](/dagster-plus/deployment/deployment-types/hybrid) offering. In both cases, Dagster+ does the hard work of managing your data orchestration control plane. Compared to a [Dagster open source deployment](guides/deploy/index.md), Dagster+ manages:
+
+- Dagster's web UI at https://dagster.plus
+- Metadata stores for data cataloging and cost insights
+- Backend services for orchestration, alerting, and more
+
+Dagster+ Serverless is fully managed and your Dagster code executes in our environment. In Dagster+ Hybrid, you run an execution environment that connects to the Dagster+ control plane.
+
+In addition to managed infrastructure, Dagster+ also adds core capabilities on top of Dagster open source to enable teams building data platforms:
+
+- [Insights](/dagster-plus/features/insights), a powerful tool for identifying trends in your data platform overtime, optimizing cost, and answering questions like "Why does it feel like our pipelines are taking longer this month?".
+- [Alerts](/dagster-plus/features/alerts) to a variety of services like Slack, PagerDuty, and email to notify your team of failed runs, data quality issues, and violated SLAs.
+- Authentication, [Role Based Access Control](/dagster-plus/features/authentication-and-access-control/rbac), and [Audit Logs](/dagster-plus/features/authentication-and-access-control/rbac/audit-logs) which help teams implement data mesh strategies while remaining compliant.
+- [Asset Catalog](/dagster-plus/features/asset-catalog/), a powerful search-first experience that builds off of Dagster's best-in-class lineage graph to include searching for assets, metadata, column lineage, and more.
+- [Branch Deployments](/dagster-plus/features/ci-cd/branch-deployments/index.md)
+
+Ready to [get started](/dagster-plus/getting-started)?
+
+## Other resources
+
+- Learn more about Dagster+ [pricing and plan types](https://dagster.io/pricing) or [contact the Dagster team](https://dagster.io/contact)
+- Dagster+ includes support, [click here](https://dagster.io/support) to learn more.
+- Dagster+ is HIPAA compliant, SOC 2 Type II certified, and meets GDPR requirements. Learn more about Dagster+[ security](https://dagster.io/security).
+- Migrate [from a Dagster open source deployment to Dagster+](/dagster-plus/deployment/migration/self-hosted-to-dagster-plus)
+- Dagster+ [status page](https://dagstercloud.statuspage.io/)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/automate-your-pipeline.md b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/automate-your-pipeline.md
new file mode 100644
index 0000000000000..e7ec7c705bb49
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/automate-your-pipeline.md
@@ -0,0 +1,56 @@
+---
+title: Automate your pipeline
+description: Set schedules and utilize asset based automation
+last_update:
+ author: Alex Noonan
+sidebar_position: 50
+---
+
+There are several ways to automate pipelines and assets [in Dagster](/guides/automate).
+
+In this step you will:
+
+- Add automation to assets to run when upstream assets are materialized.
+- Create a schedule to run a set of assets on a cron schedule.
+
+## 1. Automate asset materialization
+
+Ideally, the reporting assets created in the last step should refresh whenever the upstream data is updated. Dagster's [declarative automation](/guides/automate/declarative-automation) framework allows you do this by adding an automation condition to the asset definition.
+
+Update the `monthly_sales_performance` asset to add the automation condition to the decorator:
+
+
+
+Do the same thing for `product_performance`:
+
+
+
+## 2. Scheduled jobs
+
+Cron-based schedules are common in data orchestration. For our pipeline, assume that updated CSVs are uploaded to a file location at a specific time every week by an external process.
+
+Copy the following code underneath the `product performance` asset:
+
+
+
+## 3. Enable and test automations
+
+The final step is to enable the automations in the UI.
+
+To accomplish this:
+1. Navigate to the Automation page.
+2. Select all automations.
+3. Using actions, start all automations.
+4. Select the `analysis_update_job`.
+5. Test the schedule and evaluate for any time in the dropdown menu.
+6. Open in Launchpad.
+
+The job is now executing.
+
+Additionally, if you navigate to the Runs tab, you should see that materializations for `monthly_sales_performance` and `product_performance` have run as well.
+
+ ![2048 resolution](/images/tutorial/etl-tutorial/automation-final.png)
+
+## Next steps
+
+- Continue this tutorial with adding a [sensor based asset](create-a-sensor-asset)
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-a-sensor-asset.md b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-a-sensor-asset.md
new file mode 100644
index 0000000000000..addd550c8283a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-a-sensor-asset.md
@@ -0,0 +1,69 @@
+---
+title: Create a sensor asset
+description: Use sensors to create event driven pipelines
+last_update:
+ author: Alex Noonan
+sidebar_position: 60
+---
+
+[Sensors](/guides/automate/sensors) allow you to automate workflows based on external events or conditions, making them useful for event-driven automation, especially in situations where jobs occur at irregular cadences or in rapid succession.
+
+Consider using sensors in the following situations:
+- **Event-driven workflows**: When your workflow depends on external events, such as the arrival of a new data file or a change in an API response.
+- **Conditional execution**: When you want to execute jobs only if certain conditions are met, reducing unnecessary computations.
+- **Real-time processing**: When you need to process data as soon as it becomes available, rather than waiting for a scheduled time.
+
+In this step you will:
+
+- Create an asset that runs based on a event-driven workflow
+- Create a sensor to listen for conditions to materialize the asset
+
+## 1. Create an event-driven asset
+
+For our pipeline, we want to model a situation where an executive wants a pivot table report of sales results by department and product. They want that processed in real time from their request.
+
+For this asset, we need to define the structure of the request that it is expecting in the materialization context.
+
+Other than that, defining this asset is the same as our previous assets. Copy the following code beneath `product_performance`.
+
+
+
+## 2. Build the sensor
+
+To define a sensor in Dagster, use the `@sensor` decorator. This decorator is applied to a function that evaluates whether the conditions for triggering a job are met.
+
+Sensors include the following elements:
+
+- **Job**: The job that the sensor will trigger when the conditions are met.
+- **RunRequest**: An object that specifies the configuration for the job run. It includes a `run_key` to ensure idempotency and a `run_config` for job-specific settings.
+
+
+
+## 3. Materialize the sensor asset
+
+1. Update your Definitions object to the following:
+
+
+
+2. Reload your Definitions.
+
+3. Navigate to the Automation page.
+
+4. Turn on the `adhoc_request_sensor`.
+
+5. Click on the `adhoc_request_sensor` details.
+
+ ![2048 resolution](/images/tutorial/etl-tutorial/sensor-evaluation.png)
+
+6. Add `request.json` from the `sample_request` folder to `requests` folder.
+
+7. Click on the green tick to see the run for this request.
+
+ ![2048 resolution](/images/tutorial/etl-tutorial/sensor-asset-run.png)
+
+
+## Next steps
+
+Now that we have our complete project, the next step is to refactor the project into more a more manageable structure so we can add to it as needed.
+
+Finish the tutorial by [refactoring your project](refactor-your-project).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-and-materialize-a-downstream-asset.md b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-and-materialize-a-downstream-asset.md
new file mode 100644
index 0000000000000..5bc8443ddd4a9
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-and-materialize-a-downstream-asset.md
@@ -0,0 +1,43 @@
+---
+title: Create and materialize a downstream asset
+description: Reference Assets as dependencies to other assets
+last_update:
+ author: Alex Noonan
+sidebar_position: 20
+---
+
+Now that we have the raw data loaded into DuckDB, we need to create a [downstream asset](/guides/build/assets/defining-assets-with-asset-dependencies) that combines the upstream assets together. In this step, you will:
+
+- Create a downstream asset
+- Materialize that asset
+
+## 1. Create a downstream asset
+
+Now that we have all of our raw data loaded into DuckDB, our next step is to merge it together in a view composed of data from all three source tables.
+
+To accomplish this in SQL, we will bring in our `sales_data` table and then left join on `sales_reps` and `products` on their respective id columns. Additionally, we will keep this view concise and only have relevant columns for analysis.
+
+As you can see, the new `joined_data` asset looks a lot like our previous ones, with a few small changes. We put this asset into a different group. To make this asset dependent on the raw tables, we add the asset keys to the `deps` parameter in the asset definition.
+
+
+
+## 2. Materialize the asset
+
+1. Add the joined_data asset to the Definitions object
+
+ ```python
+ defs = dg.Definitions(
+ assets=[products,
+ sales_reps,
+ sales_data,
+ joined_data,
+ ],
+ resources={"duckdb": DuckDBResource(database="data/mydb.duckdb")},
+ )
+ ```
+
+2. In the Dagster UI, reload definitions and materialize the `joined_data` asset.
+
+## Next steps
+
+- Continue this tutorial with by [creating and materializing a partitioned asset](ensure-data-quality-with-asset-checks).
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-and-materialize-assets.md b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-and-materialize-assets.md
new file mode 100644
index 0000000000000..9a3d2d6ed6549
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-and-materialize-assets.md
@@ -0,0 +1,108 @@
+---
+title: Create and materialize assets
+description: Load project data and create and materialize assets
+last_update:
+ author: Alex Noonan
+sidebar_position: 10
+---
+
+
+In the first step of the tutorial, you created your Dagster project with the raw data files. In this step, you will:
+- Create your initial Definitions object
+- Add a DuckDB resource
+- Build software-defined assets
+- Materialize your assets
+
+## 1. Create a definitions object
+
+In Dagster, the object is where you define and organize various components within your project, such as assets and resources.
+
+Open the `definitions.py` file in the `etl_tutorial` directory and copy the following code into it:
+
+ ```python
+ import json
+ import os
+
+ from dagster_duckdb import DuckDBResource
+
+ import dagster as dg
+
+ defs = dg.Definitions(
+ assets=[],
+ resources={},
+ )
+ ```
+
+## 2. Define the DuckDB resource
+
+In Dagster, [Resources](/api/python-api/resources) are the external services, tools, and storage backends you need to do your job. For the storage backend in this project, we'll use [DuckDB](https://duckdb.org/), a fast, in-process SQL database that runs inside your application. We'll define it once in the definitions object, making it available to all assets and objects that need it.
+
+ ```python
+ defs = dg.Definitions(
+ assets=[],
+ resources={"duckdb": DuckDBResource(database="data/mydb.duckdb")},
+ )
+ ```
+
+## 3. Create assets
+
+Software defined are the main building blocks in Dagster. An asset is composed of three components:
+1. Asset key or unique identifier.
+2. An op which is a function that is invoked to produce the asset.
+3. Upstream dependencies that the asset depends on.
+
+You can read more about our philosophy behind the [asset centric approach](https://dagster.io/blog/software-defined-assets).
+
+### Products asset
+
+First, we will create an asset that creates a DuckDB table to hold data from the products CSV. This asset takes the `duckdb` resource defined earlier and returns a object.
+Additionally, this asset contains metadata in the decorator parameters to help categorize the asset, and in the `return` block to give us a preview of the asset in the Dagster UI.
+
+To create this asset, open the `definitions.py` file and copy the following code into it:
+
+
+
+### Sales reps asset
+
+The code for the sales reps asset is similar to the product asset code. In the `definitions.py` file, copy the following code below the product asset code:
+
+
+
+### Sales data asset
+
+To add the sales data asset, copy the following code into your `definitions.py` file below the sales reps asset:
+
+
+
+## 4. Add assets to the definitions object
+
+Now to pull these assets into our Definitions object. Adding them to the Definitions object makes them available to the Dagster project. Add them to the empty list in the assets parameter.
+
+ ```python
+ defs = dg.Definitions(
+ assets=[products,
+ sales_reps,
+ sales_data,
+ ],
+ resources={"duckdb": DuckDBResource(database="data/mydb.duckdb")},
+ )
+ ```
+
+## 5. Materialize assets
+
+To materialize your assets:
+1. In a browser, navigate to the URL of the Dagster server that yous started earlier.
+2. Navigate to **Deployment**.
+3. Click Reload definitions.
+4. Click **Assets**, then click "View global asset lineage" to see all of your assets.
+
+ ![2048 resolution](/images/tutorial/etl-tutorial/etl-tutorial-first-asset-lineage.png)
+
+5. Click materialize all.
+6. Navigate to the runs tab and select the most recent run. Here you can see the logs from the run.
+ ![2048 resolution](/images/tutorial/etl-tutorial/first-asset-run.png)
+
+
+## Next steps
+
+- Continue this tutorial with your [asset dependencies](create-and-materialize-a-downstream-asset)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-and-materialize-partitioned-asset.md b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-and-materialize-partitioned-asset.md
new file mode 100644
index 0000000000000..dd64ed6a8858c
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/create-and-materialize-partitioned-asset.md
@@ -0,0 +1,180 @@
+---
+title: Create and materialize partitioned assets
+description: Partitioning Assets by datetime and categories
+last_update:
+ date: 2024-11-25
+ author: Alex Noonan
+sidebar_position: 40
+---
+
+[Partitions](/guides/build/partitions-and-backfills/partitioning-assets) are a core abstraction in Dagster, that allow you to manage large datasets, process incremental updates, and improve pipeline performance. You can partition assets the following ways:
+
+- Time-based: Split data by time periods (e.g., daily, monthly)
+- Category-based: Divide by known categories (e.g., country, product type)
+- Two-dimensional: Combine two partition types (e.g., country + date)
+- Dynamic: Create partitions based on runtime conditions
+
+In this step, you will:
+
+- Create a time-based asset partitioned by month
+- Create a category-based asset partitioned by product category
+
+## 1. Create a time-based partitioned asset
+
+Dagster natively supports partitioning assets by datetime groups. We want to create an asset that calculates the monthly performance for each sales rep. To create the monthly partition copy the following code below the `missing_dimension_check` asset check.
+
+
+
+Partition data are accessed within an asset by context. We want to create an asset that does this calculation for a given month from the partition
+ and deletes any previous value for that month. Copy the following asset under the `monthly_partition` we just created.
+
+ ```python
+ @dg.asset(
+ partitions_def=monthly_partition,
+ compute_kind="duckdb",
+ group_name="analysis",
+ deps=[joined_data],
+ )
+ def monthly_sales_performance(
+ context: dg.AssetExecutionContext, duckdb: DuckDBResource
+ ):
+ partition_date_str = context.partition_key
+ month_to_fetch = partition_date_str[:-3]
+
+ with duckdb.get_connection() as conn:
+ conn.execute(
+ f"""
+ create table if not exists monthly_sales_performance (
+ partition_date varchar,
+ rep_name varchar,
+ product varchar,
+ total_dollar_amount double
+ );
+
+ delete from monthly_sales_performance where partition_date = '{month_to_fetch}';
+
+ insert into monthly_sales_performance
+ select
+ '{month_to_fetch}' as partition_date,
+ rep_name,
+ product_name,
+ sum(dollar_amount) as total_dollar_amount
+ from joined_data where strftime(date, '%Y-%m') = '{month_to_fetch}'
+ group by '{month_to_fetch}', rep_name, product_name;
+ """
+ )
+
+ preview_query = f"select * from monthly_sales_performance where partition_date = '{month_to_fetch}';"
+ preview_df = conn.execute(preview_query).fetchdf()
+ row_count = conn.execute(
+ f"""
+ select count(*)
+ from monthly_sales_performance
+ where partition_date = '{month_to_fetch}'
+ """
+ ).fetchone()
+ count = row_count[0] if row_count else 0
+
+ return dg.MaterializeResult(
+ metadata={
+ "row_count": dg.MetadataValue.int(count),
+ "preview": dg.MetadataValue.md(preview_df.to_markdown(index=False)),
+ }
+ )
+ ```
+
+## 2. Create a category-based partitioned asset
+
+Using known defined partitions is a simple way to break up your dataset when you know the different groups you want to subset it by. In our pipeline, we want to create an asset that represents the performance of each product category.
+
+1. To create the statically-defined partition for the product category, copy this code beneath the `monthly_sales_performance` asset:
+
+
+
+2. Now that the partition has been defined, we can use that in an asset that calculates the product category performance:
+
+```python
+@dg.asset(
+ deps=[joined_data],
+ partitions_def=product_category_partition,
+ group_name="analysis",
+ compute_kind="duckdb",
+)
+def product_performance(context: dg.AssetExecutionContext, duckdb: DuckDBResource):
+ product_category_str = context.partition_key
+
+ with duckdb.get_connection() as conn:
+ conn.execute(
+ f"""
+ create table if not exists product_performance (
+ product_category varchar,
+ product_name varchar,
+ total_dollar_amount double,
+ total_units_sold double
+ );
+
+ delete from product_performance where product_category = '{product_category_str}';
+
+ insert into product_performance
+ select
+ '{product_category_str}' as product_category,
+ product_name,
+ sum(dollar_amount) as total_dollar_amount,
+ sum(quantity) as total_units_sold
+ from joined_data
+ where category = '{product_category_str}'
+ group by '{product_category_str}', product_name;
+ """
+ )
+ preview_query = f"select * from product_performance where product_category = '{product_category_str}';"
+ preview_df = conn.execute(preview_query).fetchdf()
+ row_count = conn.execute(
+ f"""
+ SELECT COUNT(*)
+ FROM product_performance
+ WHERE product_category = '{product_category_str}';
+ """
+ ).fetchone()
+ count = row_count[0] if row_count else 0
+
+ return dg.MaterializeResult(
+ metadata={
+ "row_count": dg.MetadataValue.int(count),
+ "preview": dg.MetadataValue.md(preview_df.to_markdown(index=False)),
+ }
+ )
+```
+
+
+
+## 3. Materialize partitioned assets
+
+Now that we have our partitioned assets, let's add them to our Definitions object:
+
+Your Definitions object should look like this:
+
+```python
+defs = dg.Definitions(
+ assets=[products,
+ sales_reps,
+ sales_data,
+ joined_data,
+ monthly_sales_performance,
+ product_performance,
+ ],
+ asset_checks=[missing_dimension_check],
+ resources={"duckdb": DuckDBResource(database="data/mydb.duckdb")},
+)
+```
+
+To materialize these assets:
+1. Navigate to the assets page.
+2. Reload definitions.
+3. Select the `monthly_performance` asset, then **Materialize selected**.
+4. Ensure all partitions are selected, then launch a backfill.
+5. Select the `product_performance` asset, then **Materialize selected**.
+6. Ensure all partitions are selected, then launch a backfill.
+
+## Next steps
+
+Now that we have the main assets in our ETL pipeline, its time to add [automation to our pipeline](automate-your-pipeline)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/ensure-data-quality-with-asset-checks.md b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/ensure-data-quality-with-asset-checks.md
new file mode 100644
index 0000000000000..f45b99b0ba68a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/ensure-data-quality-with-asset-checks.md
@@ -0,0 +1,52 @@
+---
+title: Ensure data quality with asset checks
+description: Ensure assets are correct with asset checks
+last_update:
+ author: Alex Noonan
+sidebar_position: 30
+---
+
+Data quality is critical in data pipelines. Inspecting individual assets ensures that data quality issues are caught before they affect the entire pipeline.
+
+In Dagster, you define [asset checks](/guides/test/asset-checks) like you define assets. Asset checks run when an asset is materialized. In this step you will:
+
+- Define an asset check
+- Execute that asset check in the UI
+
+## 1. Define an asset check
+
+In this case we want to create a check to identify if there are any rows in `joined_data` that are missing a value for `rep_name` or `product_name`.
+
+Copy the following code beneath the `joined_data` asset.
+
+
+
+## 2. Run the asset check
+
+Before you can run the asset check, you need to add it to the Definitions object. Like assets, asset checks are added to their own list.
+
+Your Definitions object should look like this now:
+
+```python
+defs = dg.Definitions(
+ assets=[products,
+ sales_reps,
+ sales_data,
+ joined_data,
+ ],
+ asset_checks=[missing_dimension_check],
+ resources={"duckdb": DuckDBResource(database="data/mydb.duckdb")},
+)
+```
+Asset checks will run when an asset is materialized, but asset checks can also be executed manually in the UI:
+
+1. Reload your Definitions.
+2. Navigate to the Asset Details page for the `joined_data` asset.
+3. Select the "Checks" tab.
+4. Click the **Execute** button for `missing_dimension_check`.
+
+ ![2048 resolution](/images/tutorial/etl-tutorial/asset-check.png)
+
+## Next steps
+
+- Continue this tutorial with [Asset Checks](create-and-materialize-partitioned-asset)
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/index.md
new file mode 100644
index 0000000000000..4ff25d9918423
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/index.md
@@ -0,0 +1,137 @@
+---
+title: Build an ETL Pipeline
+description: Learn how to build an ETL pipeline with Dagster
+last_update:
+ author: Alex Noonan
+sidebar_class_name: hidden
+---
+
+# Build your first ETL pipeline
+
+In this tutorial, you'll build an ETL pipeline with Dagster that:
+
+- Imports sales data to DuckDB
+- Transforms data into reports
+- Runs scheduled reports automatically
+- Generates one-time reports on demand
+
+## You will learn to:
+
+- Set up a Dagster project with the recommended project structure
+- Create and materialize assets
+- Create and materialize dependant assets
+- Ensure data quality with asset checks
+- Create and materialize partitioned assets
+- Automate the pipeline
+- Create and materialize a sensor asset
+- Refactor your project when it becomes more complex
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- Basic Python knowledge
+- Python 3.9+ installed on your system. Refer to the [Installation guide](/getting-started/installation) for information.
+- Familiarity with SQL and Python data manipulation libraries, such as Pandas.
+- Understanding of data pipelines and the extract, transform, and load process.
+
+
+
+## Step 1: Set up your Dagster environment
+
+First, set up a new Dagster project.
+
+1. Open your terminal and create a new directory for your project:
+
+ ```bash
+ mkdir dagster-etl-tutorial
+ cd dagster-etl-tutorial
+ ```
+
+2. Create and activate a virtual environment:
+
+
+
+ ```bash
+ python -m venv dagster_tutorial
+ source dagster_tutorial/bin/activate
+ ```
+
+
+ ```bash
+ python -m venv dagster_tutorial
+ dagster_tutorial\Scripts\activate
+ ```
+
+
+
+3. Install Dagster and the required dependencies:
+
+ ```bash
+ pip install dagster dagster-webserver pandas dagster-duckdb
+ ```
+
+## Step 2: Create the Dagster project structure
+
+Run the following command to create the project directories and files for this tutorial:
+
+ ```bash
+ dagster project from-example --example getting_started_etl_tutorial
+ ```
+
+Your project should have this structure:
+{/* vale off */}
+```
+dagster-etl-tutorial/
+├── data/
+│ └── products.csv
+│ └── sales_data.csv
+│ └── sales_reps.csv
+│ └── sample_request/
+│ └── request.json
+├── etl_tutorial/
+│ └── definitions.py
+├── pyproject.toml
+├── setup.cfg
+├── setup.py
+```
+{/* vale on */}
+
+:::info
+Dagster has several example projects you can install depending on your use case. To see the full list, run `dagster project list-examples`. For more information on the `dagster project` command, see the [API documentation](https://docs-preview.dagster.io/api/cli#dagster-project).
+:::
+
+### Dagster project structure
+
+#### dagster-etl-tutorial root directory
+
+In the `dagster-etl-tutorial` root directory, there are three configuration files that are common in Python package management. These files manage dependencies and identify the Dagster modules in the project.
+| File | Purpose |
+|------|---------|
+| pyproject.toml | This file is used to specify build system requirements and package metadata for Python projects. It is part of the Python packaging ecosystem. |
+| setup.cfg | This file is used for configuration of your Python package. It can include metadata about the package, dependencies, and other configuration options. |
+| setup.py | This script is used to build and distribute your Python package. It is a standard file in Python projects for specifying package details. |
+
+#### etl_tutorial directory
+
+This is the main directory where you will define your assets, jobs, schedules, sensors, and resources.
+| File | Purpose |
+|------|---------|
+| definitions.py | This file is typically used to define jobs, schedules, and sensors. It organizes the various components of your Dagster project. This allows Dagster to load the definitions in a module. |
+
+#### data directory
+
+The data directory contains the raw data files for the project. We will reference these files in our software-defined assets in the next step of the tutorial.
+
+## Step 3: Launch the Dagster webserver
+
+To make sure Dagster and its dependencies were installed correctly, navigate to the project root directory and start the Dagster webserver:"
+
+ ```bash
+ dagster dev
+ ```
+
+## Next steps
+
+- Continue this tutorial by [creating and materializing assets](create-and-materialize-assets)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/refactor-your-project.md b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/refactor-your-project.md
new file mode 100644
index 0000000000000..58571bbdd7acf
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/etl-pipeline-tutorial/refactor-your-project.md
@@ -0,0 +1,83 @@
+---
+title: Refactor your project
+description: Refactor your completed project into a structure that is more organized and scalable.
+last_update:
+ author: Alex Noonan
+sidebar_position: 70
+---
+
+Many engineers generally leave something alone once it's working as expected. But the first time you do something is rarely the best implementation of a use case and all projects benefit from incremental improvements.
+
+## Splitting up project structure
+
+Currently, your project is contained in one definitions file. However, this file has gotten fairly complex, and adding to it would only increase its complexity. To fix that, we will create separate files for each core Dagster concept:
+
+- Assets
+- Schedules
+- Sensors
+- Partitions
+
+The final project structure should look like this:
+```
+dagster-etl-tutorial/
+├── data/
+│ └── products.csv
+│ └── sales_data.csv
+│ └── sales_reps.csv
+│ └── sample_request/
+│ └── request.json
+├── etl_tutorial/
+│ └── assets.py
+│ └── definitions.py
+│ └── partitions.py
+│ └── schedules.py
+│ └── sensors.py
+├── pyproject.toml
+├── setup.cfg
+├── setup.py
+```
+
+### Assets
+
+Assets make up a majority of our project and this will be our largest file.
+
+
+
+### Schedules
+
+The schedules file will only contain the `weekly_update_schedule`.
+
+
+
+### Sensors
+
+The sensors file will have the job and sensor for the `adhoc_request` asset.
+
+
+
+## Refactoring the Definitions object
+
+Now that we have separate files, we need to adjust how the different elements are added to the Definitions object.
+
+:::note
+The Dagster project runs from the root directory, so whenever you reference files in your project, you need to use the root as the starting point.
+Additionally, Dagster has functions to load all assets and asset checks from a module (load_assets_from_modules and load_asset_checks_from_modules, respectively).
+:::
+
+To bring your project together, copy the following code into your `definitions.py` file:
+
+
+
+## Quick validation
+
+To validate that your definitions file loads and validates, you can run `dagster definitions validate` in the same directory that you would run `dagster dev`. This command is useful for CI/CD pipelines and allows you to check that your project loads correctly without starting the web server.
+
+## Thats it!
+
+Congratulations! You have completed your first project with Dagster and have an example of how to use the building blocks to build your own data pipelines.
+
+## Recommended next steps
+
+- Join our [Slack community](https://dagster.io/slack).
+- Continue learning with [Dagster University](https://courses.dagster.io/) courses.
+- Start a [free trial of Dagster+](https://dagster.cloud/signup) for your own project.
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/getting-started/glossary.md b/docs/docs-beta/versioned_docs/version-1.9.10/getting-started/glossary.md
new file mode 100644
index 0000000000000..f2319cb8b8100
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/getting-started/glossary.md
@@ -0,0 +1,18 @@
+---
+title: Glossary
+sidebar_position: 30
+unlisted: true
+---
+
+# Glossary
+
+TODO - link to conceptual content about the following (and other key concepts if needed):
+
+* Assets
+* Definitions
+* Partitions (and backfills)
+* Resources
+* Schedules
+* Sensors
+* I/O managers?
+* Ops and jobs (and graphs?)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/getting-started/installation.md b/docs/docs-beta/versioned_docs/version-1.9.10/getting-started/installation.md
new file mode 100644
index 0000000000000..f92c8c8b09a91
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/getting-started/installation.md
@@ -0,0 +1,73 @@
+---
+title: Installing Dagster
+description: Learn how to install Dagster
+sidebar_position: 20
+sidebar_label: Installation
+---
+
+To follow the steps in this guide, you'll need:
+
+- To install Python 3.9 or higher. **Python 3.12 is recommended**.
+- To install pip, a Python package installer
+
+## Setting up a virtual environment
+
+After installing Python, it's recommended that you set up a virtual environment. This will isolate your Dagster project from the rest of your system and make it easier to manage dependencies.
+
+There are many ways to do this, but this guide will use `venv` as it doesn't require additional dependencies.
+
+
+
+```bash
+python -m venv venv
+source venv/bin/activate
+```
+
+
+```bash
+python -m venv venv
+source venv\Scripts\activate
+```
+
+
+
+:::tip
+**Looking for something more powerful than `venv`?** Try `pyenv` or `pyenv-virtualenv`, which can help you manage multiple versions of Python on a single machine. Learn more in the [pyenv GitHub repository](https://github.com/pyenv/pyenv).
+:::
+
+## Installing Dagster
+
+To install Dagster in your virtual environment, open your terminal and run the following command:
+
+```bash
+pip install dagster dagster-webserver
+```
+
+This command will install the core Dagster library and the webserver, which is used to serve the Dagster UI.
+
+## Verifying installation
+
+To verify that Dagster is installed correctly, run the following command:
+
+```bash
+dagster --version
+```
+
+The version numbers of Dagster should be printed in the terminal:
+
+```bash
+> dagster --version
+dagster, version 1.8.4
+```
+
+## Troubleshooting
+
+If you encounter any issues during the installation process:
+
+- Refer to the [Dagster GitHub repository](https://github.com/dagster-io/dagster) for troubleshooting, or
+- Reach out to the [Dagster community](/about/community)
+
+## Next steps
+
+- Get up and running with your first Dagster project in the [Quickstart](/getting-started/quickstart)
+- Learn to [create data assets in Dagster](/guides/build/assets/defining-assets)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/getting-started/quickstart.md b/docs/docs-beta/versioned_docs/version-1.9.10/getting-started/quickstart.md
new file mode 100644
index 0000000000000..c86d1453340fa
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/getting-started/quickstart.md
@@ -0,0 +1,155 @@
+---
+title: Build your first Dagster project
+description: Learn how to quickly get up and running with Dagster
+sidebar_position: 30
+sidebar_label: "Quickstart"
+---
+
+Welcome to Dagster! In this guide, you'll use Dagster to create a basic pipeline that:
+
+- Extracts data from a CSV file
+- Transforms the data
+- Loads the transformed data to a new CSV file
+
+## What you'll learn
+
+- How to set up a basic Dagster project
+- How to create a Dagster asset for each step of the Extract, Transform, and Load (ETL) process
+- How to use Dagster's UI to monitor and execute your pipeline
+
+## Prerequisites
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- Basic Python knowledge
+- Python 3.9+ installed on your system. Refer to the [Installation guide](/getting-started/installation) for information.
+
+
+## Step 1: Set up the Dagster environment
+
+1. Open the terminal and create a new directory for your project:
+
+ ```bash
+ mkdir dagster-quickstart
+ cd dagster-quickstart
+ ```
+
+2. Create and activate a virtual environment:
+
+
+
+ ```bash
+ python -m venv venv
+ source venv/bin/activate
+ ```
+
+
+ ```bash
+ python -m venv venv
+ source venv\Scripts\activate
+ ```
+
+
+
+3. Install Dagster and the required dependencies:
+
+ ```bash
+ pip install dagster dagster-webserver pandas
+ ```
+
+## Step 2: Create the Dagster project structure
+
+:::info
+The project structure in this guide is simplified to allow you to get started quickly. When creating new projects, use `dagster project scaffold` to generate a complete Dagster project.
+:::
+
+Next, you'll create a basic Dagster project that looks like this:
+
+```
+dagster-quickstart/
+├── quickstart/
+│ ├── __init__.py
+│ └── assets.py
+├── data/
+ └── sample_data.csv
+```
+
+1. To create the files and directories outlined above, run the following:
+
+ ```bash
+ mkdir quickstart data
+ touch quickstart/__init__.py quickstart/assets.py
+ touch data/sample_data.csv
+ ```
+
+2. In the `data/sample_data.csv` file, add the following content:
+
+ ```csv
+ id,name,age,city
+ 1,Alice,28,New York
+ 2,Bob,35,San Francisco
+ 3,Charlie,42,Chicago
+ 4,Diana,31,Los Angeles
+ ```
+
+ This CSV will act as the data source for your Dagster pipeline.
+
+## Step 3: Define the assets
+
+Now, create the assets for the ETL pipeline. Open `quickstart/assets.py` and add the following code:
+
+
+
+This may seem unusual if you're used to task-based orchestration. In that case, you'd have three separate steps for extracting, transforming, and loading.
+
+However, in Dagster, you'll model your pipelines using assets as the fundamental building block, rather than tasks.
+
+## Step 4: Run the pipeline
+
+1. In the terminal, navigate to your project's root directory and run:
+
+ ```bash
+ dagster dev -f quickstart/assets.py
+ ```
+
+2. Open your web browser and navigate to `http://localhost:3000`, where you should see the Dagster UI:
+
+ ![2048 resolution](/images/getting-started/quickstart/dagster-ui-start.png)
+
+3. In the top navigation, click **Assets > View global asset lineage**.
+
+4. Click **Materialize** to run the pipeline.
+
+5. In the popup that displays, click **View**. This will open the **Run details** page, allowing you to view the run as it executes.
+
+ ![2048 resolution](/images/getting-started/quickstart/run-details.png)
+
+ Use the **view buttons** in near the top left corner of the page to change how the run is displayed. You can also click the asset to view logs and metadata.
+
+## Step 5: Verify the results
+
+In your terminal, run:
+
+```bash
+cat data/processed_data.csv
+```
+
+You should see the transformed data, including the new `age_group` column:
+
+```bash
+id,name,age,city,age_group
+1,Alice,28,New York,Young
+2,Bob,35,San Francisco,Middle
+3,Charlie,42,Chicago,Senior
+4,Diana,31,Los Angeles,Middle
+```
+
+## Next steps
+
+Congratulations! You've just built and run your first pipeline with Dagster. Next, you can:
+
+- Continue with the [ETL pipeline tutorial](/etl-pipeline-tutorial/) to learn how to build a more complex ETL pipeline
+- Learn how to [Think in assets](/guides/build/assets/)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/automate/asset-sensors.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/automate/asset-sensors.md
new file mode 100644
index 0000000000000..8785086fa5698
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/automate/asset-sensors.md
@@ -0,0 +1,105 @@
+---
+title: "Asset sensors"
+sidebar_position: 40
+---
+
+Asset sensors in Dagster provide a powerful mechanism for monitoring asset materializations and triggering downstream computations or notifications based on those events.
+
+This guide covers the most common use cases for asset sensors, such as defining cross-job and cross-code location dependencies.
+
+:::note
+
+This documentation assumes familiarity with [assets](/guides/build/assets/) and [jobs](/guides/build/assets/asset-jobs)
+
+:::
+
+## Getting started
+
+Asset sensors monitor an asset for new materialization events and target a job when a new materialization occurs.
+
+Typically, asset sensors return a `RunRequest` when a new job is to be triggered. However, they may provide a `SkipReason` if the asset materialization doesn't trigger a job.
+
+For example, you may wish to monitor an asset that's materialized daily, but don't want to trigger jobs on holidays.
+
+## Cross-job and cross-code location dependencies
+
+Asset sensors enable dependencies across different jobs and different code locations. This flexibility allows for modular and decoupled workflows.
+
+```mermaid
+graph LR;
+
+AssetToWatch(AssetToWatch) --> AssetSensor(AssetSensor);
+AssetSensor--> Job(Job);
+Job --> Asset1(Asset1);
+Job --> Asset2(Asset1);
+
+subgraph CodeLocationA
+ AssetToWatch
+end
+
+subgraph CodeLocationB
+ AssetSensor
+ Job
+ Asset1
+ Asset2
+end
+```
+
+This is an example of an asset sensor that triggers a job when an asset is materialized. The `daily_sales_data` asset is in the same code location as the job and other asset for this example, but the same pattern can be applied to assets in different code locations.
+
+
+
+## Customizing the evaluation function of an asset sensor
+
+You can customize the evaluation function of an asset sensor to include specific logic for deciding when to trigger a run. This allows for fine-grained control over the conditions under which downstream jobs are executed.
+
+```mermaid
+stateDiagram-v2
+ direction LR
+
+ classDef userDefined fill: lightblue
+
+ [*] --> AssetMaterialization
+ AssetMaterialization --> [*]
+
+ AssetMaterialization --> UserEvaluationFunction:::userDefined
+ UserEvaluationFunction: User Evaluation Function
+
+ UserEvaluationFunction --> RunRequest
+ UserEvaluationFunction --> SkipReason
+ SkipReason --> [*]
+ RunRequest --> [*]
+
+ class UserEvaluationFunction userDefined
+ classDef userDefined fill: var(--theme-color-accent-lavendar)
+```
+
+In the following example, the `@asset_sensor` decorator defines a custom evaluation function that returns a `RunRequest` object when the asset is materialized and certain metadata is present, otherwise it skips the run.
+
+
+
+## Triggering a job with custom configuration
+
+By providing a configuration to the `RunRequest` object, you can trigger a job with a specific configuration. This is useful when you want to trigger a job with custom parameters based on custom logic you define.
+
+For example, you might use a sensor to trigger a job when an asset is materialized, but also pass metadata about that materialization to the job:
+
+
+
+## Monitoring multiple assets
+
+:::note
+
+The experimental `@multi_asset_sensor` has been marked as deprecated, but will not be removed from the codebase until Dagster 2.0 is released, meaning it will continue to function as it currently does for the foreseeable future. Its functionality has been largely superseded by the `AutomationCondition` system. For more information, see the [Declarative Automation documentation](/guides/automate/declarative-automation/).
+
+:::
+
+When building a pipeline, you may want to monitor multiple assets with a single sensor. This can be accomplished with a multi-asset sensor.
+
+The following example uses a `@multi_asset_sensor` to monitor multiple assets and trigger a job when any of the assets are materialized:
+
+
+
+## Next steps
+
+- Explore [Declarative Automation](/guides/automate/declarative-automation/) as an alternative to asset sensors
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/automate/declarative-automation/customizing-automation-conditions/arbitrary-python-automation-conditions.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/automate/declarative-automation/customizing-automation-conditions/arbitrary-python-automation-conditions.md
new file mode 100644
index 0000000000000..01c8cb558cd70
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/automate/declarative-automation/customizing-automation-conditions/arbitrary-python-automation-conditions.md
@@ -0,0 +1,57 @@
+---
+title: "Arbitrary Python automation conditions (Experimental)"
+sidebar_position: 400
+---
+
+Some automation use cases require custom business logic that cannot be expressed with off-the-shelf components. In these cases, you can define AutomationConditions which execute arbitrary python code, and compose them with the built-in conditions.
+
+## Setup
+
+By default, Dagster executes `AutomationConditionSensorDefinitions` in a daemon process that does not have access to your user code. In order to execute arbitrary Python code, you'll need to update this to execute on your user code server. This is the same place that your `@sensor` methods are evaluated.
+
+:::note
+
+Automation condition evaluation can be more resource-intensive than a typical sensor. A limit of 500 assets or checks per sensor is enforced.
+
+:::
+
+To do this, add an automation condition sensor to your definitions with the `use_user_code_server` flag set to `True`:
+
+```python
+import dagster as dg
+
+defs = dg.Definitions(
+ sensors=[dg.AutomationConditionSensorDefinition("automation_condition_sensor", target=dg.AssetSelection.all(), use_user_code_server=True)]
+)
+```
+
+This will allow your sensor to target automation conditions containing custom python code.
+
+## Defining a custom condition
+
+You can create your own subclass of `AutomationCondition`, defining the `evaluate()` method. For example, imagine you want to avoid executing anything on a company holiday. To do this, you can first define a condition which detects if it's currently a company holiday:
+
+```python
+import dagster as dg
+
+class IsCompanyHoliday(dg.AutomationCondition):
+ def evaluate(self, context: dg.AutomationContext) -> dg.AutomationResult:
+ if is_company_holiday(context.evaluation_time):
+ true_subset = context.candidate_subset
+ else:
+ true_subset = context.get_empty_subset()
+ return dg.AutomationResult(true_subset, context=context)
+
+```
+
+In this example, we build up a subset of the evaluated asset for which this condition is True. We use `EntitySubsets`, rather than a pure `True` / `False` to account for partitioned assets, for which individual partitions may have different results.
+
+In our case, the condition will be applied the same regardless of if it's partitioned or not, so we don't need to have any special logic to differntiate between these cases. If it's not a company holiday, we can return an empty subset (meaning that this condition is not true for any subset of the asset), and if it is a company holiday, we return the `candidate_subset`, which is the subset of the asset that we need to evaluate. This subset shrinks as we filter partitions out using the `&` condition, so if you have an expression `A & B`, and `A` returns the empty subset, then the candidate subset for `B` will be empty as well. This helps avoid expensive computation in cases where we know it won't impact the final output.
+
+Once this condition is defined, you can use this condition as part of a broader expression, for example:
+
+```python
+import dagster as dg
+
+condition = AutomationCondition.eager() & ~IsCompanyHoliday()
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/automate/declarative-automation/customizing-automation-conditions/automation-condition-operands-and-operators.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/automate/declarative-automation/customizing-automation-conditions/automation-condition-operands-and-operators.md
new file mode 100644
index 0000000000000..f91dcd51d4a9b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/automate/declarative-automation/customizing-automation-conditions/automation-condition-operands-and-operators.md
@@ -0,0 +1,154 @@
+---
+title: Automation conditions operands and operators
+sidebar_position: 100
+---
+
+## Operands
+
+Operands are base conditions which can be true or false about a given target. For partitioned assets, the target will be a given partition of the asset.
+
+| Operand | Description |
+| ------------------------------------------- | -------------------------------------------------------------------------------------------------------------------- |
+| `AutomationCondition.missing` | Target has not been executed |
+| `AutomationCondition.in_progress` | Target is part of an in-progress run |
+| `AutomationCondition.execution_failed` | Target failed to be executed in its latest run |
+| `AutomationCondition.newly_updated` | Target was updated since the previous evaluation |
+| `AutomationCondition.newly_requested` | Target was requested on the previous evaluation |
+| `AutomationCondition.code_version_changed` | Target has a new code version since the previous evaluation |
+| `AutomationCondition.cron_tick_passed` | A new tick of the provided cron schedule occurred since the previous evaluation |
+| `AutomationCondition.in_latest_time_window` | Target falls within the latest time window of the asset’s , if applicable. |
+| `AutomationCondition.will_be_requested` | Target will be requested in this tick |
+| `AutomationCondition.initial_evaluation` | This is the first evaluation of this condition |
+
+## Operators
+
+The [operands](#operands) can be built into more complex expressions using the following operators:
+
+
+
+
+
+ Operator
+
+
Description
+
+
+
+
+
+ ~ (tilde)
+
+
+ NOT; condition is not true; ex: ~A
+
+
+
+
+ | (pipe)
+
+
+ OR; either condition is true; ex: A | B
+
+
+
+
+ & (ampersand)
+
+
+ AND; both conditions are true; ex: A & B
+
+
+
+
+ A.newly_true()
+
+
Condition A was false on the previous evaluation and is now true.
+
+
+
+ A.since(B)
+
+
Condition A became true more recently than Condition B.
+
+
+
+ AutomationCondition.any_deps_match(A)
+
+
+ Condition A is true for any upstream partition. Can be used with .allow() and .ignore() to target specific upstream assets.
+
+
+
+
+ AutomationCondition.all_deps_match(A)
+
+
+ Condition A is true for at least one partition of each upstream asset.
+ Can be used with .allow() and .ignore() to
+ target specific upstream assets.
+
Similarly, you can also use to load assets from single Python files. |
+| `definitions.py` | The `definitions.py` file includes a object that contains all the definitions defined within your project. A definition can be an asset, a job, a schedule, a sensor, or a resource. This allows Dagster to load the definitions in a module.
To learn about other ways to deploy and load your Dagster code, see the [code locations documentation](/guides/deploy/code-locations/) |
+
+### my_dagster_project_tests/ directory
+
+A Python module that contains tests for `my_dagster_project`.
+
+### README.md file
+
+A description and starter guide for your Dagster project.
+
+### pyproject.toml file
+
+A file that specifies package core metadata in a static, tool-agnostic way.
+
+This file includes a `tool.dagster` section which references the Python module with your Dagster definitions defined and discoverable at the top level. This allows you to use the `dagster dev` command to load your Dagster code without any parameters. For more information. see the [code locations documentation](/guides/deploy/code-locations/).
+
+**Note:** `pyproject.toml` was introduced in [PEP-518](https://peps.python.org/pep-0518/) and meant to replace `setup.py`, but we may still include a `setup.py` for compatibility with tools that do not use this spec.
+
+### setup.py file
+
+A build script with Python package dependencies for your new project as a package. Use this file to specify dependencies.
+
+Note: If using Dagster+, add `dagster-cloud` as a dependency.
+
+### setup.cfg
+
+An ini file that contains option defaults for `setup.py` commands.
+
+## Configuration files
+
+Depending on your use case or if you're using Dagster+, you may also need to add additional configuration files to your project. Refer to the [Example project structures section](#example-project-structures) for a look at how these files might fit into your projects.
+
+| File/Directory | Description | OSS | Dagster+ |
+|----------------|-------------|-----|----------|
+| dagster.yaml | Configures your Dagster instance, including defining storage locations, run launchers, sensors, and schedules. For more information. including a list of use cases and available options, see the [`dagster.yaml`](/guides/deploy/dagster-yaml) reference.
For [Dagster+ Hybrid deployments](/dagster-plus/deployment/deployment-types/hybrid/), this file can be used to customize the [Hybrid agent](/dagster-plus/deployment/management/settings/customizing-agent-settings). | Optional | Optional |
+| dagster_cloud.yaml | Defines code locations for Dagster+. For more information, see the [`dagster_cloud.yaml` reference](/dagster-plus/deployment/code-locations/dagster-cloud-yaml). | Not applicable | Recommended |
+| deployment_settings.yaml | Configures settings for full deployments in Dagster+, including run queue priority and concurrency limits. Refer to the Deployment settings reference for more info.
**Note:** This file can be named anything, but we recommend choosing an easily understandable name. | Not applicable | Optional |
+| workspace.yaml | Defines multiple code locations for local development or deploying to your infrastructure. For more information and available options, see the [`workspace.yaml` file reference](/guides/deploy/code-locations/workspace-yaml) | Optional | Not applicable |
+
+
+## Example project structures
+
+Using the default project skeleton, let's take a look at how some example Dagster projects would be structured in different scenarios.
+
+:::note Configuration file location
+
+With the exception of [`dagster_cloud.yaml`](/dagster-plus/deployment/code-locations/dagster-cloud-yaml), it's not necessary for configuration files to be located with your project files. These files typically need to be located in `DAGSTER_HOME`. For example, in larger deployments, `DAGSTER_HOME` and Dagster infrastructure configuration can be managed separately from the code locations they support.
+
+:::
+
+### Local development
+
+
+
+
+For local development, a project with a single code location might look like this:
+
+```shell
+.
+├── README.md
+├── my_dagster_project
+│ ├── __init__.py
+│ ├── assets.py
+│ └── definitions.py
+├── my_dagster_project_tests
+├── dagster.yaml ## optional, used for instance settings
+├── pyproject.toml ## optional, used to define the project as a module
+├── setup.cfg
+├── setup.py
+└── tox.ini
+```
+
+
+
+
+For local development, a project with multiple code locations might look like this:
+
+```shell
+.
+├── README.md
+├── my_dagster_project
+│ ├── __init__.py
+│ ├── assets.py
+│ └── definitions.py
+├── my_dagster_project_tests
+├── dagster.yaml ## optional, used for instance settings
+├── pyproject.toml
+├── setup.cfg
+├── setup.py
+├── tox.ini
+└── workspace.yaml ## defines multiple code locations
+```
+
+
+
+
+### Dagster Open Source deployment
+
+Once you're ready to move from working locally to deploying Dagster to your infrastructure, use our [deployment guides](/guides/deploy/deployment-options/) to get up and running.
+
+A Dagster project deployed to your infrastructure might look like this:
+
+```shell
+.
+├── README.md
+├── my_dagster_project
+│ ├── __init__.py
+│ ├── assets.py
+│ └── definitions.py
+├── my_dagster_project_tests
+├── dagster.yaml ## optional, used for instance settings
+├── pyproject.toml
+├── setup.cfg
+├── setup.py
+├── tox.ini
+└── workspace.yaml ## defines multiple code locations
+```
+
+### Dagster+
+
+Depending on the type of deployment you're using in Dagster+ - Serverless or Hybrid - your project structure might look slightly different. Click the tabs for more info.
+
+
+
+
+#### Serverless deployment
+
+For a Dagster+ Serverless deployment, a project might look like this:
+
+```shell
+.
+├── README.md
+├── my_dagster_project
+│ ├── __init__.py
+│ ├── assets.py
+│ └── definitions.py
+├── my_dagster_project_tests
+├── dagster_cloud.yaml ## defines code locations
+├── deployment_settings.yaml ## optional, defines settings for full deployments
+├── pyproject.toml
+├── setup.cfg
+├── setup.py
+└── tox.ini
+```
+
+
+
+
+#### Hybrid deployment
+
+For a Dagster+ Hybrid deployment, a project might look like this:
+
+```shell
+.
+├── README.md
+├── my_dagster_project
+│ ├── __init__.py
+│ ├── assets.py
+│ └── definitions.py
+├── my_dagster_project_tests
+├── dagster.yaml ## optional, hybrid agent custom configuration
+├── dagster_cloud.yaml ## defines code locations
+├── deployment_settings.yaml ## optional, defines settings for full deployments
+├── pyproject.toml
+├── setup.cfg
+├── setup.py
+└── tox.ini
+```
+
+
+
+
+## Next steps
+
+You learned about the default files in a Dagster project and where they should be located, but what about the files containing your Dagster code?
+
+To sustainably scale your project, check out our best practices and recommendations in the [Structuring your project guide](structuring-your-dagster-project).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/build/projects/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/build/projects/index.md
new file mode 100644
index 0000000000000..574171d30bb74
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/build/projects/index.md
@@ -0,0 +1,8 @@
+---
+title: "Projects"
+sidebar_position: 20
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/build/projects/structuring-your-dagster-project.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/build/projects/structuring-your-dagster-project.md
new file mode 100644
index 0000000000000..8ba21a8b0d150
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/build/projects/structuring-your-dagster-project.md
@@ -0,0 +1,155 @@
+---
+title: "Structuring your Dagster project"
+sidebar_position: 200
+---
+
+:::note
+Refer to the project scaffolding tutorial to learn how to create a new Dagster project.
+:::
+
+There are many ways to structure your Dagster project, and it can be difficult to know where to start. In this guide, we will walk you through our recommendations for how to organize your Dagster project. As your project grows, you are welcome to deviate from these recommendations.
+
+## Your initial project structure
+
+When you first scaffold your project using the Dagster command-line tool, an `assets.py` and `definitions.py` are created in the root of your project.
+
+```sh
+$ dagster project scaffold --name example-dagster-project
+```
+
+```
+.
+├── README.md
+├── example_dagster_project
+│ ├── __init__.py
+│ ├── assets.py
+│ └── definitions.py
+├── example_dagster_project_tests
+│ ├── __init__.py
+│ └── test_assets.py
+├── pyproject.toml
+├── setup.cfg
+└── setup.py
+```
+
+This is a great structure as you are first getting started, however, as you begin to introduce more assets, jobs, resources, sensors, and utility code, you may find that your Python files are growing too large to manage.
+
+## Restructure your project
+
+There are several paradigms in which you can structure your project. Choosing one of these structures is often personal preference, and influenced by how you and your team members operate. This guide will outline three possible project structures:
+
+1. [Option 1: Structured by technology](#option-1-structured-by-technology)
+2. [Option 2: Structured by concept](#option-2-structured-by-concept)
+
+
+### Option 1: Structured by technology
+
+Data engineers often have a strong understanding of the underlying technologies that are used in their data pipelines. Because of that, it's often beneficial to organize your project by technology. This enables engineers to easily navigate the code base and locate files pertaining to the specific technology.
+
+Within the technology modules, sub-modules can be created to further organize your code.
+
+```
+.
+└── example_dagster_project/
+ ├── dbt/
+ │ ├── __init__.py
+ │ ├── assets.py
+ │ ├── resources.py
+ │ └── definitions.py
+ ├── dlt/
+ │ ├── __init__.py
+ │ ├── pipelines/
+ │ │ ├── __init__.py
+ │ │ ├── github.py
+ │ │ └── hubspot.py
+ │ ├── assets.py
+ │ ├── resources.py
+ │ └── definitions.py
+ └── definitions.py
+```
+
+### Option 2: Structured by concept
+
+It's also possible to introduce a layer of categorization by the overarching data processing concept. For example, whether the job is performing some kind of transformation, ingestion of data, or processing operation.
+
+This provides additional context to the engineers who may not have as strong of a familiarity with the underlying technologies that are being used.
+
+```
+.
+└── example_dagster_project/
+ ├── ingestion/
+ │ └── dlt/
+ │ ├── assets.py
+ │ ├── resources.py
+ │ └── definitions.py
+ ├── transformation/
+ │ ├── dbt/
+ │ │ ├── assets.py
+ │ │ ├── resources.py
+ │ │ └── partitions.py
+ │ │ └── definitions.py
+ │ └── adhoc/
+ │ ├── assets.py
+ │ ├── resources.py
+ │ └── definitions.py
+ └── definitions.py
+```
+
+## Merging definitions objects
+
+It's possible to define multiple `Definitions` objects, often with one for each sub-module in your project. These definitions can then be merged at the root of your project using the `Definitions.merge` method.
+
+The benefit of such a structure is that dependencies like resources and partitions can be scoped to their corresponding definitions.
+
+```py title="example-merge-definitions.py"
+from dbt.definitions import dbt_definitions
+from dlt.definitions import dlt_definitions
+
+
+defs = Definitions.merge(
+ dbt_definitions,
+ dlt_definitions,
+)
+```
+
+## Configuring multiple code locations
+
+This guide has outlined how to structure a project within a single code location, however, Dagster also allows you to structure a project spanning multiple location.
+
+In most cases, one code location should be sufficient. A helpful pattern uses multiple code locations to separate conflicting dependencies, where each definition has its own package requirements and deployment specs.
+
+To include multiple code locations in a single project, you'll need to add a configuration file to your project:
+
+- **If using Dagster+**, add a `dagster_cloud.yaml` file to the root of your project.
+- **If developing locally or deploying to your infrastructure**, add a workspace.yaml file to the root of your project.
+
+## External projects
+
+As your data platform evolves, Dagster will enable you to orchestrate other data tools, such as dbt, Sling, or Jupyter notebooks.
+
+For these projects, it's recommended to store them outside your Dagster project. See the `dbt_project` example below.
+
+```
+.
+├── dbt_project/
+│ ├── config/
+│ │ └── profiles.yml
+│ ├── dbt_project.yml
+│ ├── macros/
+│ │ ├── aggregate_actions.sql
+│ │ └── generate_schema_name.sql
+│ ├── models/
+│ │ ├── activity_analytics/
+│ │ │ ├── activity_daily_stats.sql
+│ │ │ ├── comment_daily_stats.sql
+│ │ │ └── story_daily_stats.sql
+│ │ ├── schema.yml
+│ │ └── sources.yml
+│ └── tests/
+│ └── assert_true.sql
+└── example_dagster_project/
+```
+
+## Next steps
+
+- Explore the API docs
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/code-locations/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/code-locations/index.md
new file mode 100644
index 0000000000000..7951c9ba3dbdf
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/code-locations/index.md
@@ -0,0 +1,8 @@
+---
+title: Code locations
+sidebar_position: 30
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/code-locations/managing-code-locations-with-definitions.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/code-locations/managing-code-locations-with-definitions.md
new file mode 100644
index 0000000000000..0e72698a3821f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/code-locations/managing-code-locations-with-definitions.md
@@ -0,0 +1,144 @@
+---
+title: "Managing code locations with Definitions"
+description: "A code location is a collection of Dagster definitions loadable and accessible by Dagster's tools. Learn to create, load, and deploy code locations."
+sidebar_position: 100
+---
+
+A code location is a collection of Dagster definitions loadable and accessible by Dagster's tools, such as the CLI, UI, and Dagster+. A code location comprises:
+
+- A reference to a Python module that has an instance of in a top-level variable
+- A Python environment that can successfully load that module
+
+Definitions within a code location have a common namespace and must have unique names. This allows them to be grouped and organized by code location in tools.
+
+![Code locations](/images/guides/deploy/code-locations/code-locations-diagram.png)
+
+A single deployment can have one or multiple code locations.
+
+Code locations are loaded in a different process and communicate with Dagster system processes over an RPC mechanism. This architecture provides several advantages:
+
+- When there is an update to user code, the Dagster webserver/UI can pick up the change without a restart.
+- You can use multiple code locations to organize jobs, but still work on all of your code locations using a single instance of the webserver/UI.
+- The Dagster webserver process can run in a separate Python environment from user code so job dependencies don't need to be installed into the webserver environment.
+- Each code location can be sourced from a separate Python environment, so teams can manage their dependencies (or even their Python versions) separately.
+
+## Relevant APIs
+
+| Name | Description |
+| --------------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------- |
+| | The object that contains all the definitions defined within a code location. Definitions include assets, jobs, resources, schedules, and sensors. |
+
+## Defining code locations
+
+To define a code location, create a top-level variable that contains a object in a Python module. For example:
+
+```python
+# definitions.py
+
+defs = Definitions(
+ assets=[dbt_customers_asset, dbt_orders_asset],
+ schedules=[bi_weekly_schedule],
+ sensors=[new_data_sensor],
+ resources=[dbt_resource],
+)
+```
+
+It is recommenced to include definitions in a Python module named `definitions.py`.
+
+
+## Deploying and loading code locations
+
+- [Local development](#local-development)
+- [Dagster+ deployment](#dagster-deployment)
+- [Open source deployment](#open-source-deployment)
+
+### Local development
+
+
+
+
+Dagster can load a file directly as a code location. In the following example, we used the `-f` argument to supply the name of the file:
+
+```shell
+dagster dev -f my_file.py
+```
+
+This command loads the definitions in `my_file.py` as a code location in the current Python environment.
+
+You can also include multiple files at a time, where each file will be loaded as a code location:
+
+```shell
+dagster dev -f my_file.py -f my_second_file.py
+```
+
+
+
+
+Dagster can also load Python modules as [code locations](/guides/deploy/code-locations/). When this approach is used, Dagster loads the definitions defined in the module passed to the command line.
+
+We recommend defining a variable containing the object in a submodule named `definitions` inside the Python module. In practice, the submodule can be created by adding a file named `definitions.py` at the root level of the Python module.
+
+As this style of development eliminates an entire class of Python import errors, we strongly recommend it for Dagster projects deployed to production.
+
+In the following example, we used the `-m` argument to supply the name of the module and where to find the definitions:
+
+```shell
+dagster dev -m your_module_name.definitions
+```
+
+This command loads the definitions in the variable containing the object in the `definitions` submodule in the current Python environment.
+
+You can also include multiple modules at a time, where each module will be loaded as a code location:
+
+```shell
+dagster dev -m your_module_name.definitions -m your_second_module.definitions
+```
+
+
+
+
+To load definitions without supplying command line arguments, you can use the `pyproject.toml` file. This file, included in all Dagster example projects, contains a `tool.dagster` section with a `module_name` variable:
+
+```toml
+[tool.dagster]
+module_name = "your_module_name.definitions" ## name of project's Python module and where to find the definitions
+code_location_name = "your_code_location_name" ## optional, name of code location to display in the Dagster UI
+```
+
+When defined, you can run this in the same directory as the `pyproject.toml` file:
+
+```shell
+dagster dev
+```
+
+Instead of this:
+
+```shell
+dagster dev -m your_module_name.definitions
+```
+
+You can also include multiple modules at a time using the `pyproject.toml` file, where each module will be loaded as a code location:
+
+```toml
+[tool.dagster]
+modules = [{ type = "module", name = "foo" }, { type = "module", name = "bar" }]
+```
+
+
+
+
+Fore more information about local development, including how to configure your local instance, see "[Running Dagster locally](/guides/deploy/deployment-options/running-dagster-locally)".
+
+### Dagster+ deployment
+
+See the [Dagster+ code locations documentation](/dagster-plus/deployment/code-locations/).
+
+### Open source deployment
+
+The `workspace.yaml` file is used to load code locations for open source (OSS) deployments. This file specifies how to load a collection of code locations and is typically used in advanced use cases. For more information, see "[workspace.yaml reference](workspace-yaml.md)".
+
+## Troubleshooting
+
+| Error | Description and resolution |
+|-------|----------------------------|
+| Cannot have more than one Definitions object defined at module scope | Dagster found multiple objects in a single Python module. Only one object may be in a single code location. |
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/code-locations/workspace-yaml.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/code-locations/workspace-yaml.md
new file mode 100644
index 0000000000000..d0d1db821e783
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/code-locations/workspace-yaml.md
@@ -0,0 +1,119 @@
+---
+title: "workspace.yaml reference"
+sidebar_position: 200
+---
+
+:::info
+ This reference is only applicable to Dagster OSS. For Dagster Cloud see [the Dagster Cloud Code Locations documentation](/dagster-plus/deployment/code-locations)
+:::
+
+The `workspace.yaml` file is used to configure code locations in Dagster. It tells Dagster where to find your code and how to load it.
+
+## Location of workspace.yaml
+
+Dagster command-line tools (like `dagster dev`, `dagster-webserver`, or `dagster-daemon run`) look for workspace files in the current directory when invoked. This allows you to launch from that directory without the need for command line arguments
+
+To load the workspace.yaml file from a different folder, use the -w argument:
+
+```bash
+dagster dev -w path/to/workspace.yaml
+```
+
+## File structure
+
+The `workspace.yaml` file uses the following structure:
+
+```yaml
+load_from:
+ - :
+
+```
+
+Where `` can be one of:
+- `python_file`
+- `python_module`
+- `grpc_server`
+
+
+
+## Loading methods
+
+We recommend using `python_module` for most use cases.
+
+### Python module
+
+Loads a code location from a Python module.
+
+**Options:**
+- `module_name`: Name of the Python module to load.
+- Other options are the same as `python_file`.
+
+**Example:**
+```yaml
+load_from:
+ - python_module:
+ module_name: hello_world_module.definitions
+```
+
+
+### Python file
+
+Loads a code location from a Python file.
+
+**Options:**
+- `relative_path`: Path to the Python file, relative to the `workspace.yaml` location.
+- `attribute` (optional): Name of a specific repository or function returning a `RepositoryDefinition`.
+- `working_directory` (optional): Custom working directory for relative imports.
+- `executable_path` (optional): Path to a specific Python executable.
+- `location_name` (optional): Custom name for the code location.
+
+**Example:**
+```yaml
+load_from:
+ - python_file:
+ relative_path: hello_world_repository.py
+ attribute: hello_world_repository
+ working_directory: my_working_directory/
+ executable_path: venvs/path/to/python
+ location_name: my_location
+```
+
+### gRPC server
+
+Configures a gRPC server as a code location.
+
+**Options:**
+- `host`: The host address of the gRPC server.
+- `port`: The port number of the gRPC server.
+- `location_name`: Custom name for the code location.
+
+**Example:**
+```yaml
+load_from:
+ - grpc_server:
+ host: localhost
+ port: 4266
+ location_name: "my_grpc_server"
+```
+
+## Multiple code locations
+
+You can define multiple code locations in a single `workspace.yaml` file:
+
+```yaml
+load_from:
+ - python_file:
+ relative_path: path/to/dataengineering_spark_team.py
+ location_name: dataengineering_spark_team_py_38_virtual_env
+ executable_path: venvs/path/to/dataengineering_spark_team/bin/python
+ - python_file:
+ relative_path: path/to/team_code_location.py
+ location_name: ml_team_py_36_virtual_env
+ executable_path: venvs/path/to/ml_tensorflow/bin/python
+```
+
+## Notes
+
+- Each code location is loaded in its own process.
+- Code locations can mix and match between `Definitions` objects and `@repository` decorators.
+- If a code location is renamed or its configuration is modified, running schedules and sensors in that location need to be stopped and restarted.
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/dagster-instance-configuration.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/dagster-instance-configuration.md
new file mode 100644
index 0000000000000..7d676367dd1ef
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/dagster-instance-configuration.md
@@ -0,0 +1,642 @@
+---
+title: "Dagster instance | Dagster Docs"
+description: "Define configuration options for your Dagster instance."
+sidebar_position: 200
+---
+
+:::note
+
+This article applies to Dagster Open Source (OSS) deployments. For information on Dagster+, see the [Dagster+ documentation](/dagster-plus/deployment/management/settings/customizing-agent-settings).
+
+:::
+
+The Dagster instance defines the configuration that Dagster needs for a single deployment - for example, where to store the history of past runs and their associated logs, where to stream the raw logs from op compute functions, and how to launch new runs.
+
+All of the processes and services that make up your Dagster deployment should share a single instance config file, named `dagster.yaml`, so that they can effectively share information.
+
+:::warning
+
+Some important configuration, like [execution parallelism](/guides/operate/run-executors), is set on a per-job basis rather than on the instance.
+
+:::
+
+## Default local behavior
+
+When a Dagster process like the Dagster webserver or Dagster CLI commands are launched, Dagster tries to load your instance. If the environment variable `DAGSTER_HOME` is set, Dagster looks for an instance config file at `$DAGSTER_HOME/dagster.yaml`. This file contains the configuration settings that make up the instance.
+
+If `DAGSTER_HOME` isn't set, Dagster tools will use a temporary directory for storage that is cleaned up when the process exits. This can be useful when using Dagster for temporary local development or testing, when you don't care about the results being persisted.
+
+If `DAGSTER_HOME` is set but `dagster.yaml` isn't present or is empty, Dagster will persist data on the local filesystem, structured like the following:
+
+ $DAGSTER_HOME
+ ├── dagster.yaml
+ ├── history
+ │ ├── runs
+ │ │ ├── 00636713-98a9-461c-a9ac-d049407059cd.db
+ │ │ └── ...
+ │ └── runs.db
+ └── storage
+ ├── 00636713-98a9-461c-a9ac-d049407059cd
+ │ └── compute_logs
+ │ └── ...
+ └── ...
+
+Here's a breakdown of the files and directories that are generated:
+
+| File or directory | Description |
+|-------------------|-------------|
+| history/ | A directory containing historical information for runs. |
+| history/runs.db | SQLite database file that contains information about runs. |
+| history/[run_id].db | SQLite database file that contains per-run event logs. |
+| storage/ | A directory of subdirectories, one for each run. |
+| storage/[run_id]/compute_logs | A directory specific to the run that contains the `stdout` and `stderr` logs from the execution of the compute functions of each op. |
+
+## Configuration reference
+
+In persistent Dagster deployments, you'll typically want to configure many of the components on the instance. For example, you may want to use a Postgres instance to store runs and the corresponding event logs, but stream compute logs to an Amazon S3 bucket.
+
+To do this, provide a `$DAGSTER_HOME/dagster.yaml` file, which the webserver and all other Dagster tools will look for on startup. In this file, you can configure different aspects of your Dagster instance, including:
+
+| Name | Key | Description |
+|------------------------|---------------------------|--------------|
+| Dagster storage | `storage` | Controls how job and asset history is persisted. This includes run, event log, and schedule/sensor tick metadata, as well as other useful data. |
+| Run launcher | `run_launcher` | Determines where runs are executed. |
+| Run coordinator | `run_coordinator` | Determines the policy used to set prioritization rules and concurrency limits for runs. |
+| Compute log storage | `compute_logs` | Controls the capture and persistence of raw stdout and{" "} stderr ext logs. |
+| Local artifact storage| `local_artifact_storage` | Configures storage for artifacts that require a local disk or when using the filesystem I/O manager ( ). |
+| Telemetry | `telemetry` | Used to opt in/out of Dagster collecting anonymized usage statistics. |
+| gRPC servers | `code_servers` | Configures how Dagster loads the code in a code location. |
+| Data retention | `data_retention` | Controls how long Dagster retains certain types of data that have diminishing value over time, such as schedule/sensor tick data. |
+| Sensor evaluation | `sensors` | Controls how sensors are evaluated. |
+| Schedule evaluation | `schedules` | Controls how schedules are evaluated. |
+| Auto-materialize | `auto_materialize` | Controls how assets are auto-materialized.|
+
+:::note
+
+Environment variables in YAML configuration are supported by using an `env:` key instead of a literal string value. Sample configurations in this reference include examples using environment variables.
+
+:::
+
+### Dagster storage
+
+The `storage` key allows you to configure how job and asset history is persisted. This includes metadata on runs, event logs, schedule/sensor ticks, and other useful data.
+
+Refer to the following tabs for available options and sample configuration.
+
+
+
+
+**SQLite storage (default)**
+
+To use a SQLite database for storage, configure `storage.sqlite` in `dagster.yaml`:
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_storage_sqlite endbefore=end_marker_storage_sqlite
+# there are two ways to set storage to SqliteStorage
+
+# this config manually sets the directory (`base_dir`) for Sqlite to store data in:
+storage:
+ sqlite:
+ base_dir: /path/to/dir
+
+# and this config grabs the directory from an environment variable
+storage:
+ sqlite:
+ base_dir:
+ env: SQLITE_STORAGE_BASE_DIR
+```
+
+
+
+
+**Postgres storage**
+
+:::note
+
+To use Postgres storage, you'll need to install the [dagster-postgres](/api/python-api/libraries/dagster-postgres) library.
+
+:::
+
+To use a [PostgreSQL database](/api/python-api/libraries/dagster-postgres) for storage, configure `storage.postgres` in `dagster.yaml`:
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_storage_postgres endbefore=end_marker_storage_postgres
+# Postgres storage can be set using either credentials or a connection string. This requires that
+# the `dagster-postgres` library be installed and a database configured with UTC timezone.
+
+# this config manually sets the Postgres credentials
+storage:
+ postgres:
+ postgres_db:
+ username: { DAGSTER_PG_USERNAME }
+ password: { DAGSTER_PG_PASSWORD }
+ hostname: { DAGSTER_PG_HOSTNAME }
+ db_name: { DAGSTER_PG_DB }
+ port: 5432
+
+# and this config grabs the database credentials from environment variables
+storage:
+ postgres:
+ postgres_db:
+ username:
+ env: DAGSTER_PG_USERNAME
+ password:
+ env: DAGSTER_PG_PASSWORD
+ hostname:
+ env: DAGSTER_PG_HOST
+ db_name:
+ env: DAGSTER_PG_DB
+ port: 5432
+
+# and this config sets the credentials via DB connection string / url:
+storage:
+ postgres:
+ postgres_url: { PG_DB_CONN_STRING }
+
+# This config gets the DB connection string / url via environment variables:
+storage:
+ postgres:
+ postgres_url:
+ env: PG_DB_CONN_STRING
+```
+
+
+
+
+**MySQL storage**
+
+:::note
+
+To use MySQL storage, you'll need to install the [dagster-mysql](/api/python-api/libraries/dagster-mysql) library.
+
+:::
+
+To use a [MySQL database](/api/python-api/libraries/dagster-mysql) for storage, configure `storage.mysql` in `dagster.yaml`:
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_storage_mysql endbefore=end_marker_storage_mysql
+# MySQL storage can be set using either credentials or a connection string. This requires that the
+# `dagster-mysql` library be installed.
+
+# this config manually sets the MySQL credentials
+storage:
+ mysql:
+ mysql_db:
+ username: { DAGSTER_MYSQL_USERNAME }
+ password: { DAGSTER_MYSQL_PASSWORD }
+ hostname: { DAGSTER_MYSQL_HOSTNAME }
+ db_name: { DAGSTER_MYSQL_DB }
+ port: 3306
+
+
+# and this config grabs the database credentials from environment variables
+storage:
+ mysql:
+ mysql_db:
+ username:
+ env: DAGSTER_MYSQL_USERNAME
+ password:
+ env: DAGSTER_MYSQL_PASSWORD
+ hostname:
+ env: DAGSTER_MYSQL_HOSTNAME
+ db_name:
+ env: DAGSTER_MYSQL_DB
+ port: 3306
+
+# and this config sets the credentials via DB connection string / url:
+storage:
+ mysql:
+ mysql_url: { MYSQL_DB_CONN_STRING }
+
+# this config grabs the MySQL connection string from environment variables
+storage:
+ mysql:
+ mysql_url:
+ env: MYSQL_DB_CONN_STRING
+```
+
+
+
+
+### Run launcher
+
+The `run_launcher` key allows you to configure the run launcher for your instance. Run launchers determine where runs are executed. You can use one of the Dagster-provided options or write your own custom run launcher. For more information, see "[Run launchers](/guides/deploy/execution/run-launchers)".
+
+Refer to the following tabs for available options and sample configuration. Keep in mind that databases should be configured to use UTC timezone.
+
+
+
+
+**DefaultRunLauncher**
+
+The spawns a new process in the same node as a job's code location.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_run_launcher_default endbefore=end_marker_run_launcher_default
+run_launcher:
+ module: dagster.core.launcher
+ class: DefaultRunLauncher
+```
+
+
+
+
+**DockerRunLauncher**
+
+The allocates a Docker container per run.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_run_launcher_docker endbefore=end_marker_run_launcher_docker
+run_launcher:
+ module: dagster_docker
+ class: DockerRunLauncher
+```
+
+
+
+
+**K8sRunLauncher**
+
+The allocates a Kubernetes job per run.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_run_launcher_k8s endbefore=end_marker_run_launcher_k8s
+# there are multiple ways to configure the K8sRunLauncher
+
+# you can set the follow configuration values directly
+run_launcher:
+ module: dagster_k8s.launcher
+ class: K8sRunLauncher
+ config:
+ service_account_name: pipeline_run_service_account
+ job_image: my_project/dagster_image:latest
+ instance_config_map: dagster-instance
+ postgres_password_secret: dagster-postgresql-secret
+
+# alternatively, you can grab any of these config values from environment variables:
+run_launcher:
+ module: dagster_k8s.launcher
+ class: K8sRunLauncher
+ config:
+ service_account_name:
+ env: PIPELINE_RUN_SERVICE_ACCOUNT
+ job_image:
+ env: DAGSTER_IMAGE_NAME
+ instance_config_map:
+ env: DAGSTER_INSTANCE_CONFIG_MAP
+ postgres_password_secret:
+ env: DAGSTER_POSTGRES_SECRET
+```
+
+
+
+
+### Run coordinator
+
+The `run_coordinator` key allows you to configure the run coordinator for your instance. Run coordinators determine the policy used to set the prioritization rules and concurrency limits for runs. For more information and troubleshooting help, see "[Run coordinators](/guides/deploy/execution/run-coordinators)".
+
+Refer to the following tabs for available options and sample configuration.
+
+
+
+
+**DefaultRunCoordinator (default)**
+
+The default run coordinator, the immediately sends runs to the [run launcher](#run-launcher). There isn't a notion of `Queued` runs.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_run_coordinator_default endbefore=end_marker_run_coordinator_default
+# Since DefaultRunCoordinator is the default option, omitting the `run_coordinator` key will also suffice,
+# but if you would like to set it explicitly:
+run_coordinator:
+ module: dagster.core.run_coordinator
+ class: DefaultRunCoordinator
+```
+
+
+
+
+**QueuedRunCoordinator**
+
+The allows you to set limits on the number of runs that can be executed at once. **Note** This requires an active [dagster-daemon process](/guides/deploy/execution/dagster-daemon) to launch the runs.
+
+This run coordinator supports both limiting the overall number of concurrent runs and specific limits based on run tags. For example, to avoid throttling, you can specify a concurrency limit for runs that interact with a specific cloud service.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_run_coordinator_queued endbefore=end_marker_run_coordinator_queued
+# There are a few ways to configure the QueuedRunCoordinator:
+
+# this first option has concurrency limits set to default values
+run_coordinator:
+ module: dagster.core.run_coordinator
+ class: QueuedRunCoordinator
+
+# this second option manually specifies limits:
+run_coordinator:
+ module: dagster.core.run_coordinator
+ class: QueuedRunCoordinator
+ config:
+ max_concurrent_runs: 25
+ tag_concurrency_limits:
+ - key: "database"
+ value: "redshift"
+ limit: 4
+ - key: "dagster/backfill"
+ limit: 10
+
+# as always, some or all of these values can be obtained from environment variables:
+run_coordinator:
+ module: dagster.core.run_coordinator
+ class: QueuedRunCoordinator
+ config:
+ max_concurrent_runs:
+ env: DAGSTER_OVERALL_CONCURRENCY_LIMIT
+ tag_concurrency_limits:
+ - key: "database"
+ value: "redshift"
+ limit:
+ env: DAGSTER_REDSHIFT_CONCURRENCY_LIMIT
+ - key: "dagster/backfill"
+ limit:
+ env: DAGSTER_BACKFILL_CONCURRENCY_LIMIT
+
+# for higher dequeue throughput, threading can be enabled:
+run_coordinator:
+ module: dagster.core.run_coordinator
+ class: QueuedRunCoordinator
+ config:
+ dequeue_use_threads: true
+ dequeue_num_workers: 8
+```
+
+
+
+
+### Compute log storage
+
+The `compute_logs` key allows you to configure compute log storage. Compute log storage controls the capture and persistence of raw `stdout` and `stderr` text logs.
+
+Refer to the following tabs for available options and sample configuration.
+
+
+
+
+**LocalComputeLogManager**
+
+Used by default, the writes `stdout` and `stderr` logs to disk.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_compute_log_storage_local endbefore=end_marker_compute_log_storage_local
+# there are two ways to set the directory that the LocalComputeLogManager writes
+# stdout & stderr logs to
+
+# You could directly set the `base_dir` key
+compute_logs:
+ module: dagster.core.storage.local_compute_log_manager
+ class: LocalComputeLogManager
+ config:
+ base_dir: /path/to/directory
+
+# Alternatively, you could set the `base_dir` key to an environment variable
+compute_logs:
+ module: dagster.core.storage.local_compute_log_manager
+ class: LocalComputeLogManager
+ config:
+ base_dir:
+ env: LOCAL_COMPUTE_LOG_MANAGER_DIRECTORY
+```
+
+
+
+
+**NoOpComputeLogManager**
+
+The does not store `stdout` and `stderr` logs for any step.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_compute_log_storage_noop endbefore=end_marker_compute_log_storage_noop
+compute_logs:
+ module: dagster.core.storage.noop_compute_log_manager
+ class: NoOpComputeLogManager
+```
+
+
+
+
+**AzureBlobComputeLogManager**
+
+The writes `stdout` and `stderr` to Azure Blob Storage.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_compute_log_storage_blob endbefore=end_marker_compute_log_storage_blob
+# there are multiple ways to configure the AzureBlobComputeLogManager
+
+# you can set the necessary configuration values directly:
+compute_logs:
+ module: dagster_azure.blob.compute_log_manager
+ class: AzureBlobComputeLogManager
+ config:
+ storage_account: mycorp-dagster
+ container: compute-logs
+ secret_credential:
+ client_id: ...
+ tenant_id: ...
+ client_secret: ...
+ local_dir: /tmp/bar
+ prefix: dagster-test-
+
+# alternatively, you can obtain any of these config values from environment variables
+compute_logs:
+ module: dagster_azure.blob.compute_log_manager
+ class: AzureBlobComputeLogManager
+ config:
+ storage_account:
+ env: MYCORP_DAGSTER_STORAGE_ACCOUNT_NAME
+ container:
+ env: CONTAINER_NAME
+ secret_credential:
+ client_id: ...
+ tenant_id: ...
+ client_secret: ...
+ local_dir:
+ env: LOCAL_DIR_PATH
+ prefix:
+ env: DAGSTER_COMPUTE_LOG_PREFIX
+```
+
+
+
+
+**GCSComputeLogManager**
+
+The writes `stdout` and `stderr` to Google Cloud Storage.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_compute_log_storage_gcs endbefore=end_marker_compute_log_storage_gcs
+# there are multiple ways to configure the GCSComputeLogManager
+
+# you can set the necessary configuration values directly:
+compute_logs:
+ module: dagster_gcp.gcs.compute_log_manager
+ class: GCSComputeLogManager
+ config:
+ bucket: mycorp-dagster-compute-logs
+ prefix: dagster-test-
+
+# alternatively, you can obtain any of these config values from environment variables
+compute_logs:
+ module: dagster_gcp.gcs.compute_log_manager
+ class: GCSComputeLogManager
+ config:
+ bucket:
+ env: MYCORP_DAGSTER_COMPUTE_LOGS_BUCKET
+ prefix:
+ env: DAGSTER_COMPUTE_LOG_PREFIX
+```
+
+
+
+
+**S3ComputeLogManager**
+
+The writes `stdout` and `stderr` to an Amazon Web Services S3 bucket.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_compute_log_storage_s3 endbefore=end_marker_compute_log_storage_s3
+# there are multiple ways to configure the S3ComputeLogManager
+
+# you can set the config values directly:
+compute_logs:
+ module: dagster_aws.s3.compute_log_manager
+ class: S3ComputeLogManager
+ config:
+ bucket: "mycorp-dagster-compute-logs"
+ prefix: "dagster-test-"
+
+# or grab some or all of them from environment variables
+compute_logs:
+ module: dagster_aws.s3.compute_log_manager
+ class: S3ComputeLogManager
+ config:
+ bucket:
+ env: MYCORP_DAGSTER_COMPUTE_LOGS_BUCKET
+ prefix:
+ env: DAGSTER_COMPUTE_LOG_PREFIX
+```
+
+
+
+
+### Local artifact storage
+
+The `local_artifact_storage` key allows you to configure local artifact storage. Local artifact storage is used to:
+
+- Configure storage for artifacts that require a local disk, or
+- Store inputs and outputs when using the filesystem I/O manager (). For more information on how other I/O managers store artifacts, see the [I/O managers documentation](/guides/build/io-managers/).
+
+:::note
+
+ is currently the only option for local artifact storage. This option configures the directory used by the default filesystem I/O Manager, as well as any artifacts that require a local disk.
+
+:::
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_local_artifact_storage endbefore=end_marker_local_artifact_storage
+# there are two possible ways to configure LocalArtifactStorage
+
+# example local_artifact_storage setup pointing to /var/shared/dagster directory
+local_artifact_storage:
+ module: dagster.core.storage.root
+ class: LocalArtifactStorage
+ config:
+ base_dir: "/path/to/dir"
+
+# alternatively, `base_dir` can be set to an environment variable
+local_artifact_storage:
+ module: dagster.core.storage.root
+ class: LocalArtifactStorage
+ config:
+ base_dir:
+ env: DAGSTER_LOCAL_ARTIFACT_STORAGE_DIR
+```
+
+### Telemetry
+
+The `telemetry` key allows you to opt in or out of Dagster collecting anonymized usage statistics. This is set to `true` by default.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_telemetry endbefore=end_marker_telemetry
+# Allows opting out of Dagster collecting usage statistics.
+telemetry:
+ enabled: false
+```
+
+For more information, see the [Telemetry documentation](/about/telemetry).
+
+### gRPC servers
+
+The `code_servers` key allows you to configure how Dagster loads the code in a [code location](/guides/deploy/code-locations/).
+
+When you aren't [running your own gRPC server](/guides/deploy/code-locations/workspace-yaml#grpc-server), the webserver and the Dagster daemon load your code from a gRPC server running in a subprocess. By default, if your code takes more than 180 seconds to load, Dagster assumes that it's hanging and stops waiting for it to load.
+
+If you expect that your code will take longer than 180 seconds to load, set the `code_servers.local_startup_timeout` key. The value should be an integer that indicates the maximum timeout, in seconds.
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_code_servers endbefore=end_marker_code_servers
+# Configures how long Dagster waits for code locations
+# to load before timing out.
+code_servers:
+ local_startup_timeout: 360
+```
+
+### Data retention
+
+The `retention` key allows you to configure how long Dagster retains certain types of data. Specifically, data that has diminishing value over time, such as schedule/sensor tick data. Cleaning up old ticks can help minimize storage concerns and improve query performance.
+
+By default, Dagster retains skipped sensor ticks for seven days and all other tick types indefinitely. To customize the retention policies for schedule and sensor ticks, use the `purge_after_days` key:
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_retention endbefore=end_marker_retention
+# Configures how long Dagster keeps sensor / schedule tick data
+retention:
+ schedule:
+ purge_after_days: 90 # sets retention policy for schedule ticks of all types
+ sensor:
+ purge_after_days:
+ skipped: 7
+ failure: 30
+ success: -1 # keep success ticks indefinitely
+```
+
+The `purge_after_days` key accepts either:
+
+- A single integer that indicates how long, in days, to retain ticks of all types. **Note**: A value of `-1` retains ticks indefinitely.
+- A mapping of tick types (`skipped`, `failure`, `success`) to integers. The integers indicate how long, in days, to retain the tick type.
+
+### Sensor evaluation
+
+The `sensors` key allows you to configure how sensors are evaluated. To evaluate multiple sensors in parallel simultaneously, set the `use_threads` and `num_workers` keys:
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_sensors endbefore=end_marker_sensors
+sensors:
+ use_threads: true
+ num_workers: 8
+```
+
+You can also set the optional `num_submit_workers` key to evaluate multiple run requests from the same sensor tick in parallel, which can help decrease latency when a single sensor tick returns many run requests.
+
+### Schedule evaluation
+
+The `schedules` key allows you to configure how schedules are evaluated. By default, Dagster evaluates schedules one at a time.
+
+To evaluate multiple schedules in parallel simultaneously, set the `use_threads` and `num_workers` keys:
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_schedules endbefore=end_marker_schedules
+schedules:
+ use_threads: true
+ num_workers: 8
+```
+
+You can also set the optional `num_submit_workers` key to evaluate multiple run requests from the same schedule tick in parallel, which can help decrease latency when a single schedule tick returns many run requests.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/dagster-yaml.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/dagster-yaml.md
new file mode 100644
index 0000000000000..fa1f20f9bbd77
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/dagster-yaml.md
@@ -0,0 +1,344 @@
+---
+title: 'dagster.yaml reference'
+sidebar_position: 300
+---
+
+The `dagster.yaml` file is used to configure the Dagster instance. It defines various settings for storage, run execution, logging, and other aspects of a Dagster deployment.
+
+## File location
+
+By default, Dagster looks for the `dagster.yaml` file in the directory specified by the `DAGSTER_HOME` environment variable.
+
+## Using environment variables
+
+You can use environment variables to override values in the `dagster.yaml` file.
+
+```yaml
+instance:
+ module: dagster.core.instance
+ class: DagsterInstance
+ config:
+ some_key:
+ env: ME_ENV_VAR
+```
+
+## Full configuration specification
+
+Here's a comprehensive `dagster.yaml` specification with all available options:
+
+```yaml
+local_artifact_storage:
+ module: dagster.core.storage.root
+ class: LocalArtifactStorage
+ config:
+ base_dir: '/path/to/dir'
+
+compute_logs:
+ module: dagster.core.storage.local_compute_log_manager
+ class: LocalComputeLogManager
+ config:
+ base_dir: /path/to/compute/logs
+
+# Alternatively, logs can be written to cloud storage providers like S3, GCS, and Azure blog storage. For example:
+# compute_logs:
+# module: dagster_aws.s3.compute_log_manager
+# class: S3ComputeLogManager
+# config:
+# bucket: "mycorp-dagster-compute-logs"
+# prefix: "dagster-test-"
+
+storage:
+ sqlite:
+ base_dir: /path/to/sqlite/storage
+ # Or use postgres:
+ # postgres:
+ # postgres_db:
+ # hostname: localhost
+ # username: dagster
+ # password: dagster
+ # db_name: dagster
+ # port: 5432
+
+run_queue:
+ max_concurrent_runs: 15
+ tag_concurrency_limits:
+ - key: 'database'
+ value: 'redshift'
+ limit: 4
+ - key: 'dagster/backfill'
+ limit: 10
+
+run_storage:
+ module: dagster.core.storage.runs
+ class: SqliteRunStorage
+ config:
+ base_dir: /path/to/dagster/home/history
+
+event_log_storage:
+ module: dagster.core.storage.event_log
+ class: SqliteEventLogStorage
+ config:
+ base_dir: /path/to/dagster/home/history
+
+schedule_storage:
+ module: dagster.core.storage.schedules
+ class: SqliteScheduleStorage
+ config:
+ base_dir: /path/to/dagster/home/schedules
+
+scheduler:
+ module: dagster.core.scheduler
+ class: DagsterDaemonScheduler
+
+run_coordinator:
+ module: dagster.core.run_coordinator
+ class: DefaultRunCoordinator
+ # class: QueuedRunCoordinator
+ # config:
+ # max_concurrent_runs: 25
+ # tag_concurrency_limits:
+ # - key: "dagster/backfill"
+ # value: "true"
+ # limit: 1
+
+run_launcher:
+ module: dagster.core.launcher
+ class: DefaultRunLauncher
+ # module: dagster_docker
+ # class: DockerRunLauncher
+ # module: dagster_k8s.launcher
+ # class: K8sRunLauncher
+ # config:
+ # service_account_name: pipeline_run_service_account
+ # job_image: my_project/dagster_image:latest
+ # instance_config_map: dagster-instance
+ # postgres_password_secret: dagster-postgresql-secret
+
+telemetry:
+ enabled: true
+
+run_monitoring:
+ enabled: true
+ poll_interval_seconds: 60
+
+run_retries:
+ enabled: true
+ max_retries: 3
+ retry_on_asset_or_op_failure: true
+
+code_servers:
+ local_startup_timeout: 360
+
+secrets:
+ my_secret:
+ env: MY_SECRET_ENV_VAR
+
+retention:
+ schedule:
+ purge_after_days: 90
+ sensor:
+ purge_after_days:
+ skipped: 7
+ failure: 30
+ success: -1
+
+sensors:
+ use_threads: true
+ num_workers: 8
+
+schedules:
+ use_threads: true
+ num_workers: 8
+
+auto_materialize:
+ enabled: true
+ minimum_interval_seconds: 3600
+ run_tags:
+ key: 'value'
+ respect_materialization_data_versions: true
+ max_tick_retries: 3
+ use_sensors: false
+ use_threads: false
+ num_workers: 4
+```
+
+## Configuration options
+
+### `storage`
+
+Configures how job and asset history is persisted.
+
+```yaml
+storage:
+ sqlite:
+ base_dir: /path/to/sqlite/storage
+```
+
+Options:
+
+- `sqlite`: Use SQLite for storage
+- `postgres`: Use PostgreSQL for storage (requires `dagster-postgres` library)
+- `mysql`: Use MySQL for storage (requires `dagster-mysql` library)
+
+### `run_launcher`
+
+Determines where runs are executed.
+
+```yaml
+run_launcher:
+ module: dagster.core.launcher
+ class: DefaultRunLauncher
+```
+
+Options:
+
+- `DefaultRunLauncher`: Spawns a new process on the same node as the job's code location
+- `DockerRunLauncher`: Allocates a Docker container per run
+- `K8sRunLauncher`: Allocates a Kubernetes job per run
+
+### `run_coordinator`
+
+Sets prioritization rules and concurrency limits for runs.
+
+```yaml
+run_coordinator:
+ module: dagster.core.run_coordinator
+ class: QueuedRunCoordinator
+ config:
+ max_concurrent_runs: 25
+```
+
+Options:
+
+- `DefaultRunCoordinator`: Immediately sends runs to the run launcher
+- `QueuedRunCoordinator`: Allows setting limits on concurrent runs
+
+### `compute_logs`
+
+Controls the capture and persistence of stdout and stderr logs.
+
+```yaml
+compute_logs:
+ module: dagster.core.storage.local_compute_log_manager
+ class: LocalComputeLogManager
+ config:
+ base_dir: /path/to/compute/logs
+```
+
+Options:
+
+- `LocalComputeLogManager`: Writes logs to disk
+- `NoOpComputeLogManager`: Does not store logs
+- `AzureBlobComputeLogManager`: Writes logs to Azure Blob Storage
+- `GCSComputeLogManager`: Writes logs to Google Cloud Storage
+- `S3ComputeLogManager`: Writes logs to AWS S3
+
+### `local_artifact_storage`
+
+Configures storage for artifacts that require a local disk or when using the filesystem I/O manager.
+
+```yaml
+local_artifact_storage:
+ module: dagster.core.storage.root
+ class: LocalArtifactStorage
+ config:
+ base_dir: /path/to/artifact/storage
+```
+
+### `telemetry`
+
+Controls whether Dagster collects anonymized usage statistics.
+
+```yaml
+telemetry:
+ enabled: false
+```
+
+### `code_servers`
+
+Configures how Dagster loads code in a code location.
+
+```yaml
+code_servers:
+ local_startup_timeout: 360
+```
+
+### `retention`
+
+Configures how long Dagster retains certain types of data.
+
+```yaml
+retention:
+ schedule:
+ purge_after_days: 90
+ sensor:
+ purge_after_days:
+ skipped: 7
+ failure: 30
+ success: -1
+```
+
+### `sensors`
+
+Configures how sensors are evaluated.
+
+```yaml
+sensors:
+ use_threads: true
+ num_workers: 8
+```
+
+### `schedules`
+
+Configures how schedules are evaluated.
+
+```yaml
+schedules:
+ use_threads: true
+ num_workers: 8
+```
+
+### `auto_materialize`
+
+Configures automatic materialization of assets.
+
+```yaml
+auto_materialize:
+ enabled: true
+ minimum_interval_seconds: 3600
+ run_tags:
+ key: 'value'
+ respect_materialization_data_versions: true
+ max_tick_retries: 3
+ use_sensors: false
+ use_threads: false
+ num_workers: 4
+```
+
+Options:
+
+- `enabled`: Whether auto-materialization is enabled (boolean)
+- `minimum_interval_seconds`: Minimum interval between materializations (integer)
+- `run_tags`: Tags to apply to auto-materialization runs (dictionary)
+- `respect_materialization_data_versions`: Whether to respect data versions when materializing (boolean)
+- `max_tick_retries`: Maximum number of retries for each auto-materialize tick that raises an error (integer, default: 3)
+- `use_sensors`: Whether to use sensors for auto-materialization (boolean)
+- `use_threads`: Whether to use threads for processing ticks (boolean, default: false)
+- `num_workers`: Number of threads to use for processing ticks from multiple automation policy sensors in parallel (integer)
+
+### `concurrency`
+
+Configures default concurrency limits for operations.
+
+```yaml
+concurrency:
+ default_op_concurrency_limit: 10
+```
+
+Options:
+
+- `default_op_concurrency_limit`: The default maximum number of concurrent operations for an unconfigured concurrency key (integer)
+
+## References
+
+- [Dagster Instance Config Source Code](https://github.com/dagster-io/dagster/blob/master/python_modules/dagster/dagster/_core/instance/config.py)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/aws.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/aws.md
new file mode 100644
index 0000000000000..622ff4242c342
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/aws.md
@@ -0,0 +1,252 @@
+---
+title: "Deploying Dagster to Amazon Web Services"
+description: To deploy Dagster to AWS, EC2 or ECS can host the Dagster webserver, RDS can store runs and events, and S3 can act as an IO manager.
+sidebar_position: 50
+---
+
+This guide provides instructions for deploying Dagster on Amazon Web Services (AWS). You can use EC2 or ECS to host the Dagster webserver and the Dagster daemon, RDS to store runs and events, and S3 as an I/O manager to store op inputs and outputs.
+
+## Hosting Dagster on EC2
+
+To host Dagster on a bare VM or in Docker on EC2, see "[Running Dagster as a service](/guides/deploy/deployment-options/deploying-dagster-as-a-service).
+
+## Using RDS for run and event log storage
+
+You can use a hosted RDS PostgreSQL database for your Dagster run/events data by configuring your `dagster.yaml` file:
+
+```python file=/deploying/dagster-pg.yaml
+storage:
+ postgres:
+ postgres_db:
+ username: my_username
+ password: my_password
+ hostname: my_hostname
+ db_name: my_database
+ port: 5432
+```
+
+In this case, you'll want to ensure that:
+
+- You provide the right connection strings for your RDS instance
+- The node or container hosting the webserver is able to connect to RDS
+
+Be sure that this file is present and `DAGSTER_HOME` is set on the node where the webserver is running.
+
+:::note
+
+Using RDS for run and event log storage doesn't require that the webserver be running in the cloud. If you're connecting a local webserver instance to a remote RDS storage, verify that your local node is able to connect to RDS.
+
+:::
+
+## Deploying in ECS
+
+{/* TODO turn back into once that's implemented */}
+:::tip
+
+You can find the code for this example on [GitHub](https://github.com/dagster-io/dagster/tree/master/examples/deploy_ecs).
+
+:::
+
+The Deploying on ECS example on GitHub demonstrates how to configure the [Docker Compose CLI integration with ECS](https://docs.docker.com/cloud/ecs-integration/) to manage all of the required AWS resources that Dagster needs to run on ECS. The example includes:
+
+- A webserver container for loading and launching jobs
+- A `dagster-daemon` container for managing a run queue and submitting runs from schedules and sensors
+- A Postgres container for persistent storage
+- A container with user job code
+
+The [Dagster instance](/guides/deploy/dagster-instance-configuration) uses the to launch each run in its own ECS task.
+
+### Launching runs in ECS
+
+The launches an ECS task per run. It assumes that the rest of your Dagster deployment is also running in ECS.
+
+By default, each run's task registers its own task definition. To simplify configuration, these task definitions inherit most of their configuration (networking, cpu, memory, environment, etc.) from the task that launches the run but overrides its container definition with a new command to launch a Dagster run.
+
+When using the , runs launched via the Dagster UI or GraphQL inherit their task definitions from the webserver task while runs launched from a sensor or schedule inherit their task definitions from the daemon task.
+
+Alternatively, you can define your own task definition in your `dagster.yaml`:
+
+```yaml
+run_launcher:
+ module: "dagster_aws.ecs"
+ class: "EcsRunLauncher"
+ config:
+ task_definition: "arn:aws:ecs:us-east-1:1234567890:task-definition/my-task-definition:1"
+ container_name: "my_container_name"
+```
+
+### Customizing CPU, memory, and ephemeral storage in ECS
+
+You can set the `run_launcher.config.run_resources` field to customize the default resources for Dagster runs. For example:
+
+```yaml
+run_launcher:
+ module: "dagster_aws.ecs"
+ class: "EcsRunLauncher"
+ config:
+ run_resources:
+ cpu: "256"
+ memory: "512" # In MiB
+ ephemeral_storage: 128 # In GiB
+```
+
+:::note
+
+Fargate tasks only support [certain combinations of CPU and memory](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-cpu-memory-error.html) and values of ephemeral storage between 21 and 200 GiB (the default is set to 20 GiB).
+
+:::
+
+You can also use job tags to customize the CPU, memory, or ephemeral storage of every run for a particular job:
+
+```py
+from dagster import job, op
+
+@op()
+def my_op(context):
+ context.log.info('running')
+
+@job(
+ tags = {
+ "ecs/cpu": "256",
+ "ecs/memory": "512",
+ "ecs/ephemeral_storage": "40",
+ }
+)
+def my_job():
+ my_op()
+```
+
+If these tags are set, they will override any defaults set on the run launcher.
+
+### Customizing the launched run's task
+
+The creates a new task for each run, using the current ECS task to determine network configuration. For example, the launched run will use the same ECS cluster, subnets, security groups, and launch type (e.g. Fargate or EC2).
+
+To adjust the configuration of the launched run's task, set the `run_launcher.config.run_task_kwargs` field to a dictionary with additional key-value pairs that should be passed into the `run_task` boto3 API call. For example, to launch new runs in EC2 from a task running in Fargate, you could apply this configuration:
+
+```yaml
+run_launcher:
+ module: "dagster_aws.ecs"
+ class: "EcsRunLauncher"
+ config:
+ run_task_kwargs:
+ launchType: "EC2"
+```
+
+or to set the capacity provider strategy to run in Fargate Spot instances:
+
+```yaml
+run_launcher:
+ module: "dagster_aws.ecs"
+ class: "EcsRunLauncher"
+ config:
+ run_task_kwargs:
+ capacityProviderStrategy:
+ - capacityProvider: "FARGATE_SPOT"
+```
+
+You can also use the `ecs/run_task_kwargs` tag to customize the ECS task of every run for a particular job:
+
+```py
+from dagster import job, op
+
+@op()
+def my_op(context):
+ context.log.info('running')
+
+@job(
+ tags = {
+ "ecs/run_task_kwargs": {
+ "capacityProviderStrategy": [
+ {
+ "capacityProvider": "FARGATE_SPOT",
+ },
+ ],
+ },
+ }
+)
+def my_job():
+ my_op()
+```
+
+Refer to the [boto3 docs](https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/ecs.html#ECS.Client.run_task) for the full set of available arguments to `run_task`. Additionally, note that:
+
+- Keys are in camelCase as they correspond to arguments in boto's API
+- All arguments with the exception of `taskDefinition` and `overrides` can be used in `run_launcher.config.run_task_kwargs`. `taskDefinition` can be overridden by configuring the `run_launcher.config.task_definition` field instead.
+
+### Secrets management in ECS
+
+ECS can bind [AWS Secrets Managers secrets as environment variables when runs launch](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/specifying-sensitive-data-secrets.html). By default, Dagster will fetch any Secrets Manager secrets tagged with the `dagster` key and set them as environment variables.
+
+Alternatively, you can set your own tag name in your `dagster.yaml`:
+
+```yaml
+run_launcher:
+ module: "dagster_aws.ecs"
+ class: "EcsRunLauncher"
+ config:
+ secrets_tag: "my-tag-name"
+```
+
+In this example, any secret tagged with a key `my-tag-name` will be included as an environment variable with the name and value as that secret. The value of the tag is ignored.
+
+Additionally, you can pass specific secrets using the [same structure as the ECS API](https://docs.aws.amazon.com/AmazonECS/latest/APIReference/API_Secret.html):
+
+```yaml
+run_launcher:
+ module: "dagster_aws.ecs"
+ class: "EcsRunLauncher"
+ config:
+ secrets:
+ - name: "MY_API_TOKEN"
+ valueFrom: "arn:aws:secretsmanager:us-east-1:123456789012:secret:FOO-AbCdEf:token::"
+ - name: "MY_PASSWORD"
+ valueFrom: "arn:aws:secretsmanager:us-east-1:123456789012:secret:FOO-AbCdEf:password::"
+```
+
+In this example, any secret tagged with `dagster` will be included in the environment. `MY_API_TOKEN` and `MY_PASSWORD` will also be included in the environment.
+
+## Using S3 for I/O management
+
+To enable parallel computation (e.g., with the multiprocessing or Dagster celery executors), you'll need to configure persistent [I/O managers](/guides/build/io-managers/). For example, using an S3 bucket to store data passed between ops.
+
+You'll need to use as your I/O Manager or customize your own persistent I/O managers. Refer to the [I/O managers documentation](/guides/build/io-managers/) for an example.
+
+{/* TODO convert to */}
+```python file=/deploying/aws/io_manager.py
+from dagster_aws.s3.io_manager import s3_pickle_io_manager
+from dagster_aws.s3.resources import s3_resource
+
+from dagster import Int, Out, job, op
+
+
+@op(out=Out(Int))
+def my_op():
+ return 1
+
+
+@job(
+ resource_defs={
+ "io_manager": s3_pickle_io_manager,
+ "s3": s3_resource,
+ }
+)
+def my_job():
+ my_op()
+```
+
+Then, add the following YAML block in your job's config:
+
+{/* TODO convert to */}
+```yaml file=/deploying/aws/io_manager.yaml
+resources:
+ io_manager:
+ config:
+ s3_bucket: my-cool-bucket
+ s3_prefix: good/prefix-for-files-
+```
+
+The resource uses `boto` under the hood. If you're accessing your private buckets, you'll need to provide the `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` environment variables or follow [one of the other boto authentication methods](https://boto3.amazonaws.com/v1/documentation/api/latest/guide/credentials.html#configuring-credentials).
+
+With this in place, your job runs will store data passed between ops on S3 in the location `s3:///dagster/storage//.compute`.
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/dask.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/dask.md
new file mode 100644
index 0000000000000..799a8f3983493
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/dask.md
@@ -0,0 +1,134 @@
+---
+title: "Executing on Dask"
+description: The dask_executor uses Dask to execute Dagster ops.
+---
+
+The [dagster-dask](/api/python-api/libraries/dagster-dask) module makes a **`dask_executor`** available, which can target either a local Dask cluster or a distributed cluster. Computation is distributed across the cluster at the execution step level -- that is, we use Dask to orchestrate execution of the steps in a job, not to parallelize computation within those steps.
+
+This executor takes the compiled execution plan, and converts each execution step into a [Dask Future](https://docs.dask.org/en/latest/futures.html) configured with the appropriate task dependencies to ensure tasks are properly sequenced. When the job is executed, these futures are generated and then awaited by the parent Dagster process.
+
+Data is passed between step executions via [IO Managers](/guides/build/io-managers/). As a consequence, a persistent shared storage (such as a network filesystem shared by all of the Dask nodes, S3, or GCS) must be used.
+
+Note that, when using this executor, the compute function of a single op is still executed in a single process on a single machine. If your goal is to distribute execution of workloads _within_ the logic of a single op, you may find that invoking Dask or PySpark directly from within the body of an op's compute function is a better fit than the engine layer covered in this documentation.
+
+## Requirements
+
+Install [dask.distributed](https://distributed.readthedocs.io/en/latest/install.html).
+
+## Local execution
+
+It is relatively straightforward to set up and run a Dagster job on local Dask. This can be useful for testing.
+
+First, run `pip install dagster-dask`.
+
+Then, create a job with the dask executor:
+
+{/* TODO convert to */}
+```python file=/deploying/dask_hello_world.py startafter=start_local_job_marker endbefore=end_local_job_marker
+from dagster_dask import dask_executor
+
+from dagster import job, op
+
+
+@op
+def hello_world():
+ return "Hello, World!"
+
+
+@job(executor_def=dask_executor)
+def local_dask_job():
+ hello_world()
+```
+
+Now you can run this job with a config block such as the following:
+
+{/* TODO convert to */}
+```python file=/deploying/dask_hello_world.yaml
+execution:
+ config:
+ cluster:
+ local:
+```
+
+Executing this job will spin up local Dask execution, run the job, and exit.
+
+## Distributed execution
+
+If you want to use a Dask cluster for distributed execution, you will first need to [set up a Dask cluster](https://distributed.readthedocs.io/en/latest/quickstart.html#setup-dask-distributed-the-hard-way). Note that the machine running the Dagster parent process must be able to connect to the host/port on which the Dask scheduler is running.
+
+You'll also need an IO manager that uses persistent shared storage, which should be attached to the job along with any resources on which it depends. Here, we use the :
+
+{/* TODO convert to */}
+
+```python file=/deploying/dask_hello_world_distributed.py startafter=start_distributed_job_marker endbefore=end_distributed_job_marker
+from dagster_aws.s3.io_manager import s3_pickle_io_manager
+from dagster_aws.s3.resources import s3_resource
+from dagster_dask import dask_executor
+
+from dagster import job, op
+
+
+@op
+def hello_world():
+ return "Hello, World!"
+
+
+@job(
+ executor_def=dask_executor,
+ resource_defs={"io_manager": s3_pickle_io_manager, "s3": s3_resource},
+)
+def distributed_dask_job():
+ hello_world()
+```
+
+For distributing task execution on a Dask cluster, you must provide a config block that includes the address/port of the Dask scheduler:
+
+{/* TODO convert to */}
+```python file=/deploying/dask_remote.yaml
+resources:
+ io_manager:
+ config:
+ s3_bucket: your_bucket_name
+execution:
+ config:
+ cluster:
+ existing:
+ address: "dask_scheduler.dns_name:8787"
+```
+
+Since Dask will invoke your job code on the cluster workers, you must ensure that the latest version of your Python code is available to all of the Dask workers. Ideally, you'll package this as a Python module, and target your `workspace.yaml` at this module.
+
+## Managing compute resources with Dask
+
+Dask has [basic support](https://distributed.dask.org/en/latest/resources.html) for compute resource management. In Dask you can specify that a particular worker node has, say, 3 GPUs, and then tasks which are specified with GPU requirements will be scheduled to respect that constraint on available resources.
+
+In Dask, you'd set this up by launching your workers with resource specifications:
+
+```shell
+dask-worker scheduler:8786 --resources "GPU=2"
+```
+
+and then when submitting tasks to the Dask cluster, specifying resource requirements in the Python API:
+
+```python
+client.submit(task, resources={'GPU': 1})
+```
+
+Dagster has simple support for Dask resource specification at the op level for ops that will be executed on Dask clusters. In your op definition, just add _tags_ as follows:
+
+```python
+@op(
+ ...
+ tags={'dagster-dask/resource_requirements': {"GPU": 1}},
+)
+def my_op(...):
+ pass
+```
+
+The dict passed to `dagster-dask/resource_requirements` will be passed through as the `resources` argument to the Dask client's **`~dask:distributed.Client.submit`** method for execution on a Dask cluster. Note that in non-Dask execution, this key will be ignored.
+
+## Caveats
+
+Dagster logs are not yet retrieved from Dask workers; this will be addressed in follow-up work.
+
+While this library is still nascent, we're working to improve it, and we are happy to accept contributions.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/deploying-dagster-as-a-service.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/deploying-dagster-as-a-service.md
new file mode 100644
index 0000000000000..74bf76762229e
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/deploying-dagster-as-a-service.md
@@ -0,0 +1,117 @@
+---
+title: "Deploying Dagster as a service"
+sidebar_label: Dagster as a service
+description: 'Learn how to deploy Dagster as a service on a single machine'
+sidebar_position: 20
+---
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- A Dagster project created
+- Working knowledge of containerization and managing services
+
+
+
+This guide will walk you through deploying Dagster as a service on a single machine. It includes instructions for setting up the Dagster webserver and daemon. This approach is suitable for small-scale deployments or for testing purposes. For production environments, consider using containerized deployments or cloud-based solutions
+
+## Running the Dagster Webserver
+
+The Dagster webserver is the core component of any Dagster deployment. It serves the Dagster UI and responds to GraphQL queries.
+
+### Installation
+
+First, install the Dagster webserver:
+
+```bash
+pip install dagster-webserver
+```
+
+### Starting the Dagster Webserver
+
+Before starting the webserver, set the `DAGSTER_HOME` environment variable, which tells Dagster where to store its persistent data and logs.
+
+
+
+
+```bash
+export DAGSTER_HOME="/home/yourusername/dagster_home"
+```
+
+
+
+
+```powershell
+$env:DAGSTER_HOME = "C:\Users\YourUsername\dagster_home"
+```
+
+
+
+
+
+Then, to run the webserver, use the following command:
+
+```bash
+dagster-webserver -h 0.0.0.0 -p 3000
+```
+
+This configuration will:
+
+- Set the `DAGSTER_HOME` environment variable, which tells Dagster where to store its persistent data and logs
+- Write execution logs to `$DAGSTER_HOME/logs`
+- Listen on `0.0.0.0:3000`
+
+## Running the Dagster Daemon
+
+The Dagster daemon is necessary if you're using schedules, sensors, backfills, or want to set limits on the number of runs that can be executed simultaneously.
+
+### Installation
+
+Install the Dagster daemon:
+
+```bash
+pip install dagster
+```
+
+### Starting the Daemon
+
+Make sure you've set the `DAGSTER_HOME` environment variable, see [Running the Dagster Webserver](#running-the-dagster-webserver) for instructions.
+Then, run the Dagster daemon with this command:
+
+```bash
+dagster-daemon run
+```
+
+The `dagster-daemon` process will periodically check your instance for:
+
+- New runs to be launched from your run queue
+- Runs triggered by your running schedules or sensors
+
+:::tip
+Ensure that the `dagster-daemon` process has access to:
+
+- Your `dagster.yaml` file
+- Your `workspace.yaml` file
+- The components defined on your instance
+- The repositories defined in your workspace
+ :::
+
+### Monitoring the Daemon
+
+You can check the status of your `dagster-daemon` process in the Dagster UI:
+
+1. Navigate to the Instance tab in the left-hand navigation bar
+2. View the daemon status
+
+:::important
+A deployment can have multiple instances of `dagster-webserver`, but should include only a single `dagster-daemon` process.
+:::
+
+## Next Steps
+
+Now that you have Dagster running as a service, you might want to explore:
+
+- [Configuring your Dagster instance](/guides/deploy/dagster-instance-configuration)
+- [Setting up schedules and sensors](/guides/automate/schedules)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/docker.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/docker.md
new file mode 100644
index 0000000000000..b2d5c619d3b60
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/docker.md
@@ -0,0 +1,194 @@
+---
+title: "Deploying Dagster using Docker Compose"
+description: A guide to deploying Dagster with Docker Compose.
+sidebar_position: 40
+---
+
+This guide provides instructions for deploying Dagster using Docker Compose. This is useful when you want to, for example, deploy Dagster on an AWS EC2 host. A typical Dagster Docker deployment includes a several long-running containers: one for the webserver, one for the daemon, and one for each code location. It also typically executes each run in its own container.
+
+The [full example is available on GitHub](https://github.com/dagster-io/dagster/blob/master/examples/deploy_docker).
+
+
+ Prerequisites
+- Familiarity with Docker and Docker Compose
+- Familiarity with `dagster.yaml` instance configuration
+- Familiarity with `workspace.yaml` code location configuration
+
+
+
+## Define a Docker image for the Dagster webserver and daemon
+
+The Dagster webserver and daemon are the two _host processes_ in a Dagster deployment. They typically each run in their own container, using the same Docker image. This image contains Dagster packages and configuration, but no user code.
+
+To build this Docker image, use a Dockerfile like the following, with a name like `Dockerfile_dagster`:
+
+```dockerfile
+FROM python:3.10-slim
+
+RUN pip install \
+ dagster \
+ dagster-graphql \
+ dagster-webserver \
+ dagster-postgres \
+ dagster-docker
+
+# Set $DAGSTER_HOME and copy dagster.yaml and workspace.yaml there
+ENV DAGSTER_HOME=/opt/dagster/dagster_home/
+
+RUN mkdir -p $DAGSTER_HOME
+
+COPY dagster.yaml workspace.yaml $DAGSTER_HOME
+
+WORKDIR $DAGSTER_HOME
+```
+
+Additionally, the following files should be in the same directory as the Docker file:
+- A `workspace.yaml` to tell the webserver and daemon the location of the code servers
+- A `dagster.yaml` to configure the Dagster instance
+
+## Define a Docker image for each code location
+
+Each code location typically has its own Docker image, and that image is also used for runs launched for that code location.
+
+To build a Docker image for a code location, use a Dockerfile like the following, with a name like `Dockerfile_code_location_1`:
+
+```dockerfile
+FROM python:3.10-slim
+
+RUN pip install \
+ dagster \
+ dagster-postgres \
+ dagster-docker
+
+# Add code location code
+WORKDIR /opt/dagster/app
+COPY directory/with/your/code/ /opt/dagster/app
+
+# Run dagster code server on port 4000
+EXPOSE 4000
+
+# CMD allows this to be overridden from run launchers or executors to execute runs and steps
+CMD ["dagster", "code-server", "start", "-h", "0.0.0.0", "-p", "4000", "-f", "definitions.py"]
+```
+
+## Write a Docker Compose file
+
+The following `docker-compose.yaml` defines how to run the webserver container, daemon container, code location containers, and database container:
+
+```yaml title="docker-compose.yaml"
+version: "3.7"
+
+services:
+ # This service runs the postgres DB used by dagster for run storage, schedule storage,
+ # and event log storage. Depending on the hardware you run this Compose on, you may be able
+ # to reduce the interval and timeout in the healthcheck to speed up your `docker-compose up` times.
+ docker_example_postgresql:
+ image: postgres:11
+ container_name: docker_example_postgresql
+ environment:
+ POSTGRES_USER: "postgres_user"
+ POSTGRES_PASSWORD: "postgres_password"
+ POSTGRES_DB: "postgres_db"
+ networks:
+ - docker_example_network
+ healthcheck:
+ test: ["CMD-SHELL", "pg_isready -U postgres_user -d postgres_db"]
+ interval: 10s
+ timeout: 8s
+ retries: 5
+
+ # This service runs the gRPC server that loads your user code, in both dagster-webserver
+ # and dagster-daemon. By setting DAGSTER_CURRENT_IMAGE to its own image, we tell the
+ # run launcher to use this same image when launching runs in a new container as well.
+ # Multiple containers like this can be deployed separately - each just needs to run on
+ # its own port, and have its own entry in the workspace.yaml file that's loaded by the
+ # webserver.
+ docker_example_user_code:
+ build:
+ context: .
+ dockerfile: ./Dockerfile_user_code
+ container_name: docker_example_user_code
+ image: docker_example_user_code_image
+ restart: always
+ environment:
+ DAGSTER_POSTGRES_USER: "postgres_user"
+ DAGSTER_POSTGRES_PASSWORD: "postgres_password"
+ DAGSTER_POSTGRES_DB: "postgres_db"
+ DAGSTER_CURRENT_IMAGE: "docker_example_user_code_image"
+ networks:
+ - docker_example_network
+
+ # This service runs dagster-webserver, which loads your user code from the user code container.
+ # Since our instance uses the QueuedRunCoordinator, any runs submitted from the webserver will be put on
+ # a queue and later dequeued and launched by dagster-daemon.
+ docker_example_webserver:
+ build:
+ context: .
+ dockerfile: ./Dockerfile_dagster
+ entrypoint:
+ - dagster-webserver
+ - -h
+ - "0.0.0.0"
+ - -p
+ - "3000"
+ - -w
+ - workspace.yaml
+ container_name: docker_example_webserver
+ expose:
+ - "3000"
+ ports:
+ - "3000:3000"
+ environment:
+ DAGSTER_POSTGRES_USER: "postgres_user"
+ DAGSTER_POSTGRES_PASSWORD: "postgres_password"
+ DAGSTER_POSTGRES_DB: "postgres_db"
+ volumes: # Make docker client accessible so we can terminate containers from the webserver
+ - /var/run/docker.sock:/var/run/docker.sock
+ - /tmp/io_manager_storage:/tmp/io_manager_storage
+ networks:
+ - docker_example_network
+ depends_on:
+ docker_example_postgresql:
+ condition: service_healthy
+ docker_example_user_code:
+ condition: service_started
+
+ # This service runs the dagster-daemon process, which is responsible for taking runs
+ # off of the queue and launching them, as well as creating runs from schedules or sensors.
+ docker_example_daemon:
+ build:
+ context: .
+ dockerfile: ./Dockerfile_dagster
+ entrypoint:
+ - dagster-daemon
+ - run
+ container_name: docker_example_daemon
+ restart: on-failure
+ environment:
+ DAGSTER_POSTGRES_USER: "postgres_user"
+ DAGSTER_POSTGRES_PASSWORD: "postgres_password"
+ DAGSTER_POSTGRES_DB: "postgres_db"
+ volumes: # Make docker client accessible so we can launch containers using host docker
+ - /var/run/docker.sock:/var/run/docker.sock
+ - /tmp/io_manager_storage:/tmp/io_manager_storage
+ networks:
+ - docker_example_network
+ depends_on:
+ docker_example_postgresql:
+ condition: service_healthy
+ docker_example_user_code:
+ condition: service_started
+
+networks:
+ docker_example_network:
+ driver: bridge
+ name: docker_example_network
+```
+
+## Start your deployment
+
+To start the deployment, run:
+
+```shell
+docker compose up
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/gcp.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/gcp.md
new file mode 100644
index 0000000000000..c9e4c3f1d56f5
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/gcp.md
@@ -0,0 +1,67 @@
+---
+title: "Deploying Dagster to Google Cloud Platform"
+description: To deploy Dagster to GCP, Google Compute Engine (GCE) can host the Dagster webserver, Google Cloud SQL can store runs and events, and Google Cloud Storage (GCS) can act as an IO manager.
+sidebar_position: 60
+---
+
+To deploy Dagster to GCP, Google Compute Engine (GCE) can host the Dagster webserver, Google Cloud SQL can store runs and events, and Google Cloud Storage (GCS) can act as an IO manager.
+
+## Hosting the Dagster webserver or Dagster Daemon on GCE
+
+To host the Dagster webserver or Dagster daemon on a bare VM or in Docker on GCE, see [Running Dagster as a service](/guides/deploy/deployment-options/deploying-dagster-as-a-service).
+
+## Using Cloud SQL for run and event log storage
+
+We recommend launching a Cloud SQL PostgreSQL instance for run and events data. You can configure the webserver to use Cloud SQL to run and events data by setting blocks in your `$DAGSTER_HOME/dagster.yaml` appropriately:
+
+```python file=/deploying/dagster-pg.yaml
+storage:
+ postgres:
+ postgres_db:
+ username: my_username
+ password: my_password
+ hostname: my_hostname
+ db_name: my_database
+ port: 5432
+```
+
+In this case, you'll want to ensure you provide the right connection strings for your Cloud SQL instance, and that the node or container hosting the webserver is able to connect to Cloud SQL.
+
+Be sure that this file is present, and `_DAGSTER_HOME_` is set, on the node where the webserver is running.
+
+Note that using Cloud SQL for run and event log storage does not require that the webserver be running in the cloud. If you are connecting a local webserver instance to a remote Cloud SQL storage, double check that your local node is able to connect to Cloud SQL.
+
+## Using GCS for IO Management
+
+You'll probably also want to configure a GCS bucket to store op outputs via persistent [IO Managers](/guides/build/io-managers/). This enables reexecution, review and audit of op outputs, and cross-node cooperation (e.g., with the or ).
+
+You'll first need to need to create a job using as its IO Manager (or [define a custom IO Manager](/guides/build/io-managers/defining-a-custom-io-manager)):
+
+{/* TODO convert to */}
+```python file=/deploying/gcp/gcp_job.py
+from dagster_gcp.gcs.io_manager import gcs_pickle_io_manager
+from dagster_gcp.gcs.resources import gcs_resource
+
+from dagster import job
+
+
+@job(
+ resource_defs={
+ "gcs": gcs_resource,
+ "io_manager": gcs_pickle_io_manager,
+ },
+ config={
+ "resources": {
+ "io_manager": {
+ "config": {
+ "gcs_bucket": "my-cool-bucket",
+ "gcs_prefix": "good/prefix-for-files-",
+ }
+ }
+ }
+ },
+)
+def gcs_job(): ...
+```
+
+With this in place, your job runs will store outputs on GCS in the location `gs:///dagster/storage//files/.compute`.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/index.md
new file mode 100644
index 0000000000000..5757a19717013
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/index.md
@@ -0,0 +1,8 @@
+---
+title: "Deployment options"
+sidebar_position: 1
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/customizing-your-deployment.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/customizing-your-deployment.md
new file mode 100644
index 0000000000000..e4fc3a85873d1
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/customizing-your-deployment.md
@@ -0,0 +1,409 @@
+---
+title: "Customizing your Kubernetes deployment"
+description: This section covers common ways to customize your Dagster Kubernetes deployment.
+sidebar_position: 200
+---
+
+This guide covers common ways to customize your Dagster Helm deployment.
+
+## Specifying custom Kubernetes configuration
+
+Dagster allows you to pass custom configuration to the Kubernetes Jobs and Pods created by Dagster during execution.
+
+### Instance-level Kubernetes Configuration
+
+If your instance is using the , you can configure custom configuration for every run launched by Dagster by setting the `k8sRunLauncher.runK8sConfig` dictionary in the Helm chart.
+
+`k8sRunLauncher.runK8sConfig` is a dictionary with the following keys:
+
+- `containerConfig`: The Pod's container
+- `podSpecConfig`: The Pod's PodSpec
+- `podTemplateSpecMetadata`: The Pod's Metadata
+- `jobSpecConfig`: The Job's JobSpec
+- `jobMetadata`: The Job's Metadata
+
+Refer to the [Kubernetes documentation](https://kubernetes.io/docs/home/) for more information about containers, Pod Specs, etc.
+
+The value for each of these keys is a dictionary with the YAML configuration for the underlying Kubernetes object. The Kubernetes object fields can be configured using either snake case (for example, `volume_mounts`) or camel case (`volumeMounts`). For example:
+
+{/* TODO convert to */}
+```yaml file=/deploying/kubernetes/run_k8s_config.yaml
+runLauncher:
+ type: K8sRunLauncher
+ config:
+ k8sRunLauncher:
+ runK8sConfig:
+ containerConfig: # raw config for the pod's main container
+ resources:
+ limits:
+ cpu: 100m
+ memory: 128Mi
+ podTemplateSpecMetadata: # raw config for the pod's metadata
+ annotations:
+ mykey: myvalue
+ podSpecConfig: # raw config for the spec of the launched's pod
+ nodeSelector:
+ disktype: ssd
+ jobSpecConfig: # raw config for the kubernetes job's spec
+ ttlSecondsAfterFinished: 7200
+ jobMetadata: # raw config for the kubernetes job's metadata
+ annotations:
+ mykey: myvalue
+```
+
+If your Dagster job is configured with the that runs each step in its own pod, configuration that you set in `runK8sConfig` will also be propagated to the pods that are created for each step, unless that step's configuration is overridden using one of the methods below.
+
+### Per-job Kubernetes configuration
+
+If your instance is using the or , you can use the `dagster-k8s/config` tag on a Dagster job to pass custom configuration to the Kubernetes Jobs and Pods created by Dagster for that job.
+
+`dagster-k8s/config` is a dictionary with the following keys:
+
+- `container_config`: The Pod's Container
+- `pod_spec_config`: The Pod's PodSpec
+- `pod_template_spec_metadata`: The Pod's Metadata
+- `job_spec_config`: The Job's JobSpec
+- `job_metadata`: The Job's Metadata
+
+Refer to the [Kubernetes documentation](https://kubernetes.io/docs/home/) for more information about containers, Pod Specs, etc.
+
+The value for each of these keys is a dictionary with the YAML configuration for the underlying Kubernetes object. The Kubernetes object fields can be configured using either snake case (for example, `volume_mounts`) or camel case (`volumeMounts`). For example:
+
+{/* TODO convert to */}
+```python file=/deploying/kubernetes/k8s_config_tag_job.py startafter=start_k8s_config endbefore=end_k8s_config
+@job(
+ tags={
+ "dagster-k8s/config": {
+ "container_config": {
+ "resources": {
+ "requests": {"cpu": "250m", "memory": "64Mi"},
+ "limits": {"cpu": "500m", "memory": "2560Mi"},
+ },
+ "volume_mounts": [
+ {"name": "volume1", "mount_path": "foo/bar", "sub_path": "file.txt"}
+ ],
+ },
+ "pod_template_spec_metadata": {
+ "annotations": {"cluster-autoscaler.kubernetes.io/safe-to-evict": "true"}
+ },
+ "pod_spec_config": {
+ "volumes": [{"name": "volume1", "secret": {"secret_name": "volume_secret_name"}}],
+ "affinity": {
+ "node_affinity": {
+ "required_during_scheduling_ignored_during_execution": {
+ "node_selector_terms": [
+ {
+ "match_expressions": [
+ {
+ "key": "beta.kubernetes.io/os",
+ "operator": "In",
+ "values": ["windows", "linux"],
+ }
+ ]
+ }
+ ]
+ }
+ }
+ },
+ },
+ },
+ },
+)
+def my_job():
+ my_op()
+```
+
+Other run launchers will ignore the `dagster-k8s/config` tag.
+
+The default [executor](/guides/operate/run-executors) - - will run each job in its own pod, executing each step in an individual process. If your Dagster job produces assets which have a short compute time (compared to the step overhead time), consider avoiding the step process creation cost by using the executor, which runs each step serially in a single process. This can be especially useful where parallelism is obtained through a and is unnecessary within the job.
+
+For this use-case, the is more efficient than running each step in its own process. The is contraindicated, as the delay of scheduling and starting up a new Dagster pod to execute every step would significantly slow down overall execution time.
+
+### Kubernetes configuration on every step in a run
+
+If your Dagster job is configured with the that runs each step in its own pod, configuration that you set on a job using the `dagster-k8s/config` tag will _not_ be propagated to any of those step pods. Use the `step_k8s_config` field on the executor to control the Kubernetes configuration for every step pod.
+
+`step_k8s_config` is a dictionary with the following keys:
+
+- `container_config`: The Pod's Container
+- `pod_spec_config`: The Pod's PodSpec
+- `pod_template_spec_metadata`: The Pod's Metadata
+- `job_spec_config`: The Job's JobSpec
+- `job_metadata`: The Job's Metadata
+
+Refer to the [Kubernetes documentation](https://kubernetes.io/docs/home/) for more information about containers, Pod Specs, etc.
+
+The value for each of these keys is a dictionary with the YAML configuration for the underlying Kubernetes object. The Kubernetes object fields can be configured using either snake case (for example, `volume_mounts`) or camel case (`volumeMounts`). For example:
+
+{/* TODO convert to */}
+```python file=/deploying/kubernetes/step_k8s_config.py startafter=start_step_k8s_config endbefore=end_step_k8s_config
+my_k8s_executor = k8s_job_executor.configured(
+ {
+ "step_k8s_config": {
+ "container_config": {
+ "resources": {
+ "requests": {"cpu": "200m", "memory": "32Mi"},
+ }
+ }
+ }
+ }
+)
+
+@job(executor_def=my_k8s_executor)
+def my_job():
+ ...
+```
+
+### Kubernetes configuration on individual steps in a run
+
+If your Dagster job is configured with the or that run each step in its own Kubernetes pod, you can use the `dagster-k8s/config` tag on a Dagster op to control the Kubernetes configuration for that specific op.
+
+As above when used on jobs, `dagster-k8s/config` is a dictionary with the following keys:
+
+- `container_config`: The Pod's Container
+- `pod_spec_config`: The Pod's PodSpec
+- `pod_template_spec_metadata`: The Pod's Metadata
+- `job_spec_config`: The Job's JobSpec
+- `job_metadata`: The Job's Metadata
+
+Refer to the [Kubernetes documentation](https://kubernetes.io/docs/home/) for more information about containers, Pod Specs, etc.
+
+The value for each of these keys is a dictionary with the YAML configuration for the underlying Kubernetes object. The Kubernetes object fields can be configured using either snake case (for example, `volume_mounts`) or camel case (`volumeMounts`). For example:
+
+For example, for an asset:
+
+{/* TODO convert to */}
+```python file=/deploying/kubernetes/k8s_config_tag_asset.py startafter=start_k8s_config endbefore=end_k8s_config
+@asset(
+ op_tags={
+ "dagster-k8s/config": {
+ "container_config": {
+ "resources": {
+ "requests": {"cpu": "200m", "memory": "32Mi"},
+ }
+ },
+ }
+ }
+)
+def my_asset(context: AssetExecutionContext):
+ context.log.info("running")
+
+my_job = define_asset_job(name="my_job", selection="my_asset", executor_def=k8s_job_executor)
+```
+
+or an op:
+
+{/* TODO convert to */}
+```python file=/deploying/kubernetes/k8s_config_tag_op.py startafter=start_k8s_config endbefore=end_k8s_config
+@op(
+ tags={
+ "dagster-k8s/config": {
+ "container_config": {
+ "resources": {
+ "requests": {"cpu": "200m", "memory": "32Mi"},
+ }
+ },
+ }
+ }
+)
+def my_op(context: OpExecutionContext):
+ context.log.info("running")
+
+@job(executor_def=k8s_job_executor)
+def my_job():
+ my_op()
+```
+
+Other executors will ignore the `dagster-k8s/config` tag when it is set on an op or asset.
+
+### Precedence rules
+
+Kubernetes configuration can be applied at several different scopes:
+
+- At the deployment level, applying to every run in the deployment
+- At the code location, applying to every run launched from the code location
+- At the job level, applying to every run launched for that job
+- At the step level, if using the
+
+By default, if Kubernetes configuration is specified in multiple places, the configuration is merged recursively. Scalar values will be replaced by the configuration for the more specific scope, dictionary fields will be combined, and list fields will be appended to each other, discarding duplicate values. List fields that cannot be meaningfully appended, like `command` or `args`, are replaced.
+
+Consider the following example:
+
+- **In the Helm chart**, `k8sRunLauncher.runK8sConfig.podSpecConfig` is set to:
+
+ ```json
+ {
+ "node_selector": { "disktype": "ssd" },
+ "dns_policy": "ClusterFirst",
+ "image_pull_secrets": [{ "name": "my-secret" }]
+ }
+ ```
+
+- **But a specific job** has the `dagster-k8s/config` tag set to:
+
+ ```json
+ {
+ "pod_spec_config": {
+ "node_selector": { "region": "east" },
+ "dns_policy": "Default",
+ "image_pull_secrets": [{ "name": "another-secret" }]
+ }
+ }
+ ```
+
+The job will merge the two `node_selector` dictionaries, append the two `image_pull_secrets` lists, and replace the `dns_policy` scalar value. The resulting `pod_spec_config` will be:
+
+```json
+{
+ "node_selector": { "disktype": "ssd", "region": "east" },
+ "dns_policy": "Default",
+ "image_pull_secrets": [{ "name": "my-secret" }, { "name": "another-secret" }]
+}
+```
+
+To customize this behavior, you can also set the `merge_behavior` key in the `dagster-k8s/config` tag to `SHALLOW` instead of `DEEP`. When `merge_behavior` is set to `SHALLOW`, the dictionaries will be shallowly merged. The configuration for the more specific scope takes precedence if the same key is set in both dictionaries or if scalar values need to be replaced - for example, configuration at the code location level will replace configuration at the deployment level.
+
+To modify the previous example:
+
+- **In the Helm chart**, `k8sRunLauncher.runK8sConfig.podSpecConfig` is set to:
+
+ ```json
+ {
+ "node_selector": { "disktype": "ssd" },
+ "dns_policy": "ClusterFirst",
+ "image_pull_secrets": [{ "name": "my-secret" }]
+ }
+ ```
+
+- **But a specific job** has the `dagster-k8s/config` tag set to:
+
+ ```json
+ {
+ "pod_spec_config": {
+ "node_selector": { "region": "east" },
+ "image_pull_secrets": [{ "name": "another-secret" }]
+ },
+ "merge_behavior": "SHALLOW"
+ }
+ ```
+
+Since the `merge_behavior` is set to `SHALLOW`, the `node_selector` and `image_pull_secrets` from the job and the DNS policy from the Helm chart will be applied, since only the `node_selector` and `image_pull_secrets` are overridden in the job.
+
+The resulting `pod_spec_config` will be:
+
+```json
+{
+ "node_selector": { "region": "east" },
+ "dns_policy": "ClusterFirst",
+ "image_pull_secrets": [{ "name": "another-secret" }]
+}
+```
+
+:::note
+
+In Dagster code before version 1.7.0, the default merge behavior was `SHALLOW` instead of `DEEP`.
+
+:::
+
+## Configuring an external database
+
+In a real deployment, users will likely want to set up an external PostgreSQL database and configure the `postgresql` section of `values.yaml`.
+
+```yaml
+postgresql:
+ enabled: false
+ postgresqlHost: "postgresqlHost"
+ postgresqlUsername: "postgresqlUsername"
+ postgresqlPassword: "postgresqlPassword"
+ postgresqlDatabase: "postgresqlDatabase"
+ service:
+ port: 5432
+```
+
+Supplying `.Values.postgresql.postgresqlPassword` will create a Kubernetes Secret with key `postgresql-password`, containing the encoded password. This secret is used to supply the Dagster infrastructure with an environment variable that's used when creating the storages for the Dagster instance.
+
+If you use a secrets manager like [Vault](https://www.hashicorp.com/products/vault/kubernetes), it may be convenient to manage this Secret outside of the Dagster Helm chart. In this case, the generation of this Secret within the chart should be disabled, and `.Values.global.postgresqlSecretName` should be set to the name of the externally managed Secret.
+
+```yaml
+global:
+ postgresqlSecretName: "dagster-postgresql-secret"
+
+generatePostgresqlPasswordSecret: false
+```
+
+## Security
+
+Users will likely want to permission a ServiceAccount bound to a properly scoped Role to launch Jobs and create other Kubernetes resources.
+
+Users will likely want to use [Secrets](https://kubernetes.io/docs/concepts/configuration/secret/) for managing secure information such as database logins.
+
+### Separately deploying Dagster infrastructure and user code
+
+It may be desirable to manage two Helm releases for your Dagster deployment: one release for the Dagster infrastructure, which consists of the Dagster webserver and the Dagster daemon, and another release for your User Code, which contains the definitions of your pipelines written in Dagster. This way, changes to User Code can be decoupled from upgrades to core Dagster infrastructure.
+
+To do this, we offer the [`dagster` chart](https://artifacthub.io/packages/helm/dagster/dagster) and the [`dagster-user-deployments` chart](https://artifacthub.io/packages/helm/dagster/dagster-user-deployments).
+
+```shell
+$ helm search repo dagster
+NAME CHART VERSION APP VERSION DESCRIPTION
+dagster/dagster 0.11.0 0.11.0 Dagster is a system for building modern data ap...
+dagster/dagster-user-deployments 0.11.0 0.11.0 A Helm subchart to deploy Dagster User Code dep...
+```
+
+To manage these separate deployments, we first need to isolate Dagster infrastructure to its own deployment. This can be done by disabling the subchart that deploys the User Code in the `dagster` chart. This will prevent the `dagster` chart from creating the services and deployments related to User Code, as these will be managed in a separate release.
+
+```yaml
+dagster-user-deployments:
+ enableSubchart: false
+```
+
+Next, the workspace for the webserver must be configured with the future hosts and ports of the services exposing access to the User Code.
+
+```yaml
+dagsterWebserver:
+ workspace:
+ enabled: true
+ servers:
+ - host: "k8s-example-user-code-1"
+ port: 3030
+ - ...
+```
+
+Finally, the `dagster-user-deployments` subchart can now be managed in its own release. The list of possible overrides for the subchart can be found in [its `values.yaml`](https://github.com/dagster-io/dagster/blob/master/helm/dagster/charts/dagster-user-deployments/values.yaml).
+
+```shell
+helm upgrade --install user-code dagster/dagster-user-deployments -f /path/to/values.yaml
+```
+
+## Kubernetes Job and Pod TTL management
+
+If you use a Kubernetes distribution that supports the [TTL Controller](https://kubernetes.io/docs/concepts/workloads/controllers/ttlafterfinished/#ttl-controller), then `Completed` and `Failed` [Jobs](https://kubernetes.io/docs/concepts/workloads/controllers/job/) (and their associated [Pods](https://kubernetes.io/docs/concepts/workloads/pods/)) will be deleted after 1 day. The TTL value can be modified in your job tags:
+
+{/* TODO convert to */}
+```python file=/deploying/kubernetes/ttl_config_job.py startafter=start_ttl endbefore=end_ttl
+@job(
+ tags = {
+ 'dagster-k8s/config': {
+ 'job_spec_config': {
+ 'ttl_seconds_after_finished': 7200
+ }
+ }
+ }
+)
+def my_job():
+ my_op()
+```
+
+If you do not use a Kubernetes distribution that supports the [TTL Controller](https://kubernetes.io/docs/concepts/workloads/controllers/ttlafterfinished/#ttl-controller), then you can run the following commands:
+
+- Delete Dagster [Jobs](https://kubernetes.io/docs/concepts/workloads/controllers/job/) older than one day:
+
+ ```shell
+ kubectl get job | grep -e dagster-run -e dagster-step | awk 'match($4,/[0-9]+d/) {print $1}' | xargs kubectl delete job
+ ```
+
+- Delete completed [Pods](https://kubernetes.io/docs/concepts/workloads/pods/) older than one day:
+
+ ```shell
+ kubectl get pod | grep -e dagster-run -e dagster-step | awk 'match($3,/Completed/) {print $0}' | awk 'match($5,/[0-9]+d/) {print $1}' | xargs kubectl delete pod
+ ```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/deploying-to-kubernetes.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/deploying-to-kubernetes.md
new file mode 100644
index 0000000000000..e2041949db769
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/deploying-to-kubernetes.md
@@ -0,0 +1,201 @@
+---
+title: "Deploying Dagster to Kubernetes with Helm"
+sidebar_position: 100
+---
+
+This guide will walk you through how to run the Dagster-specific components of a Dagster production deployment on a Kubernetes cluster. This includes the Dagster daemon, a webserver to serve the Dagster UI, a PostgrSQL container, and your Dagster project user code.
+
+Dagster provides [Helm charts](https://github.com/dagster-io/dagster/tree/master/helm) for deploying Dagster that you can customize for your specific needs. For each Dagster component used by the Helm chart, Dagster publishes a corresponding image to [DockerHub](https://hub.docker.com/u/dagster).
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- **Familiarity with [Docker](https://docs.docker.com/)**, and:
+ - **To have Docker installed**. [Docker installation guide](https://docs.docker.com/engine/install/)
+ - **Access to a Docker image registry**, such as Amazon Web Services ECR or DockerHub. If you're following along on your local machine, this isn't required.
+- **Familiarity with [Kubernetes](https://kubernetes.io/docs/home/)**, and:
+ - **To have `kubectl` installed**. [Kubernetes installation guide](https://kubernetes.io/docs/tasks/tools/)
+ - **An existing Kubernetes cluster**. To follow along on your local machine, [install Docker Desktop](https://docs.docker.com/desktop/kubernetes/) and turn on the included Kubernetes server.
+- **Familiarity with [Helm](https://helm.sh/docs/)**, and:
+ - **To have Helm 3 installed**. [Helm installation guide](https://helm.sh/docs/intro/install/)
+- A Dagster project to deploy. You can also use the example project:
+ ```bash
+ dagster project from-example --example deploy_k8s --name deploy_k8s
+ ```
+
+
+
+
+## Step 1: Write and build a Docker image containing your Dagster project
+### Step 1.1: Write a Dockerfile
+Next, you'll build a Docker image that contains your Dagster project and all of its dependencies. The Dockerfile should:
+1. Copy your Dagster project into the image.
+2. Install `dagster`, `dagster-postgres`, and `dagster-k8s`, along with any other libraries your project depends on. The example project has a dependency on `pandas` so it's included in the `pip install` in the following example Dockerfile.
+3. Expose port 80, which we'll use to set up port-forwarding later.
+
+
+
+
+### Step 1.2: Build and push a Docker image
+
+To build your Docker image, run the following command from the directory where your Dockerfile is located:
+
+```bash
+docker build . -t iris_analysis:1
+```
+This builds the Docker image from Step 1.1 and gives it the name `iris_analysis` and tag `1`. You can set custom values for both the name and the tag. We recommend that each time you rebuild your Docker image, you assign a new value for the tag to ensure that the correct image is used when running your code.
+
+
+If you are using a Docker image registry, push the image to your registry. If you are following along on your local machine, you can skip this command.
+
+```bash
+docker push iris_analysis:1
+```
+
+If you are pushing your image to an image registry, you can find more information about this process in your registry's documentation:
+- [Amazon ECR](https://docs.aws.amazon.com/AmazonECR/latest/userguide/docker-push-ecr-image.html)
+- [DockerHub](https://docs.docker.com/docker-hub/quickstart/#step-5-build-and-push-a-container-image-to-docker-hub-from-your-computer)
+
+
+## Step 2: Configure `kubectl` to point at a Kubernetes cluster
+Before you can deploy Dagster, you need to configure `kubectl` to develop against the Kubernetes cluster where you want Dagster to be deployed.
+
+If you are using Docker Desktop and the included Kubernetes server, you will need to create a context first. If you already have a Kubernetes cluster and context created for your Dagster deployment you can skip running this command.
+```bash
+kubectl config set-context dagster --namespace default --cluster docker-desktop --user=docker-desktop
+```
+
+Ensure that `kubectl` is using the correct context by running:
+```bash
+kubectl config use-context
+```
+Where `` is the name of the context you want to use. For example, if you ran the preceding `kubectl config set-context` command, you will run
+```bash
+kubectl config use-context dagster
+```
+
+## Step 3: Add the Dagster Helm chart repository
+
+Dagster publishes [Helm charts](https://artifacthub.io/packages/helm/dagster/dagster) for deploying Dagster, with a new chart for each Dagster version.
+
+To install the Dagster Helm charts, run the following command:
+
+```bash
+helm repo add dagster https://dagster-io.github.io/helm
+```
+
+If you have previously added the Dagster Helm charts, run the following command to update the repository:
+
+```bash
+helm repo update
+```
+
+## Step 4: Configure the Helm chart for your deployment
+
+You will need to modify some values in Dagster's Helm chart to deploy your Dagster project.
+
+### Step 4.1: Copy default Helm chart values into values.yaml
+
+Run the following command to copy the values installed from the published Helm charts:
+
+```bash
+helm show values dagster/dagster > values.yaml
+```
+
+### Step 4.2: Modify the `values.yaml` file for your deployment
+The `values.yaml` file contains configuration options you can set for your deployment. Different configuration options are explained in [inline comments in `values.yaml`](https://artifacthub.io/packages/helm/dagster/dagster?modal=values).
+
+To deploy your project, you'll need to set the following options:
+- `dagster-user-deployments.deployments.name`, which should be a unique name for your deployment
+- `dagster-user-deployments.deployments.image.name` and `dagster-user-deployments.deployments.image.tag`, which should be set to match the Docker image from Step 1
+- `dagster-user-deployments.deployments.dagsterApiGrpcArgs`, which should be set to the arguments you would pass to `dagster api grpc` to [run a gRPC server for your project](/guides/deploy/code-locations/workspace-yaml#grpc-server).
+
+If you are following this guide on your local machine, you will also need to set `pullPolicy: IfNotPresent`. This will use the local version of the image built in Step 1. However, in production use cases when your Docker images are pushed to image registries, this value should remain `pullPolicy: Always`.
+
+
+
+In this example, the image `name` and `tag` are set to `iris_analysis` and `1` to match the image that was pushed in Step 1. To run the gPRC server, the path to the Dagster project needs to be specified, so `--python-file` and `/iris_analysis/definitions.py` are set for `dagsterApiGrpcArgs`.
+
+
+## Step 5: Install the Helm chart
+Now that you have modified the Helm `values.yaml` file, you can install the changes in your Kubernetes cluster.
+
+Run the following command to install the Helm chart and create a [release](https://helm.sh/docs/intro/using_helm/#three-big-concepts).
+
+```bash
+helm upgrade --install dagster dagster/dagster -f /path/to/values.yaml
+```
+
+:::note
+If you want to run an older version of the Dagster system components, like the daemon and webserver, pass the `--version` flag to `helm upgrade` with the version of Dagster you are running. For example, if you want to run version `1.7.4` you'll run the command `helm upgrade --install dagster dagster/dagster -f /path/to/values.yaml --version 1.7.4`
+:::
+
+The `helm upgrade` command will launch several pods in your Kubernetes cluster. You can check the status of the pod with the command:
+
+```bash
+kubectl get pods
+```
+
+It may take a few minutes before all pods are in a `RUNNING` state. If the `helm upgrade` was successful, you should see a `kubectl get pods` output similar to this:
+
+```bash
+$ kubectl get pods
+NAME READY STATUS AGE
+dagster-daemon-5787ccc868-nsvsg 1/1 Running 3m41s
+dagster-webserver-7c5b5c7f5c-rqrf8 1/1 Running 3m41s
+dagster-dagster-user-deployments-iris-analysis-564cbcf9f-fbqlw 1/1 Running 3m41s
+dagster-postgresql-0 1/1 Running 3m41s
+```
+
+
+ Debugging failed pods
+
+If one of the pods is in an error state, you can view the logs using the command
+
+```bash
+kubectl logs
+```
+
+For example, if the pod `dagster-webserver-7c5b5c7f5c-rqrf8` is in a `CrashLoopBackOff` state, the logs can be viewed with the command
+
+```
+kubectl logs dagster-webserver-7c5b5c7f5c-rqrf8
+```
+
+
+
+
+## Step 6: Connect to your Dagster deployment and materialize your assets
+
+### Step 6.1: Start port-forwarding to the webserver pod
+Run the following command to set up port forwarding to the webserver pod:
+
+```bash
+DAGSTER_WEBSERVER_POD_NAME=$(kubectl get pods --namespace default \
+ -l "app.kubernetes.io/name=dagster,app.kubernetes.io/instance=dagster,component=dagster-webserver" \
+ -o jsonpath="{.items[0].metadata.name}")
+kubectl --namespace default port-forward $DAGSTER_WEBSERVER_POD_NAME 8080:80
+```
+
+This command gets the full name of the `webserver` pod from the output of `kubectl get pods`, and then sets up port forwarding with the `kubectl port-forward` command.
+
+### Step 6.2: Visit your Dagster deployment
+
+The webserver has been port-forwarded to `8080`, so you can visit the Dagster deployment by going to [http://127.0.0.1:8080](http://127.0.0.1:8080). You should see the Dagster landing page
+
+![Screenshot of Dagster landing page](/images/guides/deploy/kubernetes/dagster-kubernetes-deploy.png)
+
+### Step 6.3: Materialize an asset
+In the Dagster UI, navigate to the Asset catalog and click the **Materialize** button to materialize an asset. Dagster will start a Kubernetes job to materialize the asset. You can introspect on the Kubernetes cluster to see this job:
+
+```bash
+$ kubectl get jobs
+NAME COMPLETIONS DURATION AGE
+dagster-run-5ee8a0b3-7ca5-44e6-97a6-8f4bd86ee630 1/1 4s 11s
+```
+
+## Next steps
+- Forwarding Dagster logs from a Kubernetes deployment to AWS, Azure, GCP
+- Other configuration options for K8s deployment - secrets,
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/index.md
new file mode 100644
index 0000000000000..a42f841d4b1e6
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/index.md
@@ -0,0 +1,8 @@
+---
+title: "Kubernetes"
+sidebar_position: 30
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/kubernetes-and-celery.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/kubernetes-and-celery.md
new file mode 100644
index 0000000000000..9dbe97bfe4158
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/kubernetes-and-celery.md
@@ -0,0 +1,177 @@
+---
+title: "Using Celery with Kubernetes"
+sidebar_position: 300
+---
+
+In addition to using the `k8s_job_executor` to run each op in its own Kubernetes job, Dagster also allows you to use Celery to limit the number of ops that can concurrently connect to a resource across all running Dagster jobs.
+
+In this section, we demonstrate how to extend the [previous Helm deployment guide](/guides/deploy/deployment-options/kubernetes/deploying-to-kubernetes) to support that use case, by deploying a more complex configuration of Dagster, which utilizes the and .
+
+## Prerequisites
+
+In addition to the [previous prerequisites](deploying-to-kubernetes), this article assumes familiarity with [Celery, a distributed task queue system](https://docs.celeryq.dev/).
+
+## Deployment architecture
+
+{/* https://excalidraw.com/#json=4680957890134016,q6NWURUuPP_VThmbRQ89Jg */}
+![dagster-kubernetes-advanced-architecture.png](/images/guides/deploy/kubernetes/dagster-kubernetes-advanced-architecture.png)
+
+### Components
+
+| Component name | Type | Image |
+|----------------|------|-------|
+| Celery | [Deployment](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) | [dagster/dagster-celery-k8s](https://hub.docker.com/r/dagster/dagster-celery-k8s) _(Released weekly)_ |
+| Daemon | [Deployment](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) | [dagster/dagster-celery-k8s](https://hub.docker.com/r/dagster/dagster-celery-k8s) _(Released weekly)_ |
+| Dagster webserver | [Deployment](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) behind a [Service](https://kubernetes.io/docs/concepts/services-networking/service/) | [dagster/dagster-celery-k8s](https://hub.docker.com/r/dagster/dagster-celery-k8s) _(Released weekly)_ |
+| Database | PostgreSQL | [postgres](https://hub.docker.com/_/postgres) _(Optional)_ |
+| Flower _(Optional)_ | [Deployment](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) behind a [Service](https://kubernetes.io/docs/concepts/services-networking/service/) | [mher/flower](https://hub.docker.com/r/mher/flower) |
+| Run worker | [Job](https://kubernetes.io/docs/concepts/workloads/controllers/job/) | User-provided or [dagster/user-code-example](https://hub.docker.com/r/dagster/user-code-example) _(Released weekly)_ |
+| Step job | [Job](https://kubernetes.io/docs/concepts/workloads/controllers/job/) | User-provided or [dagster/user-code-example](https://hub.docker.com/r/dagster/user-code-example) _(Released weekly)_ |
+| User code deployment | [Deployment](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) behind a [Service](https://kubernetes.io/docs/concepts/services-networking/service/) | User-provided or [dagster/user-code-example](https://hub.docker.com/r/dagster/user-code-example) _(Released weekly)_ |
+
+The Helm chart can be configured to use this architecture by configuring the `runLauncher.type` field in your `values.yaml` file to be `CeleryK8sRunLauncher` instead of the default `K8sRunLauncher`. The resulting architecture is similar to the architecture described in the [Helm deployment guide](deploying-to-kubernetes), with the following changes:
+
+### Celery
+
+Users can configure multiple Celery queues (for example, one celery queue for each resource the user would like to limit) and multiple Celery workers per queue via the `runLauncher.config.celeryK8sRunLauncher.workerQueues` section of `values.yaml`.
+
+The Celery workers poll for new Celery tasks and execute each task in order of receipt or priority. The Celery task largely consists of launching an ephemeral Kubernetes step worker to execute that step.
+
+### Daemon
+
+The daemon now launches runs using the .
+
+### Run worker
+
+The run worker is still responsible for traversing the execution plan, but now uses the `celery_k8s_job_executor` to submit steps that are ready to be executed to the corresponding Celery queue (instead of executing the step itself).
+
+All jobs being executed on an instance that uses the `CeleryK8sRunLauncher` must have the `celery_k8s_job_executor` set in the `executor_def` field.
+
+### Step worker
+
+The step worker is responsible for executing a single step, writing the structured events to the database. The Celery worker polls for the step worker completion.
+
+### Flower
+
+[Flower](https://flower.readthedocs.io/en/latest/) is an optional component that can be useful for monitoring Celery queues and workers.
+
+## Walkthrough
+
+We assume that you've followed the initial steps in the [previous walkthrough](deploying-to-kubernetes) by building your Docker image for your user code, pushing it to a registry, adding the Dagster Helm chart repository, and configuring your Helm User Deployment values. Note that any job that you wish to run in an instance using the `CeleryK8sRunLauncher` must be using the `celery_k8s_job_executor` as its executor. The [example user code repository](https://github.com/dagster-io/dagster/tree/master/examples/deploy_k8s/example_project) includes an example job that uses the `celery_k8s_job_executor`, called `pod_per_op_celery_job`.
+
+### Configure persistent object storage
+
+We need to configure persistent object storage so that data can be serialized and passed between steps. To run the Dagster User Code example, create a S3 bucket named "dagster-test".
+
+To enable Dagster to connect to S3, provide `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` environment variables via the `env`, `envConfigMaps`, or `envSecrets` fields under `dagster-user-deployments` in `values.yaml` or (not recommended) by setting these variables directly in the User Code Deployment image.
+
+### Install the Dagster Helm chart with Celery
+
+Install the Helm chart and create a release, with the run launcher configured to use the `CeleryK8sRunLauncher`. The helm chart checks for this run launcher type and includes Celery infrastructure in your cluster. Below, we've named our release `dagster`. We use `helm upgrade --install` to create the release if it does not exist; otherwise, the existing `dagster` release will be modified:
+
+```
+ helm upgrade --install dagster dagster/dagster -f /path/to/values.yaml \
+ --set runLauncher.type=CeleryK8sRunLauncher \
+ --set rabbitmq.enabled=true
+```
+
+Helm will launch several pods. You can check the status of the installation with `kubectl`. If everything worked correctly, you should see output like the following:
+
+```
+ $ kubectl get pods
+ NAME READY STATUS RESTARTS AGE
+ dagster-celery-workers-74886cfbfb-m9cbc 1/1 Running 1 3m42s
+ dagster-daemon-68c4b8d68d-vvpls 1/1 Running 1 3m42s
+ dagster-webserver-69974dd75b-5m8gg 1/1 Running 0 3m42s
+ dagster-k8s-example-user-code-1-88764b4f4-25mbd 1/1 Running 0 3m42s
+ dagster-postgresql-0 1/1 Running 0 3m42s
+ dagster-rabbitmq-0 1/1 Running 0 3m42s
+```
+
+### Run a job in your deployment
+
+After Helm has successfully installed all the required kubernetes resources, start port forwarding to the webserver pod via:
+
+```
+ DAGSTER_WEBSERVER_POD_NAME=$(kubectl get pods --namespace default \
+ -l "app.kubernetes.io/name=dagster,app.kubernetes.io/instance=dagster,component=webserver" \
+ -o jsonpath="{.items[0].metadata.name}")
+ kubectl --namespace default port-forward $DAGSTER_WEBSERVER_POD_NAME 8080:80
+```
+
+Visit [http://127.0.0.1:8080](http://127.0.0.1:8080), and navigate to the [launchpad](http://127.0.0.1:8080/workspace/example_repo@k8s-example-user-code-1/jobs/pod_per_op_celery_job/launchpad). Notice how `resources.io_manager.config.s3_bucket` is set to `dagster-test`. You can replace this string with any other accessible S3 bucket. Then, click _Launch Run_.
+
+You can introspect the jobs that were launched with `kubectl`:
+
+```
+ $ kubectl get jobs
+ NAME COMPLETIONS DURATION AGE
+ dagster-step-9f5c92d1216f636e0d33877560818840 1/1 5s 12s
+ dagster-step-a1063317b9aac91f42ca9eacec551b6f 1/1 12s 34s
+ dagster-run-fb6822e5-bf43-476f-9e6c-6f9896cf3fb8 1/1 37s 37s
+```
+
+`dagster-step-` entries correspond to step workers and `dagster-run-` entries correspond to run workers.
+
+Within the Dagster UI, you can watch the job as it executes.
+
+### Configuring Celery queues
+
+Users can configure multiple Celery queues (for example, one queue for each resource to be limited) and multiple Celery workers per queue via the `runLauncher.config.celeryK8sRunLauncher.workerQueues` section of `values.yaml`.
+
+To use the queues, `dagster-celery/queue` can be set on op tags.
+
+By default, all ops will be sent to the default Celery queue named `dagster`.
+
+```python
+@op(
+ tags = {
+ 'dagster-celery/queue': 'snowflake_queue',
+ }
+)
+def my_op(context):
+ context.log.info('running')
+```
+
+### Celery priority
+
+Users can set `dagster-celery/run_priority` on job tags to configure the baseline priority of all ops from that job. To set priority at the op level, users can set `dagster-celery/priority` on the op tags. When priorities are set on both a job and an op, the sum of both priorities will be used.
+
+```python
+@op(
+ tags = {
+ 'dagster-celery/priority': 2,
+ }
+)
+def my_op(context):
+ context.log.info('running')
+
+@job(
+ tags = {
+ 'dagster-celery/run_priority': 3,
+ }
+)
+def my_job():
+ my_op()
+```
+
+### Configuring an External Message Broker
+
+In a real deployment, users will likely want to set up an external message broker like Redis rather than RabbitMQ, which can be done by configuring `rabbitmq` and `redis` sections of `values.yaml`.
+
+```yaml
+rabbitmq:
+ enabled: false
+
+redis:
+ enabled: true
+ internal: false
+ host: "redisHost"
+ port: 6379
+ brokerDbNumber: 0
+ backendDbNumber: 0
+```
+
+## Conclusion
+
+We deployed Dagster, configured with the , onto a Kubernetes cluster using Helm.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/migrating-while-upgrading.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/migrating-while-upgrading.md
new file mode 100644
index 0000000000000..bda569862e11b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/kubernetes/migrating-while-upgrading.md
@@ -0,0 +1,109 @@
+---
+title: "Migrating a Dagster instance while upgrading Dagster in a Kubernetes environment"
+description: We walk through how to migrate your Dagster instance using a Kubernetes Job from the Helm chart.
+---
+
+When upgrading your Dagster version, you may also need to migrate your Dagster instance. Migrations will only be required if you are upgrading your minor version.
+
+In this guide, we'll walk you through the migration process in a Kubernetes environment.
+
+## Prerequisites
+
+Before you get started, complete the following:
+
+- **Check the [Dagster migration guide](/guides/migrate/version-migration).** Use the guide to verify any additional steps you may need to take to upgrade your Dagster version.
+- **Back up your PostgreSQL database.**
+
+## Step 1: Upgrade the Helm chart
+
+:::note
+
+This article assumes only one release of the Dagster Helm chart is installed in your Kubernetes cluster. All commands should be run in the proper namespace.
+
+:::
+
+1. First, [get the latest information about available charts](https://helm.sh/docs/helm/helm_repo_update/) by running:
+
+ ```shell
+ helm repo update
+ ```
+
+ This retrieves the latest information about the Dagster Helm chart, which is updated with every Open Source (OSS) release.
+
+2. Run [`helm upgrade`](https://helm.sh/docs/helm/helm_upgrade/) with your desired Dagster chart version and Helm values:
+
+ ```shell
+ helm upgrade --install dagster dagster/dagster -f /path/to/values.yaml
+ ```
+
+ This installs the new Helm chart and creates a release if it doesn't exist. If it does, the existing `dagster` release will be modified.
+
+## Step 2: Scale down Dagster
+
+Next, you'll scale down the Dagster webserver and daemon deployments. This ensures that there aren't any ongoing job runs writing to the database as the migration happens.
+
+As you scale down the deployments, note each deployment's replica count. You'll use this to scale each one back up after the migration is complete.
+
+```shell
+# Get the names of Dagster's webserver and daemon deployments created by Helm
+export WEBSERVER_DEPLOYMENT_NAME=`kubectl get deploy \
+ --selector=component=dagster-webserver -o jsonpath="{.items[0].metadata.name}"`
+export DAEMON_DEPLOYMENT_NAME=`kubectl get deploy \
+ --selector=component=dagster-daemon -o jsonpath="{.items[0].metadata.name}"`
+
+# Save each deployment's replica count to scale back up after migrating
+export WEBSERVER_DEPLOYMENT_REPLICA_COUNT=`kubectl get deploy \
+ --selector=component=dagster-webserver -o jsonpath="{.items[0].status.replicas}"`
+export DAEMON_DEPLOYMENT_REPLICA_COUNT=`kubectl get deploy \
+ --selector=component=dagster-daemon -o jsonpath="{.items[0].status.replicas}"`
+
+# Scale down the Deployments
+kubectl scale deploy $WEBSERVER_DEPLOYMENT_NAME --replicas=0
+kubectl scale deploy $DAEMON_DEPLOYMENT_NAME --replicas=0
+```
+
+## Step 3: Migrate the Dagster instance
+
+1. Run a Kubernetes job with the `dagster instance migrate` command. In the following example, `helm template` will run a job that uses this command and migrates the instance:
+
+ ```shell
+ # Run `helm list` and save your Dagster Helm release name
+ export HELM_DAGSTER_RELEASE_NAME=
+
+ # The `helm template` command must be run from the directory containing the
+ # `values.yaml` file you used to install the Dagster Helm chart.
+ #
+ # If needed, you can retrieve the currently applied `values.yaml` file
+ # from the cluster by running:
+ #
+ # `helm get values $HELM_DAGSTER_RELEASE_NAME > values.yaml`
+ #
+ helm template $HELM_DAGSTER_RELEASE_NAME dagster/dagster \
+ --set "migrate.enabled=true" \
+ --show-only templates/job-instance-migrate.yaml \
+ --values values.yaml \
+ | kubectl apply -f -
+ ```
+
+2. Next, you'll check that the migration succeeded. Run the following to get the name of the pod:
+
+ ```shell
+ kubectl get pods -l job-name=dagster-instance-migrate
+ ```
+
+3. Lastly, run the following to inspect the resulting pod, replacing `` with the name of the pod:
+
+ ```shell
+ kubectl describe pod
+ ```
+
+ Verify that the pod completed without any errors before proceeding.
+
+## Step 4: Scale Dagster back up
+
+The last step is to scale the Dagster webserver and daemon deployments back up. Run the following commands:
+
+```shell
+kubectl scale deploy $WEBSERVER_DEPLOYMENT_NAME --replicas=$WEBSERVER_DEPLOYMENT_REPLICA_COUNT
+kubectl scale deploy $DAEMON_DEPLOYMENT_NAME --replicas=$DAEMON_DEPLOYMENT_REPLICA_COUNT
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/running-dagster-locally.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/running-dagster-locally.md
new file mode 100644
index 0000000000000..a38a3cd106dc5
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/deployment-options/running-dagster-locally.md
@@ -0,0 +1,87 @@
+---
+title: "Running Dagster locally"
+sidebar_label: Local deployment
+description: How to run Dagster on your local machine.
+sidebar_position: 1
+---
+
+In this guide, we'll walk you through how to run Dagster on your local machine using the `dagster dev` command. The `dagster dev` command launches the Dagster UI and the Dagster daemon, allowing you to start a full deployment of Dagster from the command line.
+
+:::warning
+`dagster dev` is intended for local development _only_. If you want to run Dagster for production use cases, see our other [deployment guides](/guides/deploy/deployment-options/index.md).
+:::
+
+## Locating your code
+
+Before starting local development, you need to tell Dagster how to find the Python code containing your assets and jobs.
+
+For a refresher on how to set up a Dagster project, follow our [Recommended Dagster Project Structure](/guides/build/projects/structuring-your-dagster-project) guide.
+
+
+
+ Dagster can load Python modules as code locations.
+
+ We can use the `-m` argument to supply the name of the module to start a Dagster instance loaded with our definitions:
+ ```shell
+ dagster dev -m my_module
+ ```
+
+
+
+ To load definitions from a module without supplying the `-m` command line argument, you can use a `pyproject.toml` file. This file, included in all Dagster example projects, contains a `tool.dagster` section where you can supply the `module_name`:
+
+
+
+
+
+ Dagster can load a file directly as a code location.
+
+ Given the preceding file, we can use the `-f` argument to supply the name of the file to start a Dagster instance loaded with our definitions:
+ ```shell
+ dagster dev -f defs.py
+ ```
+
+ :::note
+ We don't recommend using the `-f` argument for production deployments, to avoid a whole class of Python import errors.
+ :::
+
+
+
+
+## Creating a persistent instance
+
+Running `dagster dev` without any additional configuration starts an ephemeral instance in a temporary directory. You may see log output indicating as such:
+```shell
+Using temporary directory /Users/rhendricks/tmpqs_fk8_5 for storage.
+```
+This indicates that any runs or materialized assets created during your session won't be persisted once the session ends.
+
+To designate a more permanent home for your runs and assets, you can set the `DAGSTER_HOME` environment variable to a folder on your filesystem. Dagster will then use the specified folder for storage on all subsequent runs of `dagster dev`.
+
+```shell
+mkdir -p ~/.dagster_home
+export DAGSTER_HOME=~/.dagster_home
+dagster dev
+```
+
+## Configuring your instance
+
+To configure your Dagster instance, you can create a `dagster.yaml` file in your `$DAGSTER_HOME` folder.
+
+For example, to have your local instance limit the number of concurrent runs, you could configure the following `dagster.yaml`:
+
+
+
+For the full list of options that can be set in the `dagster.yaml` file, refer to the [Dagster instance documentation](/guides/deploy/dagster-instance-configuration).
+
+## Detecting when you're running in `dagster dev`
+
+You may want to detect whether you're running locally. For example, you might want schedules or sensors to start in the `RUNNING` state in production but not in your local test deployment.
+
+`dagster dev` sets the environment variable `DAGSTER_IS_DEV_CLI` to `1`. You can detect whether you're in a local dev environment by checking for the presence of that environment variable.
+
+## Moving to production
+
+`dagster dev` is primarily useful for running Dagster for local development and testing. It isn't suitable for the demands of most production deployments. Most importantly, `dagster dev` does not include authentication or web security. Additionally, in a production deployment, you might want to run multiple webserver replicas, have zero downtime continuous deployment of your code, or set up your Dagster daemon to automatically restart if it crashes.
+
+For information about deploying Dagster in production, see the [Dagster Open Source deployment options documentation](/guides/deploy/deployment-options/).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/dev-to-prod.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/dev-to-prod.md
new file mode 100644
index 0000000000000..77ccedb204504
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/dev-to-prod.md
@@ -0,0 +1,416 @@
+---
+title: "Transitioning from development to production"
+sidebar_position: 500
+---
+
+In this article, we'll walk through how to transition your data pipelines from local development to staging and production deployments.
+
+Let's say we’ve been tasked with fetching the N most recent entries from Hacker News and splitting the data into two datasets: one containing all of the data about stories and one containing all of the data about comments. In order to make the pipeline maintainable and testable, we have two additional requirements:
+
+- We must be able to run our data pipeline in local, staging, and production environments.
+- We need to be confident that data won't be accidentally overwritten (for example, because a user forgot to change a configuration value).
+
+Using a few Dagster concepts, we can easily tackle this task! Here’s an overview of the main concepts we’ll be using in this guide:
+
+- [Assets](/guides/build/assets/) - An asset is a software object that models a data asset. The prototypical example is a table in a database or a file in cloud storage.
+- [Resources](/guides/build/external-resources) - A resource is an object that models a connection to a (typically) external service. Resources can be shared between assets, and different implementations of resources can be used depending on the environment. For example, a resource may provide methods to send messages in Slack.
+- [I/O managers](/guides/build/io-managers/) - An I/O manager is a special kind of resource that handles storing and loading assets. For example, if we wanted to store assets in S3, we could use Dagster’s built-in S3 I/O manager.
+- [Run config](/guides/operate/configuration/run-configuration) - Assets and resources sometimes require configuration to set certain values, like the password to a database. Run config allows you to set these values at run time. In this guide, we will also use an API to set some default run configuration.
+
+Using these Dagster concepts we will:
+
+- Write three assets: the full Hacker News dataset, data about comments, and data about stories.
+- Use Dagster's Snowflake I/O manager to store the datasets in [Snowflake](https://www.snowflake.com/).
+- Set up our Dagster code so that the configuration for the Snowflake I/O manager is automatically supplied based on the environment where the code is running.
+
+## Setup
+
+{/* TODO turn back into once that's implemented */}
+
+:::tip
+
+You can find the code for this example on [GitHub](https://github.com/dagster-io/dagster/tree/1.9.8/examples/development_to_production/).
+
+:::
+
+To follow along with this guide, you can copy the full code example and install a few additional pip libraries:
+
+```bash
+dagster project from-example --name my-dagster-project --example development_to_production
+cd my-dagster-project
+pip install -e .
+```
+
+## Part one: Local development
+
+In this section we will:
+
+- Write our assets
+- Add run configuration for the Snowflake I/O manager
+- Materialize assets in the Dagster UI
+
+Let’s start by writing our three assets. We'll use Pandas DataFrames to interact with the data.
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/assets.py startafter=start_assets endbefore=end_assets
+# assets.py
+import pandas as pd
+import requests
+
+from dagster import Config, asset
+
+
+class ItemsConfig(Config):
+ base_item_id: int
+
+
+@asset(
+ io_manager_key="snowflake_io_manager",
+)
+def items(config: ItemsConfig) -> pd.DataFrame:
+ """Items from the Hacker News API: each is a story or a comment on a story."""
+ rows = []
+ max_id = requests.get(
+ "https://hacker-news.firebaseio.com/v0/maxitem.json", timeout=5
+ ).json()
+ # Hacker News API is 1-indexed, so adjust range by 1
+ for item_id in range(max_id - config.base_item_id + 1, max_id + 1):
+ item_url = f"https://hacker-news.firebaseio.com/v0/item/{item_id}.json"
+ rows.append(requests.get(item_url, timeout=5).json())
+
+ # ITEM_FIELD_NAMES is a list of the column names in the Hacker News dataset
+ result = pd.DataFrame(rows, columns=ITEM_FIELD_NAMES).drop_duplicates(subset=["id"])
+ result.rename(columns={"by": "user_id"}, inplace=True)
+ return result
+
+
+@asset(
+ io_manager_key="snowflake_io_manager",
+)
+def comments(items: pd.DataFrame) -> pd.DataFrame:
+ """Comments from the Hacker News API."""
+ return items[items["type"] == "comment"]
+
+
+@asset(
+ io_manager_key="snowflake_io_manager",
+)
+def stories(items: pd.DataFrame) -> pd.DataFrame:
+ """Stories from the Hacker News API."""
+ return items[items["type"] == "story"]
+```
+
+Now we can add these assets to our object and materialize them via the UI as part of our local development workflow. We can pass in credentials to our `SnowflakePandasIOManager`.
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/repository/repository_v1.py startafter=start endbefore=end
+# definitions.py
+from dagster_snowflake_pandas import SnowflakePandasIOManager
+
+from dagster import Definitions
+from development_to_production.assets.hacker_news_assets import comments, items, stories
+
+# Note that storing passwords in configuration is bad practice. It will be resolved later in the guide.
+resources = {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ account="abc1234.us-east-1",
+ user="me@company.com",
+ # password in config is bad practice
+ password="my_super_secret_password",
+ database="LOCAL",
+ schema="ALICE",
+ ),
+}
+
+defs = Definitions(assets=[items, comments, stories], resources=resources)
+```
+
+Note that we have passwords in our configuration in this code snippet. This is bad practice, and we will resolve it shortly.
+
+This results in an asset graph that looks like this:
+
+![Hacker News asset graph](/images/guides/deploy/hacker_news_asset_graph.png)
+
+We can materialize the assets in the UI and ensure that the data appears in Snowflake as we expect:
+
+![Snowflake data](/images/guides/deploy/snowflake_data.png)
+
+While we define our assets as Pandas DataFrames, the Snowflake I/O manager automatically translates the data to and from Snowflake tables. The Python asset name determines the Snowflake table name. In this case three tables will be created: `ITEMS`, `COMMENTS` and `STORIES`.
+
+## Part two: Deployment
+
+In this section we will:
+
+- Modify the configuration for the Snowflake I/O manager to handle staging and production environments
+- Discuss different options for managing a staging environment
+
+Now that our assets work locally, we can start the deployment process! We'll first set up our assets for production, and then discuss the options for our staging deployment.
+
+### Production
+
+We want to store the assets in a production Snowflake database, so we need to update the configuration for the `SnowflakePandasIOManager`. But if we were to simply update the values we set for local development, we would run into an issue: the next time a developer wants to work on these assets, they will need to remember to change the configuration back to the local values. This leaves room for a developer to accidentally overwrite the production asset during local development.
+
+Instead, we can determine the configuration for resources based on the environment:
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/repository/repository_v2.py startafter=start endbefore=end
+# definitions.py
+
+# Note that storing passwords in configuration is bad practice. It will be resolved soon.
+resources = {
+ "local": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ account="abc1234.us-east-1",
+ user="me@company.com",
+ # password in config is bad practice
+ password="my_super_secret_password",
+ database="LOCAL",
+ schema="ALICE",
+ ),
+ },
+ "production": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ account="abc1234.us-east-1",
+ user="dev@company.com",
+ # password in config is bad practice
+ password="company_super_secret_password",
+ database="PRODUCTION",
+ schema="HACKER_NEWS",
+ ),
+ },
+}
+deployment_name = os.getenv("DAGSTER_DEPLOYMENT", "local")
+
+defs = Definitions(
+ assets=[items, comments, stories], resources=resources[deployment_name]
+)
+```
+
+Note that we still have passwords in our configuration in this code snippet. This is bad practice, and we will resolve it next.
+
+Now, we can set the environment variable `DAGSTER_DEPLOYMENT=production` in our deployment and the correct resources will be applied to the assets.
+
+We still have some problems with this setup:
+
+1. Developers need to remember to change `user` and `password` to their credentials and `schema` to their name when developing locally.
+2. Passwords are being stored in code.
+
+We can easily solve these problems using , which lets us source configuration for resources from environment variables. This allows us to store Snowflake configuration values as environment variables and point the I/O manager to those environment variables:
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/repository/repository_v3.py startafter=start endbefore=end
+# definitions.py
+
+
+resources = {
+ "local": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ account="abc1234.us-east-1",
+ user=EnvVar("DEV_SNOWFLAKE_USER"),
+ password=EnvVar("DEV_SNOWFLAKE_PASSWORD"),
+ database="LOCAL",
+ schema=EnvVar("DEV_SNOWFLAKE_SCHEMA"),
+ ),
+ },
+ "production": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ account="abc1234.us-east-1",
+ user="system@company.com",
+ password=EnvVar("SYSTEM_SNOWFLAKE_PASSWORD"),
+ database="PRODUCTION",
+ schema="HACKER_NEWS",
+ ),
+ },
+}
+deployment_name = os.getenv("DAGSTER_DEPLOYMENT", "local")
+
+defs = Definitions(
+ assets=[items, comments, stories], resources=resources[deployment_name]
+)
+```
+
+### Staging
+
+Depending on your organization’s Dagster setup, there are a couple of options for a staging environment.
+
+- **For Dagster+ users**, we recommend using [branch deployments](/dagster-plus/features/ci-cd/branch-deployments/) as your staging step. A branch deployment is a new Dagster deployment that is automatically generated for each git branch, and can be used to verify data pipelines before deploying them to production.
+
+- **For a self-hosted staging deployment**, we’ve already done most of the necessary work to run our assets in staging! All we need to do is add another entry to the `resources` dictionary and set `DAGSTER_DEPLOYMENT=staging` in our staging deployment.
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/repository/repository_v3.py startafter=start_staging endbefore=end_staging
+resources = {
+ "local": {...},
+ "production": {...},
+ "staging": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ account="abc1234.us-east-1",
+ user="system@company.com",
+ password=EnvVar("SYSTEM_SNOWFLAKE_PASSWORD"),
+ database="STAGING",
+ schema="HACKER_NEWS",
+ ),
+ },
+}
+```
+
+## Advanced: Unit tests with stubs and mocks
+
+You may have noticed a missing step in the development workflow presented in this guide — unit tests! While the main purpose of the guide is to help you transition your code from local development to a production deployment, unit testing is still an important part of the development cycle. In this section, we'll explore a pattern you may find useful when writing your own unit tests.
+
+When we write unit tests for the `items` asset, we could make more precise assertions if we knew exactly what data we'd receive from Hacker News. If we refactor our interactions with the Hacker News API as a resource, we can leverage Dagster's resource system to provide a stub resource in our unit tests.
+
+Before we get into implementation, let's go over some best practices:
+
+### When to use resources
+
+In many cases, interacting with an external service directly in assets or ops is more convenient than refactoring the interactions with the service as a resource. We recommend refactoring code to use resources in the following cases:
+
+- Multiple assets or ops need to interact with the service in a consistent way
+- Different implementations of a service need to be used in certain scenarios (ie. a staging environment, or unit tests)
+
+### When to use stub resources
+
+Determining when it makes sense to stub a resource for a unit test can be a topic of much debate. There are certainly some resources where it would be too complicated to write and maintain a stub. For example, it would be difficult to mock a database like Snowflake with a lightweight database since the SQL syntax and runtime behavior may vary. In general, if a resource is relatively simple, writing a stub can be helpful for unit testing the assets and ops that use the resource.
+
+We'll start by writing the "real" Hacker News API Client:
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/resources/resources_v1.py startafter=start_resource endbefore=end_resource
+# resources.py
+from typing import Any, Dict, Optional
+
+import requests
+
+from dagster import ConfigurableResource
+
+
+class HNAPIClient(ConfigurableResource):
+ """Hacker News client that fetches live data."""
+
+ def fetch_item_by_id(self, item_id: int) -> Optional[dict[str, Any]]:
+ """Fetches a single item from the Hacker News API by item id."""
+ item_url = f"https://hacker-news.firebaseio.com/v0/item/{item_id}.json"
+ item = requests.get(item_url, timeout=5).json()
+ return item
+
+ def fetch_max_item_id(self) -> int:
+ return requests.get(
+ "https://hacker-news.firebaseio.com/v0/maxitem.json", timeout=5
+ ).json()
+
+ @property
+ def item_field_names(self) -> list:
+ # omitted for brevity, see full code example for implementation
+ return []
+```
+
+We'll also need to update the `items` asset to use this client as a resource:
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/assets_v2.py startafter=start_items endbefore=end_items
+# assets.py
+
+
+class ItemsConfig(Config):
+ base_item_id: int
+
+
+@asset(
+ io_manager_key="snowflake_io_manager",
+)
+def items(config: ItemsConfig, hn_client: HNAPIClient) -> pd.DataFrame:
+ """Items from the Hacker News API: each is a story or a comment on a story."""
+ max_id = hn_client.fetch_max_item_id()
+ rows = []
+ # Hacker News API is 1-indexed, so adjust range by 1
+ for item_id in range(max_id - config.base_item_id + 1, max_id + 1):
+ rows.append(hn_client.fetch_item_by_id(item_id))
+
+ result = pd.DataFrame(rows, columns=hn_client.item_field_names).drop_duplicates(
+ subset=["id"]
+ )
+ result.rename(columns={"by": "user_id"}, inplace=True)
+ return result
+```
+
+:::note
+
+For the sake of brevity, we've omitted the implementation of the property `item_field_names` in `HNAPIClient`. You can find the full implementation of this resource in the [full code example](https://github.com/dagster-io/dagster/tree/master/examples/development_to_production) on GitHub.
+
+:::
+
+We'll also need to add an instance of `HNAPIClient` to `resources` in our `Definitions` object.
+
+```python file=/guides/dagster/development_to_production/repository/repository_v3.py startafter=start_hn_resource endbefore=end_hn_resource
+resource_defs = {
+ "local": {"hn_client": HNAPIClient(), "snowflake_io_manager": {...}},
+ "production": {"hn_client": HNAPIClient(), "snowflake_io_manager": {...}},
+ "staging": {"hn_client": HNAPIClient(), "snowflake_io_manager": {...}},
+}
+```
+
+Now we can write a stubbed version of the Hacker News resource. We want to make sure the stub has implementations for each method `HNAPIClient` implements.
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/resources/resources_v2.py startafter=start_mock endbefore=end_mock
+# resources.py
+
+
+class StubHNClient:
+ """Hacker News Client that returns fake data."""
+
+ def __init__(self):
+ self.data = {
+ 1: {
+ "id": 1,
+ "type": "comment",
+ "title": "the first comment",
+ "by": "user1",
+ },
+ 2: {"id": 2, "type": "story", "title": "an awesome story", "by": "user2"},
+ }
+
+ def fetch_item_by_id(self, item_id: int) -> Optional[dict[str, Any]]:
+ return self.data.get(item_id)
+
+ def fetch_max_item_id(self) -> int:
+ return 2
+
+ @property
+ def item_field_names(self) -> list:
+ return ["id", "type", "title", "by"]
+```
+
+:::note
+
+Since the stub Hacker News resource and the real Hacker News resource need to implement the same methods, this would be a great time to write an interface. We’ll skip the implementation in this guide, but you can find it in the [full code example](https://github.com/dagster-io/dagster/tree/master/examples/development_to_production).
+
+:::
+
+Now we can use the stub Hacker News resource to test that the `items` asset transforms the data in the way we expect:
+
+{/* TODO convert to */}
+```python file=/guides/dagster/development_to_production/test_assets.py startafter=start endbefore=end
+# test_assets.py
+
+
+def test_items():
+ hn_dataset = items(
+ config=ItemsConfig(base_item_id=StubHNClient().fetch_max_item_id()),
+ hn_client=StubHNClient(),
+ )
+ assert isinstance(hn_dataset, pd.DataFrame)
+
+ expected_data = pd.DataFrame(StubHNClient().data.values()).rename(
+ columns={"by": "user_id"}
+ )
+
+ assert (hn_dataset == expected_data).all().all()
+```
+
+:::note
+
+While we focused on assets in this article, the same concepts and APIs can be used to swap out run configuration for jobs.
+
+:::
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/celery.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/celery.md
new file mode 100644
index 0000000000000..b442048acc3a0
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/celery.md
@@ -0,0 +1,166 @@
+---
+title: "Executing Dagster on Celery"
+sidebar_position: 700
+---
+
+[Celery](https://docs.celeryq.dev/) is an open-source Python distributed task queue system, with support for a variety of queues (brokers) and result persistence strategies (backends).
+
+The `dagster-celery` executor uses Celery to satisfy three common requirements when running jobs in production:
+
+- Parallel execution capacity that scales horizontally across multiple compute nodes.
+- Separate queues to isolate execution and control external resource usage at the op level.
+- Priority-based execution at the op level.
+
+The dagster-celery executor compiles a job and its associated configuration into a concrete execution plan, and then submits each execution step to the broker as a separate Celery task. The dagster-celery workers then pick up tasks from the queues to which they are subscribed, according to the priorities assigned to each task, and execute the steps to which the tasks correspond.
+
+## Prerequisites
+
+To complete the steps in this guide, you'll need to install `dagster` and `dagster-celery`:
+
+ ```shell
+ pip install dagster dagster-celery
+ ```
+
+- You will also need **a running broker**, which is required to run the Celery executor. Refer to the [Celery documentation](https://docs.celeryq.dev/en/stable/getting-started/first-steps-with-celery.html#choosing-a-broker) for more info about choosing a broker.
+
+## Part 1: Write and execute a job
+
+To demonstrate, we'll start by constructing a parallel toy job that uses the Celery executor.
+
+In your Dagster project, create a new file named `celery_job.py` and paste in the following:
+
+{/* TODO convert to */}
+```python file=/deploying/celery_job.py
+from dagster_celery import celery_executor
+
+from dagster import job, op
+
+
+@op
+def not_much():
+ return
+
+
+@job(executor_def=celery_executor)
+def parallel_job():
+ for i in range(50):
+ not_much.alias("not_much_" + str(i))()
+```
+
+Now, run the Celery executor. In our case, we're running RabbitMQ as our broker. With Docker, this is something like the following:
+
+```shell
+docker run -p 5672:5672 rabbitmq:3.8.2
+```
+
+You'll also need to run a Celery worker to execute tasks. From the same directory where you saved the `celery_job.py` file, run:
+
+```shell
+dagster-celery worker start -A dagster_celery.app
+```
+
+Next, execute this job with Celery by running the following:
+
+```shell
+dagster dev -f celery_job.py
+```
+
+Now you can execute the parallel job from the [Dagster UI](/guides/operate/webserver).
+
+## Part 2: Ensuring workers are in sync
+
+In Part 1, we took a few shortcuts:
+
+- **We ran a single Celery worker on the same node as the Dagster webserver.** This allowed us to share local ephemeral run storage and event log storage between them and use the filesystem I/O manager to exchange values between the worker's task executions.
+- **We ran the Celery worker in the same directory as `celery_job.py`**. This meant that our Dagster code was available to both the webserver and the worker, specifically that they could both find the job definition in the same file (`-f celery_job.py`)
+
+In production, more configuration is required.
+
+### Step 1: Configure persistent run and event log storage
+
+First, configure appropriate persistent run and event log storage, e.g., `PostgresRunStorage` and `PostgresEventLogStorage` on your [Dagster instance](/guides/deploy/dagster-instance-configuration) (via [`dagster.yaml`](/guides/deploy/dagster-yaml)). This allows the webserver and workers to communicate information about the run and events with each other. Refer to the [Dagster storage section of the Dagster instance documentation](/guides/deploy/dagster-instance-configuration#dagster-storage) for information on how to do this.
+
+:::note
+
+The same instance config must be present in the webserver's environment and in the workers' environments. Refer to the [Dagster instance](/guides/deploy/dagster-instance-configuration) documentation for more information.
+
+:::
+
+### Step 2: Configure a persistent I/O manager
+
+When using the Celery executor for job runs, you'll need to use storage that's accessible from all nodes where Celery workers are running. This is necessary because data is exchanged between ops that might be in different worker processes, possibly on different nodes. Common options for such accessible storage include an Amazon S3 or Google Cloud Storage (GCS) bucket or an NFS mount.
+
+To do this, include an appropriate I/O manager in the job's resource. For example, any of the following I/O managers would be suitable:
+
+-
+-
+-
+
+### Step 3: Supply executor and worker config
+
+If using custom config for your runs - such as using a different Celery broker URL or backend - you'll need to ensure that your workers start up with the config.
+
+To do this:
+
+1. Make sure the engine config is in a YAML file accessible to the workers
+2. Start the workers with the `-y` parameter as follows:
+
+ ```shell
+ dagster-celery worker start -y /path/to/celery_config.yaml
+ ```
+
+### Step 4: Ensure Dagster code is accessible
+
+Lastly, you'll need to make sure that the Dagster code you want the workers to execute is:
+
+1. Present in the workers' environment, and
+2. The code is in sync with the code present on the node running the webserver
+
+The easiest way to do this is typically to package the code into a Python module and to configure your project's [`workspace.yaml`](/guides/deploy/code-locations/workspace-yaml) to have the webserver load from that module.
+
+In Part 1, we accomplished this by starting the webserver with the `-f` parameter:
+
+```shell
+dagster dev -f celery_job.py
+```
+
+This told the webserver the file containing the job (`celery_job.py`) and to start the Celery worker from the same point in the file system, so the job was available in the same location.
+
+## Additional information
+
+### Using the dagster-celery CLI
+
+In the walkthrough, we started our workers using the `dagster-celery` CLI instead of invoking Celery directly. This CLI is intended as a convenient wrapper that shields you from the complexity of full Celery configuration. **Note**: It's still possible to directly start Celery workers - let us know if your use case requires this.
+
+For all of these commands, it's essential that your broker is running.
+
+```shell
+## Start new workers
+dagster-celery worker start
+
+## View running workers
+dagster-celery worker list
+
+## Terminate workers
+dagster-celery worker terminate
+```
+
+:::note
+
+If running Celery with custom config, include the config file path in these commands to ensure workers start with the correct config. Refer to [Step 3](#step-3-supply-executor-and-worker-config) of the walkthrough for more information.
+
+:::
+
+While `dagster-celery` is designed to make the full range of Celery configuration available on an as-needed basis, keep in mind that some combinations of config may not be compatible with each other. However, if you're may be comfortable tuning Celery, changing some of the settings may work better for your use case.
+
+### Monitoring and debugging
+
+There are several available tools for monitoring and debugging your queues and workers. First is the Dagster UI, which will display event logs and the `stdout`/`stderr` from runs. You can also view the logs generated by the broker and by the worker processes.
+
+To debug broker/queue level issues, use the monitoring tools provided by the broker you're running. RabbitMQ includes a [monitoring API](https://www.rabbitmq.com/monitoring.html) and has first class support for Prometheus and Grafana integration in production.
+
+To monitor celery workers and queues, you can use Celery's [Flower](https://flower.readthedocs.io/en/latest/) tool. This can be useful in understanding how workers interact with the queue.
+
+### Broker and backend
+
+`dagster-celery` has been tested using the RabbitMQ broker and default RPC backend.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/customizing-run-queue-priority.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/customizing-run-queue-priority.md
new file mode 100644
index 0000000000000..c3f6f50eef759
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/customizing-run-queue-priority.md
@@ -0,0 +1,102 @@
+---
+title: "Customizing run queue priority"
+sidebar_position: 400
+---
+
+When using a [run coordinator](run-coordinators), you can define custom prioritization rules for your Dagster instance.
+
+By the end of this guide, you’ll:
+
+- Understand how the run queue works
+- Learn how to define custom prioritization rules
+- Understand how prioritization rules and concurrency limits work together
+
+## Understanding the run queue
+
+The run queue is a sequence of Dagster runs waiting to be executed. Dagster pulls runs from the queue and calls `launch_run` on submitted runs. It operates as a first-in, first-out priority queue.
+
+For example, if three runs are submitted in the following order:
+
+1. Run `A`
+2. Run `B`
+3. Run `C`
+
+Then the runs will be launched in the same order: Run `A`, then `B`, then `C`. This will be true unless there are [tag concurrency limits](/guides/operate/managing-concurrency) or prioritization rules in place, which we’ll cover later in this guide.
+
+By default, all runs have a priority of `0`. Dagster launches runs with higher priority first. If multiple runs have the same priority, Dagster will launch the runs in the order they're submitted to the queue.
+
+Negative priorities are also allowed and can be useful for de-prioritizing sets of runs, such as backfills.
+
+## Defining queue prioritization rules
+
+Custom priority is specified using the `dagster/priority` tag, which can be set in code on definitions or in the launchpad of the Dagster UI.
+
+When defining a priority value, note that:
+
+- Values must be integers specified as a string. For example: `"1"`
+- Negative values are allowed. For example: `"-1"`
+
+
+
+
+**In Python**
+
+In this example, the priority is set to `-1` with a `dagster/priority` tag value of `"-1"`:
+
+{/* TODO convert to */}
+```python startafter=start_marker_priority endbefore=end_marker_priority file=/deploying/concurrency_limits/concurrency_limits.py
+@job(tags={"dagster/priority": "3"})
+def important_job(): ...
+
+
+@schedule(
+ cron_schedule="* * * * *",
+ job_name="important_job",
+ execution_timezone="US/Central",
+ tags={"dagster/priority": "-1"},
+)
+def less_important_schedule(_): ...
+```
+
+
+
+
+**In the Dagster UI**
+
+Using the launchpad in the Dagster UI, you can also override priority tag values. In this example, we clicked the **Edit tags** button to display the following modal:
+
+![Add tags to run modal in Dagster UI Launchpad](/images/guides/deploy/execution/dagster-priority-in-launchpad.png)
+
+
+
+
+**Understanding prioritization rules and concurrency limits**
+
+Unless tag concurrency limits and/or prioritization rules are in place, queued runs are executed in the order they’re submitted to the queue. However, a run blocked by tag concurrency limits won’t block runs submitted after it.
+
+Let’s walk through an example to demonstrate. In this example, three runs are submitted in the following order:
+
+1. Run `A`, tagged as `team: docs`
+2. Run `B`, tagged as `team: docs`
+3. Run `C`, which isn’t tagged
+
+Without configured limits, these runs will be launched in the order they were submitted, or run `A`, then `B`, then `C`.
+
+Before any more runs are launched, let’s add the following configuration to our instance’s settings:
+
+```yaml
+tag_concurrency_limits:
+ - key: "team"
+ limit: 1
+```
+
+Now, runs `A` and `B` can’t execute concurrently, while there isn’t a limit on run `C`. Assuming each run executes for a minimum of five minutes, the order in which the runs are launched will change.
+
+If the runs are submitted in the same order as before - that is, `A`, `B`, `C` - then the following will occur:
+
+- Run `A` launches
+- Run `B` B is skipped, as run `A` is in progress and concurrent runs are limited to `1` for the `team` tag
+- Run `C` launches
+- Run `B` launches after run `A` finishes
+
+To summarize, due to the concurrency limit, this configuration will change the run launching order to `A`, `C`, `B`.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/dagster-daemon.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/dagster-daemon.md
new file mode 100644
index 0000000000000..2b850a1937bb2
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/dagster-daemon.md
@@ -0,0 +1,68 @@
+---
+title: "Dagster daemon"
+sidebar_position: 100
+---
+
+Several Dagster features, like [schedules](/guides/automate/schedules/), [sensors](/guides/automate/sensors/), and [run queueing](/guides/deploy/execution/customizing-run-queue-priority), require a long-running `dagster-daemon` process to be included with your deployment.
+
+## Starting the daemon
+
+- [Running locally](#running-locally)
+- [Deploying the daemon](#deploying-the-daemon)
+
+### Running locally
+
+
+
+
+The easiest way to run the Dagster daemon locally is to run the `dagster dev` command:
+
+```shell
+dagster dev
+```
+
+This command launches both the Dagster webserver and the Dagster daemon, allowing you to start a full local deployment of Dagster from the command line. For more information about `dagster dev`, see "[Running Dagster locally](/guides/deploy/deployment-options/running-dagster-locally).
+
+
+
+
+To run the Dagster daemon by itself:
+
+```shell
+dagster-daemon run
+```
+
+This command takes the same arguments as `dagster dev` for specifying where to find your code.
+
+
+
+
+### Deploying the daemon
+
+For information on deploying the daemon to environments like Docker or Kubernetes, see the [deployment options documentation](/guides/deploy/deployment-options/).
+
+## Available daemons
+
+The `dagster-daemon` process reads from your [Dagster instance](/guides/deploy/dagster-instance-configuration) file to determine which daemons should be included. Each of the included daemons then runs on a regular interval in its own threads.
+
+The following daemons are currently available:
+
+| Name | Description | Enabled by |
+|----------------------------|---------------------|------------------|
+| Scheduler daemon | Creates runs from active schedules | Enabled / runs as long as the default isn't overriden as the scheduler on your instance. |
+| Run queue daemon | Launches queued runs, taking into account any limits and prioritization rules set on your instance | Setting the [run coordinator](run-coordinators) on your instance . |
+| Sensor daemon | Creates runs from active [sensors](/guides/automate/sensors/) that are turned on | Always enabled. |
+| Run monitoring daemon | Handles [run worker](/guides/deploy/oss-deployment-architecture#job-execution-flow) failures | Using the `run_monitoring` field in your instance. For more information, see "[Run monitoring](run-monitoring)".|
+
+If the daemon is configured to use a [workspace file](/guides/deploy/code-locations/workspace-yaml) to load [code location(s)](/guides/deploy/code-locations/), note that they will periodically reload the file. This means that the `dagster-daemon` process doesn't need to be restarted when workspace files are changed.
+
+## Checking daemon status in the Dagster UI
+
+To check the status of the `dagster-daemon` process in the UI:
+
+1. In the top navigation, click **Deployment**.
+2. Click the **Daemons** tab.
+
+This tab displays information about all the daemons currently configured on your instance.
+
+Each daemon periodically writes a heartbeat to your instance storage. If a daemon doesn't show a recent heartbeat, check the logs from your `dagster-daemon` process for errors.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/dask.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/dask.md
new file mode 100644
index 0000000000000..a344a223ef83e
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/dask.md
@@ -0,0 +1,133 @@
+---
+title: "Executing Dagster on Dask"
+sidebar_position: 800
+---
+
+The [dagster-dask](https://github.com/dagster-io/dagster/tree/master/python_modules/libraries/dagster-dask) module makes a **`dask_executor`** available, which can target either a local Dask cluster or a distributed cluster. Computation is distributed across the cluster at the execution step level -- that is, we use Dask to orchestrate execution of the steps in a job, not to parallelize computation within those steps.
+
+This executor takes the compiled execution plan, and converts each execution step into a [Dask Future](https://docs.dask.org/en/latest/futures.html) configured with the appropriate task dependencies to ensure tasks are properly sequenced. When the job is executed, these futures are generated and then awaited by the parent Dagster process.
+
+Data is passed between step executions via [IO Managers](/guides/build/io-managers/). As a consequence, a persistent shared storage (such as a network filesystem shared by all of the Dask nodes, S3, or GCS) must be used.
+
+Note that, when using this executor, the compute function of a single op is still executed in a single process on a single machine. If your goal is to distribute execution of workloads _within_ the logic of a single op, you may find that invoking Dask or PySpark directly from within the body of an op's compute function is a better fit than the engine layer covered in this documentation.
+
+## Requirements
+
+Install [dask.distributed](https://distributed.readthedocs.io/en/latest/install.html).
+
+## Local execution
+
+It is relatively straightforward to set up and run a Dagster job on local Dask. This can be useful for testing.
+
+First, run `pip install dagster-dask`.
+
+Then, create a job with the dask executor:
+
+{/* TODO convert to */}
+```python file=/deploying/dask_hello_world.py startafter=start_local_job_marker endbefore=end_local_job_marker
+from dagster_dask import dask_executor
+
+from dagster import job, op
+
+
+@op
+def hello_world():
+ return "Hello, World!"
+
+
+@job(executor_def=dask_executor)
+def local_dask_job():
+ hello_world()
+```
+
+Now you can run this job with a config block such as the following:
+
+{/* TODO convert to */}
+```python file=/deploying/dask_hello_world.yaml
+execution:
+ config:
+ cluster:
+ local:
+```
+
+Executing this job will spin up local Dask execution, run the job, and exit.
+
+## Distributed execution
+
+If you want to use a Dask cluster for distributed execution, you will first need to [set up a Dask cluster](https://distributed.readthedocs.io/en/latest/quickstart.html#setup-dask-distributed-the-hard-way). Note that the machine running the Dagster parent process must be able to connect to the host/port on which the Dask scheduler is running.
+
+You'll also need an IO manager that uses persistent shared storage, which should be attached to the job along with any resources on which it depends. Here, we use the :
+
+{/* TODO convert to */}
+```python file=/deploying/dask_hello_world_distributed.py startafter=start_distributed_job_marker endbefore=end_distributed_job_marker
+from dagster_aws.s3.io_manager import s3_pickle_io_manager
+from dagster_aws.s3.resources import s3_resource
+from dagster_dask import dask_executor
+
+from dagster import job, op
+
+
+@op
+def hello_world():
+ return "Hello, World!"
+
+
+@job(
+ executor_def=dask_executor,
+ resource_defs={"io_manager": s3_pickle_io_manager, "s3": s3_resource},
+)
+def distributed_dask_job():
+ hello_world()
+```
+
+For distributing task execution on a Dask cluster, you must provide a config block that includes the address/port of the Dask scheduler:
+
+{/* TODO convert to */}
+```python file=/deploying/dask_remote.yaml
+resources:
+ io_manager:
+ config:
+ s3_bucket: your_bucket_name
+execution:
+ config:
+ cluster:
+ existing:
+ address: "dask_scheduler.dns_name:8787"
+```
+
+Since Dask will invoke your job code on the cluster workers, you must ensure that the latest version of your Python code is available to all of the Dask workers. Ideally, you'll package this as a Python module, and target your `workspace.yaml` at this module.
+
+## Managing compute resources with Dask
+
+Dask has [basic support](https://distributed.dask.org/en/latest/resources.html) for compute resource management. In Dask you can specify that a particular worker node has, say, 3 GPUs, and then tasks which are specified with GPU requirements will be scheduled to respect that constraint on available resources.
+
+In Dask, you'd set this up by launching your workers with resource specifications:
+
+```shell
+dask-worker scheduler:8786 --resources "GPU=2"
+```
+
+and then when submitting tasks to the Dask cluster, specifying resource requirements in the Python API:
+
+```python
+client.submit(task, resources={'GPU': 1})
+```
+
+Dagster has simple support for Dask resource specification at the op level for ops that will be executed on Dask clusters. In your op definition, just add _tags_ as follows:
+
+```python
+@op(
+ ...
+ tags={'dagster-dask/resource_requirements': {"GPU": 1}},
+)
+def my_op(...):
+ pass
+```
+
+The dict passed to `dagster-dask/resource_requirements` will be passed through as the `resources` argument to the Dask client's **`~dask:distributed.Client.submit`** method for execution on a Dask cluster. Note that in non-Dask execution, this key will be ignored.
+
+## Caveats
+
+Dagster logs are not yet retrieved from Dask workers; this will be addressed in follow-up work.
+
+While this library is still nascent, we're working to improve it, and we are happy to accept contributions.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/index.md
new file mode 100644
index 0000000000000..c1bbbc2b8a5e0
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/index.md
@@ -0,0 +1,8 @@
+---
+title: "Execution"
+sidebar_position: 20
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-coordinators.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-coordinators.md
new file mode 100644
index 0000000000000..26060bb83fc85
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-coordinators.md
@@ -0,0 +1,23 @@
+---
+title: "Run coordinators"
+sidebar_position: 300
+---
+
+In production Dagster deployments, there are often many runs being launched at once. The _run coordinator_ lets you control the policy that Dagster uses to manage the set of runs in your deployment.
+
+When you submit a run from the Dagster UI or the Dagster command line, it’s first sent to the run coordinator, which applies any limits or prioritization policies before eventually sending it to the [run launcher](run-launchers) to be launched.
+
+## Run coordinator types
+
+The following run coordinators can be configured on your [Dagster instance](/guides/deploy/dagster-instance-configuration):
+
+| Term | Definition |
+|------|------------|
+| | The `DefaultRunCoordinator` calls launch_run on the instance’s run launcher immediately in the same process, without applying any limits or prioritization rules. When this coordinator is set, clicking **Launch Run** in the Dagster UI will immediately launch the run from the Dagster daemon process. Similarly, scheduled runs will immediately launch from the scheduler process. |
+| | The `QueuedRunCoordinator` sends runs to the Dagster daemon via a run queue. The daemon pulls runs from the queue and calls launch_run on submitted runs. Using this run coordinator enables instance-level limits on run concurrency, as well as custom run prioritization rules. |
+
+## Configuring run coordinators
+
+If you opt to use the `DefaultRunCoordinator`, no configuration is required on your part.
+
+However, if using the `QueuedRunCoordinator` or building a custom implementation, you can define [custom run prioritization rules](customizing-run-queue-priority) and [instance-level concurrency limits](/guides/operate/managing-concurrency).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-launchers.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-launchers.md
new file mode 100644
index 0000000000000..1498eb7320f86
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-launchers.md
@@ -0,0 +1,42 @@
+---
+title: "Run launchers"
+sidebar_position: 200
+---
+
+:::note
+
+This article applies to Dagster Open Source (OSS) deployments. For information on Dagster+, see the [Dagster+ documentation](/dagster-plus/).
+
+:::
+
+Runs initiated from the Dagster UI, the scheduler, or the `dagster job launch` CLI command are launched in Dagster. This is a distinct operation from executing a job using the `execute_job` Python API or the CLI `execute` command. A launch operation allocates computational resources (e.g. a process, a container, a Kubernetes pod, etc) to carry out a run execution and then instigates the execution.
+
+The core abstraction in the launch process is the _run launcher_, which is configured as part of the [Dagster instance](/guides/deploy/dagster-instance-configuration) The run launcher is the interface to the computational resources that will be used to actually execute Dagster runs. It receives the ID of a created run and a representation of the pipeline that is about to undergo execution.
+
+## Relevant APIs
+
+| Name | Description |
+| ----------------------------------------------------------------- | ----------------------------- |
+| | Base class for run launchers. |
+
+## Built-in run launchers
+
+The simplest run launcher is the built-in run launcher, . This run launcher spawns a new process per run on the same node as the job's code location.
+
+Other run launchers include:
+
+| Name | Description | Documentation |
+|------|-------------|---------------|
+| | A run launcher that allocates a Kubernetes job per run. | [Deploying Dagster to Kubernetes](/guides/deploy/deployment-options/kubernetes/deploying-to-kubernetes) |
+| | A run launcher that launches an Amazon ECS task per run. | [Deploying Dagster to Amazon Web Services](/guides/deploy/deployment-options/aws) |
+| | A run launcher that launches runs in a Docker container. | [Deploying Dagster using Docker Compose](/guides/deploy/deployment-options/) |
+| | A run launcher that launches runs as single Kubernetes jobs with extra configuration to support the `celery_k8s_job_executor`. | [Using Celery with Kubernetes](/guides/deploy/deployment-options/kubernetes/kubernetes-and-celery) |
+
+## Custom run launchers
+
+A few examples of when a custom run launcher is needed:
+
+- You have custom infrastructure or custom APIs for allocating nodes for execution.
+- You have custom logic for launching runs on different clusters, platforms, etc.
+
+We refer to the process or computational resource created by the run launcher as the [run worker](/guides/deploy/oss-deployment-architecture#job-execution-flow). The run launcher only determines the behavior of the run worker. Once execution starts within the run worker, it is the executor - an in-memory abstraction in the run worker process - that takes over management of computational resources.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-monitoring.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-monitoring.md
new file mode 100644
index 0000000000000..a43164a9c962f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-monitoring.md
@@ -0,0 +1,96 @@
+---
+title: "Detect and restart crashed workers with run monitoring"
+sidebar_position: 500
+---
+
+Dagster can detect hanging runs and restart crashed [run workers](/guides/deploy/oss-deployment-architecture#job-execution-flow). Using run monitoring requires:
+
+- Running the Dagster Daemon
+- Enabling run monitoring in the Dagster Instance:
+
+{/* TODO convert to */}
+```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_run_monitoring endbefore=end_run_monitoring
+# Opt in to run monitoring
+run_monitoring:
+ enabled: true
+ # values below are the defaults, and don't need to be specified except to override them
+ start_timeout_seconds: 180
+ cancel_timeout_seconds: 180
+ max_resume_run_attempts: 3 # experimental if above 0
+ poll_interval_seconds: 120
+```
+
+:::note
+
+In Dagster+, run monitoring is always enabled and can be configured in [deployment settings](/dagster-plus/deployment/management/deployments/deployment-settings-reference)
+
+:::
+
+## Run start timeouts
+
+When Dagster launches a run, the run stays in STARTING status until the run worker spins up and marks the run as STARTED. In the event that some failure causes the run worker to not spin up, the run might be stuck in STARTING status. The `start_timeout_seconds` offers a time limit for how long runs can hang in this state before being marked as failed.
+
+## Run cancelation timeouts
+
+When Dagster terminates a run, the run moves into CANCELING status and sends a termination signal to the run worker. When the run worker cleans up its resources, it moves into CANCELED status. In the event that some failure causes the run worker to not spin down cleanly, the run might be stuck in CANCELING status. The `cancel_timeout_seconds` offers a time limit for how long runs can hang in this state before being marked as canceled.
+
+## General run timeouts
+
+After a run is marked as STARTED, it may hang indefinitely for various reasons (user API errors, network issues, etc.). You can configure a maximum runtime for every run in a deployment by setting the `run_monitoring.max_runtime_seconds` field in your dagster.yaml or [Dagster+ deployment settings](/dagster-plus/deployment/management/deployments/deployment-settings-reference) to the maximum runtime in seconds. If a run exceeds this timeout and run monitoring is enabled, it will be marked as failed. The `dagster/max_runtime` tag can also be used to set a timeout in seconds on a per-run basis.
+
+For example, to configure a maximum of 2 hours for every run in your deployment:
+
+```yaml
+run_monitoring:
+ enabled: true
+ max_runtime_seconds: 7200
+```
+
+or in Dagster+, add the following to your [deployment settings](/dagster-plus/deployment/management/deployments/deployment-settings-reference):
+
+```yaml
+run_monitoring:
+ max_runtime_seconds: 7200
+```
+
+The below code example shows how to set a run timeout of 10 seconds on a per-job basis:
+
+```python file=/deploying/monitoring_daemon/run_timeouts.py startafter=start_timeout
+from dagster import define_asset_job, job
+
+
+@job(tags={"dagster/max_runtime": 10})
+def my_job(): ...
+
+
+asset_job = define_asset_job(
+ name="some_job", selection="*", tags={"dagster/max_runtime": 10}
+)
+# end_timeout
+```
+
+## Detecting run worker crashes
+
+:::note
+
+Detecting run worker crashes only works when using a run launcher other than the .
+
+:::
+
+It's possible for a run worker process to crash during a run. This can happen for a variety of reasons (the host it's running on could go down, it could run out of memory, etc.). Without the monitoring daemon, there are two possible outcomes, neither desirable:
+
+- If the run worker was able to catch the interrupt, it will mark the run as failed
+- If the run worker goes down without a grace period, the run could be left hanging in STARTED status
+
+If a run worker crashes, the run it's managing can hang. The monitoring daemon can run health checks on run workers for all active runs to detect this. If a failed run worker is detected (e.g. by the K8s Job having a non-zero exit code), the run is either marked as failed or resumed (see below).
+
+## Resuming runs after run worker crashes (Experimental)
+
+This feature is experimental and currently only supported when using:
+
+- [`K8sRunLauncher`](/api/python-api/libraries/dagster-k8s#dagster_k8s.K8sRunLauncher) with the [`k8s_job_executor`](/api/python-api/libraries/dagster-k8s#dagster_k8s.k8s_job_executor)
+- [`DockerRunLauncher`](/api/python-api/libraries/dagster-docker#dagster_docker.DockerRunLauncher) with the [`docker_executor`](/api/python-api/libraries/dagster-docker#dagster_docker.docker_executor)
+
+The monitoring daemon handles these by performing health checks on the run workers. If a failure is detected, the daemon can launch a new run worker which resumes execution of the existing run. The run worker crash will be show in the event log, and the run will continue to completion. If the run worker continues to crash, the daemon will mark the run as failed after the configured number of attempts.
+
+To enable, set `max_resume_run_attempts` to a value greater than 0.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-retries.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-retries.md
new file mode 100644
index 0000000000000..05a9c7ea795a2
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/execution/run-retries.md
@@ -0,0 +1,82 @@
+---
+title: "Configuring run retries"
+sidebar_position: 600
+---
+
+If you configure run retries, a new run will be kicked off whenever a run fails for any reason. Compared to [op](/guides/build/ops) retries, the maximum retry limit for run retries applies to the whole run instead of each individual op. Run retries also handle the case where the run process crashes or is unexpectedly terminated.
+
+## Configuration
+
+How to configure run retries depends on whether you're using Dagster+ or Dagster Open Source:
+
+- **Dagster+**: Use the [Dagster+ UI or the dagster-cloud CLI](/dagster-plus/deployment/management/deployments/deployment-settings-reference) to set a default maximum number of retries. Run retries do not need to be explicitly enabled.
+- **Dagster Open Source**: Use your instance's `dagster.yaml` to enable run retries.
+
+For example, the following will set a default maximum number of retries of `3` for all runs:
+
+```yaml
+run_retries:
+ enabled: true # Omit this key if using Dagster+, since run retries are enabled by default
+ max_retries: 3
+```
+
+In both Dagster+ and Dagster Open Source, you can also configure retries using tags either on Job definitions or in the Dagster UI [Launchpad](/guides/operate/webserver).
+
+{/* TODO convert to */}
+```python file=/deploying/job_retries.py
+from dagster import job
+
+
+@job(tags={"dagster/max_retries": 3})
+def sample_job():
+ pass
+
+
+@job(tags={"dagster/max_retries": 3, "dagster/retry_strategy": "ALL_STEPS"})
+def other_sample_sample_job():
+ pass
+```
+
+### Retry Strategy
+
+The `dagster/retry_strategy` tag controls which ops the retry will run.
+
+By default, retries will re-execute from failure (tag value `FROM_FAILURE`). This means that any successful ops will be skipped, but their output will be used for downstream ops. If the `dagster/retry_strategy` tag is set to `ALL_STEPS`, all the ops will run again.
+
+:::note
+
+`FROM_FAILURE` requires an I/O manager that can access outputs from other runs. For example, on Kubernetes the would work but the would not, since the new run is in a new Kubernetes job with a separate filesystem.
+
+:::
+
+### Combining op and run retries
+
+By default, if a run fails due to an op failure and both op and run retries are enabled, the overlapping retries might cause the op to be retried more times than desired. This is because the op retry count will reset for each retried run.
+
+To prevent this, you can configure run retries to only retry when the failure is for a reason other than an op failure, like a crash or an unexpected termination of the run worker. This behavior is controlled by the `run_retries.retry_on_asset_or_op_failure` setting, which defaults to `true` but can be overridden to `false`.
+
+For example, the following configures run retries so that they ignore runs that failed due to a step failure:
+
+```yaml
+run_retries:
+ enabled: true # Omit this key if using Dagster+, since run retries are enabled by default
+ max_retries: 3
+ retry_on_asset_or_op_failure: false
+```
+
+You can also apply the `dagster/retry_on_asset_or_op_failure` tag on specific jobs using tags to override the default value for runs of that job:
+
+```python
+from dagster import job
+
+
+@job(tags={"dagster/max_retries": 3, "dagster/retry_on_asset_or_op_failure": False})
+def sample_job():
+ pass
+```
+
+:::note
+
+Setting `retry_on_asset_or_op_failure` to `false` will only change retry behavior for runs on Dagster version 1.6.7 or greater.
+
+:::
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/index.md
new file mode 100644
index 0000000000000..6ac717748d2f6
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/index.md
@@ -0,0 +1,12 @@
+---
+title: "Deploying Dagster"
+sidebar_class_name: hidden
+---
+
+This section is about self-hosting Dagster.
+
+:::info
+To deploy to Dagster+, see the [Dagster+ getting started guide](/dagster-plus/getting-started).
+
+To migrate from self-hosted to Dagster+, see [Migrate from self-hosted to Dagster+](/dagster-plus/deployment/migration/self-hosted-to-dagster-plus).
+:::
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/oss-deployment-architecture.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/oss-deployment-architecture.md
new file mode 100644
index 0000000000000..29e43ae5b45e3
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/oss-deployment-architecture.md
@@ -0,0 +1,65 @@
+---
+title: "Open Source deployment architecture overview"
+sidebar_position: 100
+---
+
+
+:::note
+
+This guide is applicable to Dagster Open Source (OSS) deployments. For
+Dagster+ deployments, refer to the [Dagster+](/dagster-plus/) documentation.
+
+:::
+
+This page covers general information about deploying Dagster on your own infrastructure. For guides on specific platforms, see the [deployment options documentation](/guides/deploy/deployment-options/).
+
+Let's take a look at a generic Dagster deployment, after which we'll walk through each of its components:
+
+{/* https://excalidraw.com/#json=4771198026121216,nmLfShZ_frrUPJTUxaeI5g */}
+
+![Dagster deployment](/images/guides/deploy/dagster-deployment.png)
+
+## Long-running services
+
+Dagster requires three long-running services, which are outlined in the table below:
+
+| Service | Description | Replicas |
+|---------|-------------|----------|
+| [Dagster webserver](/guides/operate/webserver) | `dagster-webserver` serves the user interface and responds to GraphQL queries. | The Dagster webserver can have one or more replicas. |
+| [Dagster daemon](/guides/deploy/execution/dagster-daemon) | The Dagster daemon operates schedules, sensors, and run queuing. | Not supported. |
+| [Code location](/guides/deploy/code-locations/) server | Code location servers serve metadata about the collection of its Dagster definitions. | You can have many code location servers, but each code location can only have one replica for its server. |
+
+## Deployment configuration
+
+Dagster OSS deployments are composed of multiple components, such as storages, executors, and run launchers. One of the core features of Dagster is that each of these components is swappable and configurable. If custom configuration isn't provided, Dagster will automatically use a default implementation of each component. For example, by default Dagster uses to store information about pipeline runs. This can be swapped with the Dagster-provided instead or or a custom storage class.
+
+Based on the component's scope, configuration occurs at either the **Dagster instance** or **Job run** level. Access to user code is configured at the **Workspace** level. Refer to the following table for info on how components are configured at each of these levels:
+
+| Level | Configuration | Description |
+|-------|---------------|-------------|
+| [Dagster instance](/guides/deploy/dagster-instance-configuration) | `dagster.yaml` | The Dagster instance is responsible for managing all deployment-wide components, such as the database. You can specify the configuration for instance-level components in `dagster.yaml`. |
+| [Workspace](/guides/deploy/code-locations/workspace-yaml) | `workspace.yaml` | Workspace files define how to access and load your code. You can define workspace configuration using `workspace.yaml`. |
+| Job run | Run config | A job run is responsible for managing all job-scoped components, such as the executor, ops, and resources. These components dictate job behavior, such as how to execute ops or where to store outputs. Configuration for run-level components is specified using the job run's run config, and defined in either Python code or in the UI launchpad. |
+
+:::note
+
+Dagster provides a few vertically-integrated deployment options that abstract
+away some of the configuration options described above. For example, with
+Dagster's provided [Kubernetes Helm chart deployment](/guides/deploy/deployment-options/kubernetes/deploying-to-kubernetes), configuration is defined through Helm values, and the Kubernetes deployment automatically generates Dagster Instance and Workspace configuration.
+
+:::
+
+## Job execution flow
+
+Job execution flows through several parts of the Dagster system. The following table describes runs launched by the UI, specifically the components that handle execution and the order in which they are executed.
+
+| Order | Component | Description | Configured by |
+|-------|-----------|-------------|---------------|
+| [Run coordinator](/guides/deploy/execution/run-coordinators) | The run coordinator is a class invoked by the webserver process when runs are launched from the Dagster UI. This class can be configured to pass runs to the daemon via a queue. | Instance |
+| [Run launcher](/guides/deploy/execution/run-launchers) | The run launcher is a class invoked by the daemon when it receives a run from the queue. This class initializes a new run worker to handle execution. Depending on the launcher, this could mean spinning up a new process, container, Kubernetes pod, etc. | Instance |
+| Run worker | The run worker is a process which traverses a graph and uses the executor to execute each op. | n/a |
+| [Executor](/guides/operate/run-executors) | The executor is a class invoked by the run worker for running user ops. Depending on the executor, ops run in local processes, new containers, Kubernetes pods, etc. | Run config |
+
+Additionally, note that runs launched by schedules and sensors go through the same flow, but the first step is called by the [Dagster daemon](/guides/deploy/execution/dagster-daemon) instead of the webserver.
+
+In a deployment without the [Dagster daemon](/guides/deploy/execution/dagster-daemon), the webserver directly calls the **run launcher** and skips the **run coordinator**.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/using-environment-variables-and-secrets.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/using-environment-variables-and-secrets.md
new file mode 100644
index 0000000000000..12321c96aeba1
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/deploy/using-environment-variables-and-secrets.md
@@ -0,0 +1,298 @@
+---
+title: "Using environment variables and secrets in Dagster code"
+sidebar_position: 400
+---
+
+Environment variables, which are key-value pairs configured outside your source code, allow you to dynamically modify application behavior depending on environment.
+
+Using environment variables, you can define various configuration options for your Dagster application and securely set up secrets. For example, instead of hard-coding database credentials - which is bad practice and cumbersome for development - you can use environment variables to supply user details. This allows you to parameterize your pipeline without modifying code or insecurely storing sensitive data.
+
+## Declaring environment variables
+
+How environment variables are declared depends on whether you're developing locally or have already deployed your Dagster project.
+
+
+
+
+**Local development**
+
+As of Dagster 1.1.0, using `.env` files is supported for loading environment variables into local environments. A `.env` file is a text file containing key-value pairs that is used locally, but not checked into source control. Using a `.env` file allows you to develop and test locally without putting sensitive info at risk. For example:
+
+```shell
+# .env
+
+DATABASE_NAME=staging
+DATABASE_SCHEMA=sales
+DATABASE_USERNAME=salesteam
+DATABASE_PASSWORD=supersecretstagingpassword
+```
+
+If Dagster detects a `.env` file in the same folder where `dagster-webserver` or `dagster-daemon` is launched, it will automatically load the environment variables in the file. This also applies to variables [exported from Dagster+](/dagster-plus/deployment/management/environment-variables/dagster-ui#export)
+
+
+When using a `.env` file, keep the following in mind:
+
+- The `.env` file must be in the same folder where `dagster-webserver` or `dagster-daemon` is launched
+- Any time the `.env` file is modified, the workspace must be re-loaded to make the Dagster webserver/UI aware of the changes
+
+
+
+**Dagster+**
+
+Environment variables can be set a variety of ways in Dagster+:
+
+- Directly in the UI
+- Via agent configuration (Hybrid deployments only)
+
+If using the UI, you can also [export locally-scoped variables to a `.env` file](/dagster-plus/deployment/management/environment-variables/dagster-ui#export), which you can then use to develop locally.
+
+Refer to the [Dagster+ environment variables guide](/dagster-plus/deployment/management/environment-variables/) for more info.
+
+
+
+
+
+
+**Dagster open source**
+
+How environment variables are set for Dagster projects deployed on your infrastructure depends on **where** Dagster is deployed. Refer to the deployment guide for your platform for more info:
+
+* [Amazon Web Services EC2 / ECS](/guides/deploy/deployment-options/aws)
+* [GCP](/guides/deploy/deployment-options/gcp)
+* [Docker](/guides/deploy/deployment-options/docker)
+* [Kubernetes](/guides/deploy/deployment-options/kubernetes/deploying-to-kubernetes)
+
+
+
+
+## Accessing environment variables
+
+In this section, we'll demonstrate how to access environment variables once they've been declared. There are two ways to do this:
+
+- [In Python code](#in-python-code), which isn't specific to Dagster
+- [From Dagster configuration](#from-dagster-configuration), which incorporates environment variables into the Dagster config system
+
+### In Python code
+
+To access environment variables in your Dagster code, you can use [`os.getenv`](https://docs.python.org/3/library/os.html#os.getenv):
+
+```python
+import os
+
+database_name = os.getenv("DATABASE_NAME")
+```
+
+This approach also works for accessing [built-in environment variables for Dagster+](/dagster-plus/deployment/management/environment-variables/built-in):
+
+```python
+import os
+
+deployment_name = os.getenv("DAGSTER_CLOUD_DEPLOYMENT_NAME")
+```
+
+For a real-world example, see the [Dagster+ branch deployments example](#dagster-branch-deployments).
+
+You can also call the `get_value()` method on the `EnvVar`:
+
+```python
+from dagster import EnvVar
+
+database_name = EnvVar('DATABASE_NAME').get_value()
+```
+
+### From Dagster configuration
+
+[Configurable Dagster objects](/guides/operate/configuration/run-configuration) - such as ops, assets, resources, I/O managers, and so on - can accept configuration from environment variables. Dagster provides a native way to specify environment variables in your configuration. These environment variables are retrieved at launch time, rather than on initialization as with `os.getenv`. Refer to the [next section](#using-envvar-vs-osgetenv) for more info.
+
+
+
+
+**In Python code**
+
+To access an environment variable as part of a Dagster configuration in Python code, you may use the following special syntax:
+
+```python
+"PARAMETER_NAME": EnvVar("ENVIRONMENT_VARIABLE_NAME")
+```
+
+For example:
+
+```python
+"access_token": EnvVar("GITHUB_ACCESS_TOKEN")
+```
+
+And when specifying an integer number:
+
+```python
+"database_port": EnvVar.int("DATABASE_PORT")
+```
+
+
+
+
+**In YAML or config dictionaries**
+
+To access an environment variable as part of a Dagster configuration in YAML or in a config dictionary, use the following syntax:
+
+```python
+"PARAMETER_NAME": {"env": "ENVIRONMENT_VARIABLE_NAME"}
+```
+
+For example:
+
+```python
+"access_token": {"env": "GITHUB_ACCESS_TOKEN"}
+```
+
+Refer to the [Handling secrets section](#handling-secrets) and [Per-environment configuration example](#per-environment-configuration-example) for examples.
+
+
+
+
+### Using EnvVar vs os.getenv
+
+We just covered two different ways to access environment variables in Dagster. So, which one should you use? When choosing an approach, keep the following in mind:
+
+- **When `os.getenv` is used**, the variable's value is retrieved when Dagster loads the [code location](/guides/deploy/code-locations/) and **will** be visible in the UI.
+- **When `EnvVar` is used**, the variable's value is retrieved at runtime and **won't** be visible in the UI.
+
+Using the `EnvVar` approach has a few unique benefits:
+
+- **Improved observability.** The UI will display information about configuration values sourced from environment variables.
+- **Secret values are hidden in the UI.** Secret values are hidden in the Launchpad, Resources page, and other places where configuration is displayed.
+- **Simplified testing.** Because you can provide string values directly to configuration rather than environment variables, testing may be easier.
+
+## Handling secrets
+
+Using environment variables to provide secrets ensures sensitive info won't be visible in your code or the launchpad in the UI. In Dagster, best practice for handling secrets uses [configuration](/guides/operate/configuration/run-configuration) and [resources](/guides/build/external-resources/).
+
+A resource is typically used to connect to an external service or system, such as a database. Resources can be configured separately from the rest of your app, allowing you to define it once and reuse it as needed.
+
+Let's take a look at an example from the [Dagster Crash Course](https://dagster.io/blog/dagster-crash-course-oct-2022), which creates a GitHub resource and supplies it to assets. Let's start by looking at the resource:
+
+```python
+## resources.py
+
+from dagster import StringSource, resource
+from github import Github
+
+class GithubClientResource(ConfigurableResource):
+ access_token: str
+
+ def get_client(self) -> Github:
+ return Github(self.access_token)
+```
+
+Let's review what's happening here:
+
+- This code creates a GitHub resource named `GithubClientResource`
+- By subclassing and specifying the `access_token` field, we're telling Dagster that we want to be able to configure the resource with an `access_token` parameter
+- Since `access_token` is a string value, this config parameter can either be:
+ - An environment variable, or
+ - Provided directly in the configuration
+
+As storing secrets in configuration is bad practice, we'll opt for using an environment variable. In this code, we're configuring the resource supplying it to our assets:
+
+{/* TODO convert to */}
+```python file=/guides/dagster/using_environment_variables_and_secrets/repository.py startafter=start endbefore=end
+# definitions.py
+
+from my_dagster_project import assets
+from my_dagster_project.resources import GithubClientResource
+
+from dagster import Definitions, EnvVar, load_assets_from_package_module
+
+defs = Definitions(
+ assets=load_assets_from_package_module(assets),
+ resources={
+ "github_api": GithubClientResource(access_token=EnvVar("GITHUB_ACCESS_TOKEN"))
+ },
+)
+```
+
+Let's review what's happening here:
+
+- We pass configuration info to the resource when we construct it. In this example, we're telling Dagster to load the `access_token` from the `GITHUB_ACCESS_TOKEN` environment variable by wrapping it in `EnvVar`.
+- We're adding that resource to our object so it's available for our assets.
+
+## Parameterizing pipeline behavior
+
+Using environment variables, you define how your code should execute at runtime.
+
+- [Per-environment configuration example](#per-environment-configuration-example)
+
+### Per-environment configuration example
+
+In this example, we'll demonstrate how to use different I/O manager configurations for `local` and `production` environments using [configuration](/guides/operate/configuration/run-configuration) (specifically the configured API) and [resources](/guides/build/external-resources/).
+
+This example is adapted from the [Transitioning data pipelines from development to production guide](/guides/deploy/dev-to-prod):
+
+{/* TODO convert to */}
+```python file=/guides/dagster/using_environment_variables_and_secrets/repository_v2.py startafter=start_new endbefore=end_new
+# definitions.py
+
+resources = {
+ "local": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ account="abc1234.us-east-1",
+ user=EnvVar("DEV_SNOWFLAKE_USER"),
+ password=EnvVar("DEV_SNOWFLAKE_PASSWORD"),
+ database="LOCAL",
+ schema=EnvVar("DEV_SNOWFLAKE_SCHEMA"),
+ ),
+ },
+ "production": {
+ "snowflake_io_manager": SnowflakePandasIOManager(
+ account="abc1234.us-east-1",
+ user="system@company.com",
+ password=EnvVar("SYSTEM_SNOWFLAKE_PASSWORD"),
+ database="PRODUCTION",
+ schema="HACKER_NEWS",
+ ),
+ },
+}
+
+deployment_name = os.getenv("DAGSTER_DEPLOYMENT", "local")
+
+defs = Definitions(
+ assets=[items, comments, stories], resources=resources[deployment_name]
+)
+```
+
+Let's review what's happening here:
+
+- We've created a dictionary of resource definitions, `resources`, named after our `local` and `production` environments. In this example, we're using a [Pandas Snowflake I/O manager](/api/python-api/libraries/dagster-snowflake-pandas).
+- For both `local` and `production`, we constructed the I/O manager using environment-specific run configuration. Note the differences in configuration between `local` and `production`, specifically where environment variables were used.
+- Following the `resources` dictionary, we define the `deployment_name` variable, which determines the current executing environment. This variable defaults to `local`, ensuring that `DAGSTER_DEPLOYMENT=PRODUCTION` must be set to use the `production` configuration.
+
+### Dagster+ branch deployments
+
+:::note
+
+This section is only applicable to Dagster+.
+
+:::
+
+This example demonstrates how to determine the current deployment type at runtime - [branch deployment](/dagster-plus/features/ci-cd/branch-deployments/) or [full deployment](/dagster-plus/deployment/management/deployments/) - without using resources or configuration.
+
+Let's look at a function that determines the current deployment using the `DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT` environment variable:
+
+```python
+
+def get_current_env():
+ is_branch_depl = os.getenv("DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT") == "1"
+ assert is_branch_depl != None # env var must be set
+ return "branch" if is_branch_depl else "prod"
+
+```
+
+This function checks the value of `DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT` and, if equal to `1`, returns a variable with the value of `branch`. This indicates that the current deployment is a branch deployment. Otherwise, the deployment is a full deployment and is_branch_depl will be returned with a value of prod.
+
+Using this info, we can write code that executes differently when in a branch deployment or a full deployment.
+
+## Troubleshooting
+
+| Error | Description | Solution |
+|-------|-------------|--------------|
+| **You have attempted to fetch the environment variable "[variable]" which is not set. In order for this execution to succeed it must be set in this environment.** | Surfacing when a run is launched in the UI, this error means that an environment variable set using could not be found in the executing environment. | Verify that the environment variable is named correctly and accessible in the environment.
**If developing locally and using a `.env` file**, try reloading the workspace in the UI. The workspace must be reloaded any time this file is modified for the UI to be aware of the changes.
**If using Dagster+**:
Verify that the environment variable is [scoped to the environment and code location](/dagster-plus/deployment/management/environment-variables/dagster-ui#scope) if using the built-in secrets manager
Verify that the environment variable was correctly configured and added to your [agent's configuration](/dagster-plus/deployment/management/environment-variables/agent-config)
|
+| **No environment variables in `.env` file.** | Dagster located and attempted to load a local `.env` file while launching `dagster-webserver`, but couldn't find any environment variables in the file. | If this is unexpected, verify that your `.env` is correctly formatted and located in the same folder where you're running `dagster-webserver`. |
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/migrate/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/migrate/index.md
new file mode 100644
index 0000000000000..c10a38f869b4b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/migrate/index.md
@@ -0,0 +1,9 @@
+---
+title: "Migrate"
+sidebar_class_name: hidden
+---
+
+You can easily migrate Dagster versions, or migrate to Dagster from other platforms, such as Airflow:
+
+* [Migrating Dagster versions](version-migration)
+* [Migrating from Airflow to Dagster](/integrations/libraries/airlift/airflow-to-dagster/)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/migrate/version-migration.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/migrate/version-migration.md
new file mode 100644
index 0000000000000..384c4e0bd134d
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/migrate/version-migration.md
@@ -0,0 +1,10 @@
+---
+title: 'Version migration'
+hide_title: true
+---
+
+import VersionMigration, {toc as VersionMigrationTOC} from "@site/../../MIGRATION.md"
+
+
+
+export const toc = VersionMigrationTOC;
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/alerting.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/alerting.md
new file mode 100644
index 0000000000000..a5abd2e7ea3b4
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/alerting.md
@@ -0,0 +1,13 @@
+---
+title: Alerts
+sidebar_position: 20
+unlisted: true
+---
+
+(This will likely be a category)
+
+Alerting if my pipeline didn't execute
+Tracking when a run or sensor fails
+Knowing when a pipeline never ran
+Knowing if a pipeline is running slow, or an asset is late
+Knowing if my Dagster instance is having issues
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/index.md
new file mode 100644
index 0000000000000..7188ca3ff0365
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/index.md
@@ -0,0 +1,8 @@
+---
+title: "Monitoring pipelines"
+sidebar_class_name: hidden
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/logging/custom-logging.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/logging/custom-logging.md
new file mode 100644
index 0000000000000..742fbd5afc296
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/logging/custom-logging.md
@@ -0,0 +1,129 @@
+---
+title: "Customizing Dagster's built-in loggers"
+sidebar_position: 100
+---
+
+Custom loggers are used to alter the structure of the logs being produced by your Dagster pipelines. For example, JSON logs can be produced to more easily be processed by log management systems. For a list of all built-in loggers, see the [API documentation](/api/python-api/loggers#built-in-loggers).
+
+:::note Limitations
+
+It's not currently possible to globally configure the logger for all jobs in a repository.
+
+:::
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- A basic understanding of Dagster concepts such as assets, jobs and definitions
+- A working knowledge of the Python logging module
+
+
+
+## Step 1: Add a prebuilt custom logger to your jobs
+
+This step shows how to add an existing custom logger, the , to your jobs. This will override the default and produce logs in JSON format.
+
+### Add the custom logger to your asset jobs
+
+The following example shows how to add the custom logger to your code location definitions and configure an asset job to use it.
+
+
+
+
+### Add the custom logger to your ops-based jobs
+
+Configuring a ops job to use the custom logger slightly differs from the asset job example. The following example shows how:
+
+
+
+
+### Expected `json_console_logger` output
+
+The `json_console_logger` will emit an exhaustive single line JSON document containing the full log record, including the Dagster metadata fields.
+
+Here's an example of the output for reference, formatted for readability:
+
+```json
+{
+ "args": [],
+ "created": 1725455358.2311811,
+ "dagster_meta": {
+ "dagster_event": null,
+ "dagster_event_batch_metadata": null,
+ "job_name": "hackernews_topstory_ids_job",
+ "job_tags": {
+ ".dagster/grpc_info": "{\"host\": \"localhost\", \"socket\": \"/var/folders/5b/t062dlpj3j716l4w1d3yq6vm0000gn/T/tmpds_hvzm9\"}",
+ "dagster/preset_name": "default",
+ "dagster/solid_selection": "*"
+ },
+ "log_message_id": "3850cfb8-f9fb-458a-a986-3efd26e4b859",
+ "log_timestamp": "2024-09-04T13:09:18.225289",
+ "op_name": "get_hackernews_topstory_ids",
+ "orig_message": "Compute Logger - Got 500 top stories.",
+ "resource_fn_name": null,
+ "resource_name": null,
+ "run_id": "11528a21-38d5-43e7-8b13-993e47ce7f7d",
+ "step_key": "get_hackernews_topstory_ids"
+ },
+ "exc_info": null,
+ "exc_text": null,
+ "filename": "log_manager.py",
+ "funcName": "emit",
+ "levelname": "INFO",
+ "levelno": 20,
+ "lineno": 272,
+ "module": "log_manager",
+ "msecs": 231.0,
+ "msg": "hackernews_topstory_ids_job - 11528a21-38d5-43e7-8b13-993e47ce7f7d - get_hackernews_topstory_ids - Compute Logger - Got 500 top stories.",
+ "name": "dagster",
+ "pathname": "/home/dagster/workspace/quickstart-etl/.venv/lib/python3.11/site-packages/dagster/_core/log_manager.py",
+ "process": 35373,
+ "processName": "SpawnProcess-2:1",
+ "relativeCreated": 813.4410381317139,
+ "stack_info": null,
+ "thread": 8584465408,
+ "threadName": "MainThread"
+}
+```
+
+### Changing the logger configuration in the Dagster UI
+
+You can also change the logger configuration in the Dagster UI. This is useful if you want to change the logger configuration without changing the code, to use the custom logger on a manual asset materialization launch, or change the verbosity of the logs. Add the following lines to your `config.yaml`:
+
+```yaml
+loggers:
+ console:
+ config:
+ log_level: DEBUG
+```
+
+## Step 2: Write your custom logger
+
+In this example, we'll create a logger implementation that produces comma separated values from selected fields in the
+log record. Other examples can be found in the codebase, in the built-in loggers such as `json_console_logger`.
+
+
+
+Sample output:
+
+```csv
+2024-09-04T09:29:33.643818,dagster,INFO,cc76a116-4c8f-400f-9c4d-c42b66cdee3a,topstory_ids_job,hackernews_topstory_ids,Compute Logger - Got 500 top stories.
+```
+
+The available fields emitted by the logger are defined in the [`LogRecord`](https://docs.python.org/3/library/logging.html#logrecord-objects) object.
+In addition, Dagster specific information can be found in the `dagster_meta` attribute of the log record. The previous
+example already some of these attributes.
+
+It contains the following fields:
+
+- `dagster_event`: string
+- `dagster_event_batch_metadata`: string
+- `job_name`: string
+- `job_tags`: a dictionary of strings
+- `log_message_id`: string
+- `log_timestamp`: string
+- `op_name`: string
+- `run_id`: string
+- `step_key`: string
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/logging/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/logging/index.md
new file mode 100644
index 0000000000000..06b18c8ee1dac
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/logging/index.md
@@ -0,0 +1,92 @@
+---
+title: "Logging"
+sidebar_position: 10
+---
+
+Dagster supports a variety of [built-in logging options](/api/python-api/loggers#built-in-loggers), as well as the ability to extend and customize them. Logs can be produced by runs, sensor and schedule evaluations, and processes like the [Dagster webserver](/guides/operate/webserver) and daemon.
+
+By default, Dagster automatically tracks and captures all execution events, which includes:
+
+- **Information about what Dagster is doing**, such as events that occur while Dagster executes a run to materialize an asset.
+- **Events produced by user code**, such as a custom message showing how many records were processed to create an asset.
+
+During these events, logs like the following are generated:
+
+```bash
+12:40:05 - DEBUG - RUN_START - Started execution of run for "__ASSET_JOB_0".
+12:40:05 - DEBUG - ENGINE_EVENT - Executing steps using multiprocess executor: parent process (pid: 86387)
+12:40:05 - DEBUG - taxi_zones_file - STEP_WORKER_STARTING - Launching subprocess for "taxi_zones_file".
+12:40:07 - DEBUG - STEP_WORKER_STARTED - Executing step "taxi_zones_file" in subprocess.
+12:40:07 - DEBUG - taxi_zones_file - RESOURCE_INIT_STARTED - Starting initialization of resources [io_manager].
+12:40:07 - DEBUG - taxi_zones_file - RESOURCE_INIT_SUCCESS - Finished initialization of resources [io_manager].
+12:40:07 - DEBUG - LOGS_CAPTURED - Started capturing logs in process (pid: 86390).
+12:40:07 - DEBUG - taxi_zones_file - STEP_START - Started execution of step "taxi_zones_file".
+12:40:09 - DEBUG - taxi_zones_file - STEP_OUTPUT - Yielded output "result" of type "Any". (Type check passed).
+12:40:09 - DEBUG - __ASSET_JOB_0 - taxi_zones_file - Writing file at: /Users/erincochran/Desktop/dagster-examples/project-dagster-university/tmpfxsoltsc/storage/taxi_zones_file using PickledObjectFilesystemIOManager...
+12:40:09 - DEBUG - taxi_zones_file - ASSET_MATERIALIZATION - Materialized value taxi_zones_file.
+12:40:09 - DEBUG - taxi_zones_file - HANDLED_OUTPUT - Handled output "result" using IO manager "io_manager"
+12:40:09 - DEBUG - taxi_zones_file - STEP_SUCCESS - Finished execution of step "taxi_zones_file" in 1.17s.
+12:40:09 - DEBUG - ENGINE_EVENT - Multiprocess executor: parent process exiting after 4.38s (pid: 86387)
+12:40:09 - DEBUG - RUN_SUCCESS - Finished execution of run for "__ASSET_JOB_0".
+```
+
+These logs can be exported to a local file or viewed in the Dagster UI.
+
+## Log types
+
+When jobs are run, the logs stream back to the UI's **Run details** page in real time. The UI contains two types of logs: [structured event logs](#structured-event-logs) and [raw compute logs](#raw-compute-logs).
+
+### Structured event logs
+
+Structured logs are enriched and categorized with metadata. For example, a label of which asset a log is about, links to an asset’s metadata, and what type of event it is available. This structuring also enables easier filtering and searching in the logs.
+
+#### Logs streaming back to the UI in real time
+
+![Real time logs in the Dagster UI](/images/guides/monitor/logging/job-log-ui.png)
+
+#### Log messages filtered based on execution steps and log levels
+
+![Log filtering in the Dagster UI](/images/guides/monitor/logging/job-ui-filter.png)
+
+### Raw compute logs
+
+The raw compute logs contain logs for both [`stdout` and `stderr`](https://stackoverflow.com/questions/3385201/confused-about-stdin-stdout-and-stderr), which you can toggle between. To download the logs, click the **arrow icon** near the top right corner of the logs.
+
+Custom log messages are also included in these logs. Notice in the following image that the `Hello world!` message is included on line three:
+
+![Raw compute logs in the Run details page](/images/guides/monitor/logging/loggers-compute-logs.png)
+
+:::note
+
+Windows / Azure users may need to enable the environment variable `PYTHONLEGACYWINDOWSSTDIO` in order for compute logs to be displayed in the Dagster UI. To do that in PowerShell, run `$Env:PYTHONLEGACYWINDOWSSTDIO = 1` and then restart the Dagster instance.
+
+:::
+
+## Configuring loggers
+
+Loggers can be configured when you run a job. For example, to filter all messages below `ERROR` out of the colored console logger, add the following lines to your `config.yaml`:
+
+```yaml file=/concepts/logging/config.yaml
+loggers:
+ console:
+ config:
+ log_level: ERROR
+```
+
+When a job with the above configuration is executed, you'll only see the `ERROR` level logs.
+
+## Customizing Dagster's built-in loggers
+
+Dagster's [built-in loggers](/api/python-api/loggers#built-in-loggers):
+
+- Support all levels of Python logs, such as `INFO`, `DEBUG`, `ERROR`, etc.
+- Can be configured to capture only specified levels, such as `ERROR`
+- Can include manually-defined messages produced inside certain Dagster definitions like assets, ops, and sensors
+
+For more information on customizing loggers, see "[Customizing Dagster's built-in loggers](custom-logging)".
+
+## Integrating external loggers (Experimental)
+
+In addition to built-in loggers, you can also integrate external loggers to augment Dagster's default logs and configure them to display in the UI. Other options, such as custom handlers and formatters, can be configured in your project's `dagster.yaml`.
+
+Refer to the [Python logging guide](python-logging) for more information and examples.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/logging/python-logging.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/logging/python-logging.md
new file mode 100644
index 0000000000000..bc5408e1d1777
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/monitor/logging/python-logging.md
@@ -0,0 +1,111 @@
+---
+title: "Python logging"
+sidebar_position: 200
+---
+
+Dagster is compatible and configurable with [Python's logging module](https://docs.python.org/3/library/logging.html). Configuration options are set in a [`dagster.yaml` file](/guides/deploy/dagster-yaml), which will apply the contained settings to any run launched from the instance.
+
+Configuration settings include:
+
+- The Python loggers to capture from
+- The log level loggers are set to
+- The handlers/formatters used to process log messages produced by runs
+
+## Relevant APIs
+
+| Name | Description |
+| -------------------------------------------------------- | --------------------------------------------------------------------------------------- |
+| | A function that returns a Python logger that will automatically be captured by Dagster. |
+
+## Production environments and event log storage
+
+In a production context, it's recommended that you be selective about which logs are captured. It's possible to overload the event log storage with these events, which may cause some pages in the UI to take a long time to load.
+
+To mitigate this, you can:
+
+- Capture only the most critical logs
+- Avoid including debug information if a large amount of run history will be maintained
+
+## Capturing Python logs (Experimental) \{#capturing-python-logs}
+
+By default, logs generated using the Python logging module aren't captured into the Dagster ecosystem. This means that they aren't stored in the Dagster event log, will not be associated with any Dagster metadata (such as step key, run ID, etc.), and won't show up in the default view of the [Dagster UI](/guides/operate/webserver).
+
+For example, imagine you have the following code:
+
+
+
+Because this code uses a custom Python logger instead of `context.log`, the log statement won't be added as an event to the Dagster event log or show up in the UI.
+
+However, this default behavior can be changed to treat these sort of log statements the same as `context.log` calls. This can be accomplished by setting the `managed_python_loggers` key in `dagster.yaml` file to a list of Python logger names that you would like to capture:
+
+
+
+Once this key is set, Dagster will treat any normal Python log call from one of the listed loggers in the exact same way as a `context.log` call. This means you should be able to see this log statement in the UI:
+
+![Log Python error](/images/guides/monitor/logging/log-python-error.png)
+
+:::note
+
+If `python_log_level` is set, the loggers listed here will be set to the given level before a run is launched. Refer to the [Configuring global log levels](#configuring-global-log-levels) for more info and an example.
+
+:::
+
+
+## Configuring global log levels (Experimental) \{#configuring-global-log-levels}
+
+To set a global log level in a Dagster instance, set the `python_log_level` parameter in your instance's `dagster.yaml` file.
+
+This setting controls the log level of all loggers managed by Dagster. By default, this will just be the `context.log` logger. If there are custom Python loggers that you want to capture, refer to the [Capturing Python logs section](#capturing-python-logs).
+
+Setting a global log level allows you to filter out logs below a given level. For example, setting a log level of `INFO` will filter out all `DEBUG` level logs:
+
+
+
+## Configuring Python log handlers (Experimental)
+
+In your `dagster.yaml` file, you can configure handlers, formatters and filters that will apply to the Dagster instance. This will apply the same logging configuration to all runs.
+
+For example:
+
+
+
+Handler, filter and formatter configuration follows the [dictionary config schema format](https://docs.python.org/3/library/logging.config.html#logging-config-dictschema) in the Python logging module. Only the `handlers`, `formatters` and `filters` dictionary keys will be accepted, as Dagster creates loggers internally.
+
+From there, standard `context.log` calls will output with the configured handlers, formatters and filters. After execution, read the output log file `my_dagster_logs.log`. As expected, the log file contains the formatted log:
+
+![Log file output](/images/guides/monitor/logging/log-file-output.png)
+
+## Examples
+
+### Creating a captured Python logger without modifying dagster.yaml
+
+To create a logger that's captured by Dagster without modifying your `dagster.yaml` file, use the utility function. This pattern is useful when logging from inside nested functions, or other cases where it would be inconvenient to thread through the context parameter to enable calls to `context.log`.
+
+For example:
+
+
+
+:::note
+
+The logging module retains global state, meaning the logger returned by this function will be identical if is called multiple times with the same arguments in the same process. This means that there may be unpredictable or unituitive results if you set the level of the returned Python logger to different values in different parts of your code.
+
+:::
+
+### Outputting Dagster logs to a file
+
+If you want to output all Dagster logs to a file, use the Python logging module's built-in [`logging.FileHandler`](https://docs.python.org/3/library/logging.handlers.html#logging.FileHandler) class. This sends log output to a disk file.
+
+To enable this, define a new `myHandler` handler in your `dagster.yaml` file to be a `logging.FileHandler` object:
+
+
+
+
+You can also configure a formatter to apply a custom format to the logs. For example, to include a timestamp with the logs, we defined a custom formatter named `timeFormatter` and attached it to `myHandler`.
+
+If we execute the following job:
+
+
+
+And then read the `my_dagster_logs.log` output log file, we'll see the log file contains the formatted log:
+
+![Log file output](/images/guides/monitor/logging/log-file-output.png)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/configuration/advanced-config-types.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/configuration/advanced-config-types.md
new file mode 100644
index 0000000000000..52475b2fa7bfd
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/configuration/advanced-config-types.md
@@ -0,0 +1,368 @@
+---
+title: "Advanced config types"
+description: Dagster's config system supports a variety of more advanced config types.
+sidebar_position: 200
+---
+
+In some cases, you may want to define a more complex [config schema](run-configuration) for your assets and ops. For example, you may want to define a config schema that takes in a list of files or complex data. In this guide, we'll walk through some common patterns for defining more complex config schemas.
+
+## Attaching metadata to config fields
+
+Config fields can be annotated with metadata, which can be used to provide additional information about the field, using the Pydantic class.
+
+For example, we can annotate a config field with a description, which will be displayed in the documentation for the config field. We can add a value range to a field, which will be validated when config is specified.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_metadata_config endbefore=end_metadata_config dedent=4
+from dagster import Config
+from pydantic import Field
+
+class MyMetadataConfig(Config):
+ person_name: str = Field(description="The name of the person to greet")
+ age: int = Field(gt=0, lt=100, description="The age of the person to greet")
+
+# errors, since age is not in the valid range!
+MyMetadataConfig(person_name="Alice", age=200)
+```
+
+## Defaults and optional config fields
+
+Config fields can have an attached default value. Fields with defaults are not required, meaning they do not need to be specified when constructing the config object.
+
+For example, we can attach a default value of `"hello"` to the `greeting_phrase` field, and can construct `MyAssetConfig` without specifying a phrase. Fields which are marked as `Optional`, such as `person_name`, implicitly have a default value of `None`, but can also be explicitly set to `None` as in the example below.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_optional_config endbefore=end_optional_config dedent=4
+from typing import Optional
+from dagster import asset, Config, materialize, RunConfig
+from pydantic import Field
+
+class MyAssetConfig(Config):
+ person_name: Optional[str] = None
+
+ # can pass default to pydantic.Field to attach metadata to the field
+ greeting_phrase: str = Field(
+ default="hello", description="The greeting phrase to use."
+ )
+
+@asset
+def greeting(config: MyAssetConfig) -> str:
+ if config.person_name:
+ return f"{config.greeting_phrase} {config.person_name}"
+ else:
+ return config.greeting_phrase
+
+asset_result = materialize(
+ [greeting],
+ run_config=RunConfig({"greeting": MyAssetConfig()}),
+)
+```
+
+### Required config fields
+
+By default, fields which are typed as `Optional` are not required to be specified in the config, and have an implicit default value of `None`. If you want to require that a field be specified in the config, you may use an ellipsis (`...`) to [require that a value be passed](https://docs.pydantic.dev/usage/models/#required-fields).
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_required_config endbefore=end_required_config dedent=4
+from typing import Optional, Callable
+from dagster import asset, Config
+from pydantic import Field
+
+class MyAssetConfig(Config):
+ # ellipsis indicates that even though the type is Optional,
+ # an input is required
+ person_first_name: Optional[str] = ...
+
+ # ellipsis can also be used with pydantic.Field to attach metadata
+ person_last_name: Optional[Callable] = Field(
+ default=..., description="The last name of the person to greet"
+ )
+
+@asset
+def goodbye(config: MyAssetConfig) -> str:
+ full_name = f"{config.person_first_name} {config.person_last_name}".strip()
+ if full_name:
+ return f"Goodbye, {full_name}"
+ else:
+ return "Goodbye"
+
+# errors, since person_first_name and person_last_name are required
+goodbye(MyAssetConfig())
+
+# works, since both person_first_name and person_last_name are provided
+goodbye(MyAssetConfig(person_first_name="Alice", person_last_name=None))
+```
+
+## Basic data structures
+
+Basic Python data structures can be used in your config schemas along with nested versions of these data structures. The data structures which can be used are:
+
+- `List`
+- `Dict`
+- `Mapping`
+
+For example, we can define a config schema that takes in a list of user names and a mapping of user names to user scores.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_basic_data_structures_config endbefore=end_basic_data_structures_config dedent=4
+from dagster import Config, materialize, asset, RunConfig
+from typing import List, Dict
+
+class MyDataStructuresConfig(Config):
+ user_names: list[str]
+ user_scores: dict[str, int]
+
+@asset
+def scoreboard(config: MyDataStructuresConfig): ...
+
+result = materialize(
+ [scoreboard],
+ run_config=RunConfig(
+ {
+ "scoreboard": MyDataStructuresConfig(
+ user_names=["Alice", "Bob"],
+ user_scores={"Alice": 10, "Bob": 20},
+ )
+ }
+ ),
+)
+```
+
+## Nested schemas
+
+Schemas can be nested in one another, or in basic Python data structures.
+
+Here, we define a schema which contains a mapping of user names to complex user data objects.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_nested_schema_config endbefore=end_nested_schema_config dedent=4
+from dagster import asset, materialize, Config, RunConfig
+from typing import Dict
+
+class UserData(Config):
+ age: int
+ email: str
+ profile_picture_url: str
+
+class MyNestedConfig(Config):
+ user_data: dict[str, UserData]
+
+@asset
+def average_age(config: MyNestedConfig): ...
+
+result = materialize(
+ [average_age],
+ run_config=RunConfig(
+ {
+ "average_age": MyNestedConfig(
+ user_data={
+ "Alice": UserData(
+ age=10,
+ email="alice@gmail.com",
+ profile_picture_url=...,
+ ),
+ "Bob": UserData(
+ age=20,
+ email="bob@gmail.com",
+ profile_picture_url=...,
+ ),
+ }
+ )
+ }
+ ),
+)
+```
+
+## Permissive schemas
+
+By default, `Config` schemas are strict, meaning that they will only accept fields that are explicitly defined in the schema. This can be cumbersome if you want to allow users to specify arbitrary fields in their config. For this purpose, you can use the `PermissiveConfig` base class, which allows arbitrary fields to be specified in the config.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_permissive_schema_config endbefore=end_permissive_schema_config dedent=4
+from dagster import asset, PermissiveConfig
+from typing import Optional
+import requests
+
+class FilterConfig(PermissiveConfig):
+ title: Optional[str] = None
+ description: Optional[str] = None
+
+@asset
+def filtered_listings(config: FilterConfig):
+ # extract all config fields, including those not defined in the schema
+ url_params = config.dict()
+ return requests.get("https://my-api.com/listings", params=url_params).json()
+
+# can pass in any fields, including those not defined in the schema
+filtered_listings(FilterConfig(title="hotel", beds=4))
+```
+
+## Union types
+
+Union types are supported using Pydantic [discriminated unions](https://docs.pydantic.dev/usage/types/#discriminated-unions-aka-tagged-unions). Each union type must be a subclass of . The `discriminator` argument to specifies the field that will be used to determine which union type to use. Discriminated unions provide comparable functionality to the `Selector` type in the legacy Dagster config APIs.
+
+Here, we define a config schema which takes in a `pet` field, which can be either a `Cat` or a `Dog`, as indicated by the `pet_type` field.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_union_schema_config endbefore=end_union_schema_config dedent=4
+from dagster import asset, materialize, Config, RunConfig
+from pydantic import Field
+from typing import Union
+from typing_extensions import Literal
+
+class Cat(Config):
+ pet_type: Literal["cat"] = "cat"
+ meows: int
+
+class Dog(Config):
+ pet_type: Literal["dog"] = "dog"
+ barks: float
+
+class ConfigWithUnion(Config):
+ pet: Union[Cat, Dog] = Field(discriminator="pet_type")
+
+@asset
+def pet_stats(config: ConfigWithUnion):
+ if isinstance(config.pet, Cat):
+ return f"Cat meows {config.pet.meows} times"
+ else:
+ return f"Dog barks {config.pet.barks} times"
+
+result = materialize(
+ [pet_stats],
+ run_config=RunConfig(
+ {
+ "pet_stats": ConfigWithUnion(
+ pet=Cat(meows=10),
+ )
+ }
+ ),
+)
+```
+
+### YAML and config dictionary representations of union types
+
+The YAML or config dictionary representation of a discriminated union is structured slightly differently than the Python representation. In the YAML representation, the discriminator key is used as the key for the union type's dictionary. For example, a `Cat` object would be represented as:
+
+```yaml
+pet:
+ cat:
+ meows: 10
+```
+
+In the config dictionary representation, the same pattern is used:
+
+```python
+{
+ "pet": {
+ "cat": {
+ "meows": 10,
+ }
+ }
+}
+```
+
+## Enum types
+
+Python enums which subclass `Enum` are supported as config fields. Here, we define a schema that takes in a list of users, whose roles are specified as enum values:
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_enum_schema_config endbefore=end_enum_schema_config dedent=4
+from dagster import Config, RunConfig, op, job
+from enum import Enum
+
+class UserPermissions(Enum):
+ GUEST = "guest"
+ MEMBER = "member"
+ ADMIN = "admin"
+
+class ProcessUsersConfig(Config):
+ users_list: dict[str, UserPermissions]
+
+@op
+def process_users(config: ProcessUsersConfig):
+ for user, permission in config.users_list.items():
+ if permission == UserPermissions.ADMIN:
+ print(f"{user} is an admin")
+
+@job
+def process_users_job():
+ process_users()
+
+op_result = process_users_job.execute_in_process(
+ run_config=RunConfig(
+ {
+ "process_users": ProcessUsersConfig(
+ users_list={
+ "Bob": UserPermissions.GUEST,
+ "Alice": UserPermissions.ADMIN,
+ }
+ )
+ }
+ ),
+)
+```
+
+### YAML and config dictionary representations of enum types
+
+The YAML or config dictionary representation of a Python enum uses the enum's name. For example, a YAML specification of the user list above would be:
+
+```yaml
+users_list:
+ Bob: GUEST
+ Alice: ADMIN
+```
+
+In the config dictionary representation, the same pattern is used:
+
+```python
+{
+ "users_list": {
+ "Bob": "GUEST",
+ "Alice": "ADMIN",
+ }
+}
+```
+
+## Validated config fields
+
+Config fields can have custom validation logic applied using [Pydantic validators](https://docs.pydantic.dev/usage/validators/). Pydantic validators are defined as methods on the config class, and are decorated with the `@validator` decorator. These validators are triggered when the config class is instantiated. In the case of config defined at runtime, a failing validator will not prevent the launch button from being pressed, but will raise an exception and prevent run start.
+
+Here, we define some validators on a configured user's name and username, which will throw exceptions if incorrect values are passed in the launchpad or from a schedule or sensor.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_validated_schema_config endbefore=end_validated_schema_config dedent=4
+from dagster import Config, RunConfig, op, job
+from pydantic import validator
+
+class UserConfig(Config):
+ name: str
+ username: str
+
+ @validator("name")
+ def name_must_contain_space(cls, v):
+ if " " not in v:
+ raise ValueError("must contain a space")
+ return v.title()
+
+ @validator("username")
+ def username_alphanumeric(cls, v):
+ assert v.isalnum(), "must be alphanumeric"
+ return v
+
+executed = {}
+
+@op
+def greet_user(config: UserConfig) -> None:
+ print(f"Hello {config.name}!")
+ executed["greet_user"] = True
+
+@job
+def greet_user_job() -> None:
+ greet_user()
+
+# Input is valid, so this will work
+op_result = greet_user_job.execute_in_process(
+ run_config=RunConfig(
+ {"greet_user": UserConfig(name="Alice Smith", username="alice123")}
+ ),
+)
+
+# Name has no space, so this will fail
+op_result = greet_user_job.execute_in_process(
+ run_config=RunConfig(
+ {"greet_user": UserConfig(name="John", username="johndoe44")}
+ ),
+)
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/configuration/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/configuration/index.md
new file mode 100644
index 0000000000000..a2c811618f4ab
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/configuration/index.md
@@ -0,0 +1,8 @@
+---
+title: "Configuration"
+sidebar_position: 10
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/configuration/run-configuration.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/configuration/run-configuration.md
new file mode 100644
index 0000000000000..a5541448ddf42
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/configuration/run-configuration.md
@@ -0,0 +1,180 @@
+---
+title: "Run configuration"
+description: Job run configuration allows providing parameters to jobs at the time they're executed.
+sidebar_position: 100
+---
+
+Run configuration allows providing parameters to jobs at the time they're executed.
+
+It's often useful to provide user-chosen values to Dagster jobs or asset definitions at runtime. For example, you might want to provide a connection URL for a database resource. Dagster exposes this functionality through a configuration API.
+
+Various Dagster entities (assets, ops, resources) can be individually configured. When launching a job that materializes (assets), executes (ops), or instantiates (resources) a configurable entity, you can provide _run configuration_ for each entity. Within the function that defines the entity, you can access the passed-in configuration through the `config` parameter. Typically, the provided run configuration values correspond to a _configuration schema_ attached to the asset/op/resource definition. Dagster validates the run configuration against the schema and proceeds only if validation is successful.
+
+A common use of configuration is for a [schedule](/guides/automate/schedules/) or [sensor](/guides/automate/sensors/) to provide configuration to the job run it is launching. For example, a daily schedule might provide the day it's running on to one of the assets as a config value, and that asset might use that config value to decide what day's data to read.
+
+## Defining and accessing configuration
+
+Configurable parameters accepted by an asset or op are specified by defining a config model subclass of and a `config` parameter to the corresponding asset or op function. Under the hood, these config models utilize [Pydantic](https://docs.pydantic.dev/), a popular Python library for data validation and serialization.
+
+During execution, the specified config is accessed within the body of the op or asset using the `config` parameter.
+
+
+
+
+Here, we define a subclass of holding a single string value representing the name of a user. We can access the config through the `config` parameter in the asset body.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_basic_asset_config endbefore=end_basic_asset_config dedent=4
+from dagster import asset, Config
+
+class MyAssetConfig(Config):
+ person_name: str
+
+@asset
+def greeting(config: MyAssetConfig) -> str:
+ return f"hello {config.person_name}"
+```
+
+
+
+
+Here, we define a subclass of holding a single string value representing the name of a user. We can access the config through the `config` parameter in the op body.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_basic_op_config endbefore=end_basic_op_config dedent=4
+from dagster import op, Config
+
+class MyOpConfig(Config):
+ person_name: str
+
+@op
+def print_greeting(config: MyOpConfig):
+ print(f"hello {config.person_name}")
+```
+
+You can also build config into jobs.
+
+
+
+
+These examples showcase the most basic config types that can be used. For more information on the set of config types Dagster supports, see [the advanced config types documentation](advanced-config-types).
+
+## Defining and accessing Pythonic configuration for a resource
+
+Configurable parameters for a resource are defined by specifying attributes for a resource class, which subclasses . The below resource defines a configurable connection URL, which can be accessed in any methods defined on the resource.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_basic_resource_config endbefore=end_basic_resource_config dedent=4
+from dagster import op, ConfigurableResource
+
+class MyDatabaseResource(ConfigurableResource):
+ connection_url: str
+
+ def query(self, query: str):
+ return get_engine(self.connection_url).execute(query)
+```
+
+For more information on using resources, refer to the [Resources guide](/guides/build/external-resources/).
+
+## Specifying runtime configuration
+
+To execute a job or materialize an asset that specifies config, you'll need to provide values for its parameters. How we provide these values depends on the interface we are using: Python, the Dagster UI, or the command line (CLI).
+
+
+
+
+When specifying config from the Python API, we can use the `run_config` argument for or . This takes a object, within which we can supply config on a per-op or per-asset basis. The config is specified as a dictionary, with the keys corresponding to the op/asset names and the values corresponding to the config values.
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_execute_with_config endbefore=end_execute_with_config dedent=4
+from dagster import job, materialize, op, RunConfig
+
+@job
+def greeting_job():
+ print_greeting()
+
+job_result = greeting_job.execute_in_process(
+ run_config=RunConfig({"print_greeting": MyOpConfig(person_name="Alice")})
+)
+
+asset_result = materialize(
+ [greeting],
+ run_config=RunConfig({"greeting": MyAssetConfig(person_name="Alice")}),
+)
+```
+
+
+
+
+From the UI's **Launchpad** tab, you can supply config as YAML using the config editor. Here, the YAML schema matches the layout of the defined config class. The editor has typeahead, schema validation, and schema documentation.
+
+You can also click the **Scaffold Missing Config** button to generate dummy values based on the config schema. Note that a modal containing the Launchpad editor will pop up if you attempt to materialize an asset with a defined `config`.
+
+![Config in the Dagster UI](/images/guides/operate/config-ui.png)
+
+
+
+
+### Command line
+
+When executing a job from Dagster's CLI with [`dagster job execute`](/api/python-api/cli#dagster-job), you can put config in a YAML file:
+
+```YAML file=/concepts/configuration/good.yaml
+ops:
+ op_using_config:
+ config:
+ person_name: Alice
+```
+
+And then pass the file path with the `--config` option:
+
+```bash
+dagster job execute --config my_config.yaml
+```
+
+
+
+
+## Validation
+
+Dagster validates any provided run config against the corresponding Pydantic model. It will abort execution with a or Pydantic `ValidationError` if validation fails. For example, both of the following will fail, because there is no `nonexistent_config_value` in the config schema:
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_execute_with_bad_config endbefore=end_execute_with_bad_config dedent=4
+@job
+def greeting_job():
+ print_greeting()
+
+op_result = greeting_job.execute_in_process(
+ run_config=RunConfig(
+ {"print_greeting": MyOpConfig(nonexistent_config_value=1)}
+ ),
+)
+
+asset_result = materialize(
+ [greeting],
+ run_config=RunConfig({"greeting": MyAssetConfig(nonexistent_config_value=1)}),
+)
+```
+
+### Using environment variables with config
+
+Assets and ops can be configured using environment variables by passing an when constructing a config object. This is useful when the value is sensitive or may vary based on environment. If using Dagster+, environment variables can be [set up directly in the UI](/guides/deploy/using-environment-variables-and-secrets).
+
+```python file=/guides/dagster/pythonic_config/pythonic_config.py startafter=start_execute_with_config_envvar endbefore=end_execute_with_config_envvar dedent=4
+from dagster import job, materialize, op, RunConfig, EnvVar
+
+job_result = greeting_job.execute_in_process(
+ run_config=RunConfig(
+ {"print_greeting": MyOpConfig(person_name=EnvVar("PERSON_NAME"))}
+ )
+)
+
+asset_result = materialize(
+ [greeting],
+ run_config=RunConfig(
+ {"greeting": MyAssetConfig(person_name=EnvVar("PERSON_NAME"))}
+ ),
+)
+```
+
+Refer to the [Environment variables and secrets guide](/guides/deploy/using-environment-variables-and-secrets) for more general info about environment variables in Dagster.
+
+## Next steps
+
+Config is a powerful tool for making Dagster pipelines more flexible and observable. For a deeper dive into the supported config types, see [the advanced config types documentation](advanced-config-types). For more information on using resources, which are a powerful way to encapsulate reusable logic, see [the Resources guide](/guides/build/external-resources).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/graphql/graphql-client.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/graphql/graphql-client.md
new file mode 100644
index 0000000000000..bdd1d59aa7632
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/graphql/graphql-client.md
@@ -0,0 +1,174 @@
+---
+title: "Dagster GraphQL Python client"
+description: Dagster provides a Python client to interact with its GraphQL API
+---
+
+Dagster provides a GraphQL Python Client to interface with [Dagster's GraphQL API](index.md) from Python.
+
+## Relevant APIs
+
+| Name | Description |
+| ----------------------------------------------------------------------- | -------------------------------------------------------------------- |
+| | The client class to interact with Dagster's GraphQL API from Python. |
+| | The exception that the client raises upon a response error. |
+
+## Overview
+
+The Dagster Python Client provides bindings in Python to programmatically interact with Dagster's GraphQL API.
+
+When is this useful? Dagster exposes a powerful GraphQL API, but this level of flexibility is not always necessary. For example, when submitting a new job run, you may only want to think about the job name and configuration and to think less about maintaining a long GraphQL query.
+
+`DagsterGraphQLClient` provides a way to solve this issue by providing a module with a simple interface to interact with the GraphQL API.
+
+Note that all GraphQL methods on the API are not yet available in Python - the `DagsterGraphQLClient` currently only provides the following methods:
+
+-
+-
+-
+-
+
+## Using the GraphQL Client
+
+The snippet below shows example instantiation of the client:
+
+{/* TODO convert to */}
+```python file=/concepts/webserver/graphql/client_example.py startafter=start_setup_marker endbefore=end_setup_marker
+from dagster_graphql import DagsterGraphQLClient
+
+client = DagsterGraphQLClient("localhost", port_number=3000)
+```
+
+If you are using Dagster+, you can configure your client against the Dagster+ API by passing your deployment-specific URL and a User Token to the client as follows:
+
+```python file=/concepts/webserver/graphql/client_example.py startafter=start_cloud_usage endbefore=end_cloud_usage
+url = "yourorg.dagster.cloud/prod" # Your deployment-scoped url
+user_token = ( # a User Token generated from the Organization Settings page in Dagster+.
+ "your_token_here"
+)
+client = DagsterGraphQLClient(url, headers={"Dagster-Cloud-Api-Token": user_token})
+```
+
+## Examples
+
+### Getting a job run's status
+
+You can use the client to get the status of a job run as follows:
+
+{/* TODO convert to */}
+```python file=/concepts/webserver/graphql/client_example.py startafter=start_run_status_marker endbefore=end_run_status_marker
+from dagster_graphql import DagsterGraphQLClientError
+
+from dagster import DagsterRunStatus
+
+try:
+ status: DagsterRunStatus = client.get_run_status(RUN_ID)
+ if status == DagsterRunStatus.SUCCESS:
+ do_something_on_success()
+ else:
+ do_something_else()
+except DagsterGraphQLClientError as exc:
+ do_something_with_exc(exc)
+ raise exc
+```
+
+### Reloading all repositories in a repository location
+
+You can also reload a repository location in a Dagster deployment.
+
+This reloads all repositories in that repository location. This is useful in a variety of contexts, including refreshing the Dagster UI without restarting the server. Example usage is as follows:
+
+```python file=/concepts/webserver/graphql/client_example.py startafter=start_reload_repo_location_marker endbefore=end_reload_repo_location_marker
+from dagster_graphql import ReloadRepositoryLocationInfo, ReloadRepositoryLocationStatus
+
+reload_info: ReloadRepositoryLocationInfo = client.reload_repository_location(REPO_NAME)
+if reload_info.status == ReloadRepositoryLocationStatus.SUCCESS:
+ do_something_on_success()
+else:
+ raise Exception(
+ "Repository location reload failed because of a "
+ f"{reload_info.failure_type} error: {reload_info.message}"
+ )
+```
+
+### Submitting a job run
+
+You can use the client to submit a job run as follows:
+
+```python file=/concepts/webserver/graphql/client_example.py startafter=start_submit_marker_default endbefore=end_submit_marker_default
+from dagster_graphql import DagsterGraphQLClientError
+
+try:
+ new_run_id: str = client.submit_job_execution(
+ JOB_NAME,
+ repository_location_name=REPO_LOCATION_NAME,
+ repository_name=REPO_NAME,
+ run_config={},
+ )
+ do_something_on_success(new_run_id)
+except DagsterGraphQLClientError as exc:
+ do_something_with_exc(exc)
+ raise exc
+```
+
+### Shutting down a repository location server
+
+If you're running your own gRPC server, we generally recommend updating your repository code by building a new Docker image with a new tag and redeploying your server using that new image, but sometimes you may want to restart your server without changing the image (for example, if your job definitions are generated programatically from a database, and you want to restart the server and re-generate your repositories even though the underlying Python code hasn't changed). In these situations, `reload_repository_location` is insufficient, since it refreshes the UI's information about the repositories but doesn't actually restart the server or reload the repository definition.
+
+One way to cause your server to restart and your repositories to be reloaded is to run your server in an environment like Kubernetes that automatically restarts services when they fail (or docker-compose with `restart: always` set on the service), and then use the `shutdown_repository_location` function on the GraphQL client to shut down the server. The server will then be restarted by your environment, which will be automatically detected by the UI.
+
+Example usage:
+
+{/* TODO convert to */}
+```python file=/concepts/webserver/graphql/client_example.py startafter=start_shutdown_repo_location_marker endbefore=end_shutdown_repo_location_marker
+from dagster_graphql import (
+ ShutdownRepositoryLocationInfo,
+ ShutdownRepositoryLocationStatus,
+)
+
+shutdown_info: ShutdownRepositoryLocationInfo = client.shutdown_repository_location(
+ REPO_NAME
+)
+if shutdown_info.status == ShutdownRepositoryLocationStatus.SUCCESS:
+ do_something_on_success()
+else:
+ raise Exception(f"Repository location shutdown failed: {shutdown_info.message}")
+```
+
+#### Repository location and repository inference
+
+Note that specifying the repository location name and repository name are not always necessary; the GraphQL client will infer the repository name and repository location name if the job name is unique.
+
+{/* TODO convert to */}
+```python file=/concepts/webserver/graphql/client_example.py startafter=start_submit_marker_job_name_only endbefore=end_submit_marker_job_name_only
+from dagster_graphql import DagsterGraphQLClientError
+
+try:
+ new_run_id: str = client.submit_job_execution(
+ JOB_NAME,
+ run_config={},
+ )
+ do_something_on_success(new_run_id)
+except DagsterGraphQLClientError as exc:
+ do_something_with_exc(exc)
+ raise exc
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/graphql/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/graphql/index.md
new file mode 100644
index 0000000000000..8d6f6408a0b06
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/graphql/index.md
@@ -0,0 +1,233 @@
+---
+title: "Dagster GraphQL API"
+description: Dagster exposes a GraphQL API that allows clients to interact with Dagster programmatically
+sidebar_position: 60
+---
+
+:::note
+
+The GraphQL API is still evolving and is subject to breaking changes. A large portion of the API is primarily for internal use by the [Dagster webserver](/guides/operate/webserver).
+For any of the queries below, we will be clear about breaking changes in release notes.
+
+:::
+
+
+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
+
+## Using the GraphQL API
+
+The GraphQL API is served from the[webserver](/guides/operate/webserver). To start the server, run the following:
+
+```shell
+dagster dev
+```
+
+The webserver serves the GraphQL endpoint at the `/graphql` endpoint. If you are running the webserver locally on port 3000, you can access the API at [http://localhost:3000/graphql](http://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](/images/guides/operate/graphql/playground.png)
+
+### 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](/images/guides/operate/graphql/docs.png)
+
+## Python client
+
+Dagster also provides a Python client to interface with Dagster's GraphQL API from Python. For more information, see "[Dagster Python GraphQL client](graphql-client)".
+
+## Example queries
+
+- [Get a list of Dagster runs](#get-a-list-of-dagster-runs)
+- [Get a list of repositories](#get-a-list-of-repositories)
+- [Get a list of jobs within a repository](#get-a-list-of-jobs-within-a-repository)
+- [Launch a run](#launch-a-run)
+- [Terminate an in-progress run](#terminate-an-in-progress-run)
+
+### Get a list of Dagster runs
+
+
+
+
+You may eventually accumulate too many runs to return in one query. The `runsOrError` query takes in optional `cursor` and `limit` arguments for pagination:
+
+```shell
+query PaginatedRunsQuery($cursor: String) {
+ runsOrError(
+ cursor: $cursor
+ limit: 10
+ ) {
+ __typename
+ ... on Runs {
+ results {
+ runId
+ jobName
+ status
+ runConfigYaml
+ startTime
+ endTime
+ }
+ }
+ }
+}
+```
+
+
+
+
+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:
+
+```shell
+query FilteredRunsQuery($cursor: String) {
+ runsOrError(
+ filter: { statuses: [FAILURE] }
+ cursor: $cursor
+ limit: 10
+ ) {
+ __typename
+ ... on Runs {
+ results {
+ runId
+ jobName
+ status
+ runConfigYaml
+ startTime
+ endTime
+ }
+ }
+ }
+}
+```
+
+
+
+
+### Get a list of repositories
+
+This query returns the names and location names of all the repositories currently loaded:
+
+```shell
+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.
+
+```shell
+query JobsQuery(
+ $repositoryLocationName: String!
+ $repositoryName: String!
+) {
+ repositoryOrError(
+ repositorySelector: {
+ repositoryLocationName: $repositoryLocationName
+ repositoryName: $repositoryName
+ }
+ ) {
+ ... on Repository {
+ jobs {
+ name
+ }
+ }
+ }
+}
+```
+
+### Launch a run
+
+To launch a run, use the `launchRun` mutation. Here, we define `LaunchRunMutation` to wrap our mutation and pass in the required arguments as query variables. For this query, the required arguments are:
+
+- `selector` - A dictionary that contains the repository location name, repository name, and job name.
+- `runConfigData` - The run config for the job execution. **Note**: 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 doesn't, the mutation returns a `RunConfigValidationInvalid` response.
+
+```shell
+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.
+
+```shell
+mutation TerminateRun($runId: String!) {
+ terminateRun(runId: $runId){
+ __typename
+ ... on TerminateRunSuccess{
+ run {
+ runId
+ }
+ }
+ ... on TerminateRunFailure {
+ message
+ }
+ ... on RunNotFoundError {
+ runId
+ }
+ ... on PythonError {
+ message
+ stack
+ }
+ }
+}
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/index.md
new file mode 100644
index 0000000000000..8d1bc9055dc8d
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/index.md
@@ -0,0 +1,8 @@
+---
+title: "Operating pipelines"
+sidebar_class_name: hidden
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/managing-concurrency.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/managing-concurrency.md
new file mode 100644
index 0000000000000..38131329a1b9b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/managing-concurrency.md
@@ -0,0 +1,165 @@
+---
+title: "Managing concurrency of Dagster assets, jobs, and Dagster instances"
+sidebar_label: Managing concurrency
+description: How to limit the number of runs a job, or assets for an instance of Dagster.
+sidebar_position: 50
+---
+
+You often want to control the number of concurrent runs for a Dagster job, a specific asset, or for a type of asset or job. Limiting concurrency in your data pipelines can help prevent performance problems and downtime.
+
+
+:::note
+
+This article assumes familiarity with [assets](/guides/build/assets/) and [jobs](/guides/build/assets/asset-jobs)
+
+:::
+
+## Limit how many jobs can be running at the same time
+
+
+* Dagster Core, add the following to your [dagster.yaml](/guides/deploy/dagster-yaml)
+* In Dagster+, add the following to your [deployment settings](/dagster-plus/deployment/management/deployments/deployment-settings-reference)
+
+```yaml
+run_queue:
+ max_concurrent_runs: 15
+```
+
+
+
+
+## Limit how many ops or assets can be running at the same time
+
+You can control the number of assets or ops that are running concurrently within a job using the `config` argument of `dg.define_asset_job()` or `dg.@job()` for ops.
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+## Limit how many of a certain type of op or asset can run across all runs
+
+You can configure a limit for all ops or assets with a specific tag key or key-value pair. Ops or assets above that limit will be queued. Use `tag_concurrency_limits` in the job's config, either in Python or using the Launchpad in the Dagster UI.
+
+For example, you might want to limit the number of ops or assets that are running with a key of `database` across all runs (to limit the load on that database).
+
+:::warning
+This feature is experimental and is only supported with Postgres/MySQL storage.
+:::
+
+
+```yaml
+# dagster.yaml for Dagster Core; Deployment Settings for Dagster+
+run_coordinator:
+ module: dagster.core.run_coordinator
+ class: QueuedRunCoordinator
+ config:
+ tag_concurrency_limits:
+ - key: "dagster/concurrency_key"
+ value: "database"
+ limit: 1
+```
+
+To specify a global concurrency limit using the CLI, use:
+
+```
+dagster instance concurrency set database 1
+```
+
+A default concurrency limit can be configured for the instance, for any concurrency keys that don't have an explicit limit set:
+
+* Dagster+: Edit the `concurrency` config in deployment settings via the [Dagster+ UI](/guides/operate/webserver) or the [`dagster-cloud` CLI](/dagster-plus/deployment/management/dagster-cloud-cli/).
+* Dagster Open Source: Use your instance's [dagster.yaml](/guides/deploy/dagster-yaml)
+
+To enable this default value, use `concurrency.default_op_concurrency_limit`. For example, the following would set the default concurrency value for the deployment to 1:
+```yaml
+concurrency:
+ default_op_concurrency_limit: 1
+```
+
+
+
+
+
+
+
+
+
+
+
+
+You can also limit concurrency for a tag within the job definition, for example to limit the number of specific assets running at the same time *within* that run.
+
+
+
+
+
+
+
+
+
+
+
+
+## Override job level concurrency in the Launchpad
+
+You can override the default job-level settings, such as the value of the `max_concurrent` key for a job, by launching a job in the Launchpad in the Dagster UI.
+
+Need screenshot here
+
+## Prevent runs from starting if another run is already occurring (advanced)
+
+You can use Dagster's rich metadata to use a schedule or a sensor to only start a run when there are no currently running jobs.
+
+{/* TODO fix this code example */}
+
+
+
+## Troubleshooting
+
+When limiting concurrency, you might run into some issues until you get the configuration right.
+
+### Runs going to STARTED status and skipping QUEUED
+
+:::info
+This only applies to Dagster Open Source.
+:::
+
+The `run_queue` key may not be set in your instance's settings. In the Dagster UI, navigate to Deployment > Configuration and verify that the `run_queue` key is set.
+
+### Runs remaining in QUEUED status
+
+The possible causes for runs remaining in `QUEUED` status depend on whether you're using Dagster+ or Dagster Open Source.
+
+
+
+ If runs aren't being dequeued in Dagster+, the root causes could be:
+ * **If using a [hybrid deployment](/dagster-plus/deployment/deployment-types/hybrid)**, the agent serving the deployment may be down. In this situation, runs will be paused.
+ * **Dagster+ is experiencing downtime**. Check the [status page](https://dagstercloud.statuspage.io/) for the latest on potential outages.
+
+
+
+ If runs aren't being dequeued in Dagster Open Source, the root cause is likely an issue with the Dagster daemon or the run queue configuration.
+
+ **Troubleshoot the Dagster daemon**
+
+ * **Verify the Dagster daemon is set up and running.** In the Dagster UI, navigate to **Deployment > Daemons** and verify that the daemon is running. The **Run queue** should also be running. If you used [dagster dev](/guides/operate/webserver) to start the Dagster UI, the daemon should have been started for you. If the daemon isn't running, proceed to step 2.
+ * **Verify the Dagster daemon can access the same storage as the Dagster webserver process.** Both the webserver process and the Dagster daemon should access the same storage, meaning they should use the same `dagster.yaml`. Locally, this means both processes should have the same set `DAGSTER_HOME` environment variable. If you used dagster dev to start the Dagster UI, both processes should be using the same storage. Refer to the [Dagster Instance docs](/guides/deploy/dagster-instance-configuration) for more information.
+
+ **Troubleshoot the run queue configuration**
+ If the daemon is running, runs may intentionally be left in the queue due to concurrency rules. To investigate:
+ * **Check the output logged from the daemon process**, as this will include skipped runs.
+ * **Check the max_concurrent_runs setting in your instance's dagster.yaml**. If set to 0, this may block the queue. You can check this setting in the Dagster UI by navigating to Deployment > Configuration and locating the run_queue.max_concurrent_runs setting. Refer to the Limiting overall runs section for more info.
+ * **Check the state of your run queue**. In some cases, the queue may be blocked by some number of in-progress runs. To view the status of your run queue, click **Runs** in the top navigation of the Dagster UI and then open the **Queued** and **In Progress** tabs.
+
+ If there are queued or in-progress runs blocking the queue, you can terminate them to allow other runs to proceed.
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/run-executors.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/run-executors.md
new file mode 100644
index 0000000000000..c2e14f8934628
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/run-executors.md
@@ -0,0 +1,99 @@
+---
+title: "Run executors"
+description: Executors are responsible for executing steps within a job run.
+sidebar_position: 40
+---
+
+Executors are responsible for executing steps within a job run. Once a run has launched and the process for the run (the [run worker](/guides/deploy/oss-deployment-architecture#job-execution-flow)) is allocated and started, the executor assumes responsibility for execution.
+
+Executors can range from single-process serial executors to managing per-step computational resources with a sophisticated control plane.
+
+## Relevant APIs
+
+| Name | Description |
+| ---------------------------------------- | -------------------------------------------------------------------------------------------- |
+| | The decorator used to define executors. Defines an . |
+| | An executor definition. |
+
+## Specifying executors
+
+- [Directly on jobs](#directly-on-jobs)
+- [For a code location](#for-a-code-location)
+
+### Directly on jobs
+
+Every job has an executor. The default executor is the , which by default executes each step in its own process. This executor can be configured to execute each step within the same process.
+
+An executor can be specified directly on a job by supplying an to the `executor_def` parameter of or :
+
+{/* TODO convert to */}
+```python file=/deploying/executors/executors.py startafter=start_executor_on_job endbefore=end_executor_on_job
+from dagster import graph, job, multiprocess_executor
+
+
+# Providing an executor using the job decorator
+@job(executor_def=multiprocess_executor)
+def the_job(): ...
+
+
+@graph
+def the_graph(): ...
+
+
+# Providing an executor using graph_def.to_job(...)
+other_job = the_graph.to_job(executor_def=multiprocess_executor)
+```
+
+### For a code location
+
+To specify a default executor for all jobs and assets provided to a code location, supply the `executor` argument to the object.
+
+If a job explicitly specifies an executor, then that executor will be used. Otherwise, jobs that don't specify an executor will use the default provided to the code location:
+
+{/* TODO convert to */}
+```python file=/deploying/executors/executors.py startafter=start_executor_on_repo endbefore=end_executor_on_repo
+from dagster import multiprocess_executor, define_asset_job, asset, Definitions
+
+
+@asset
+def the_asset():
+ pass
+
+
+asset_job = define_asset_job("the_job", selection="*")
+
+
+@job
+def op_job(): ...
+
+
+# op_job and asset_job will both use the multiprocess_executor,
+# since neither define their own executor.
+
+defs = Definitions(
+ assets=[the_asset], jobs=[asset_job, op_job], executor=multiprocess_executor
+)
+```
+
+:::note
+
+Executing a job via overrides the job's executor and uses instead.
+
+:::
+
+## Example executors
+
+| Name | Description |
+|------|-------------|
+| | Execution plan executes serially within the run worker itself. |
+| | Executes each step within its own spawned process. Has a configurable level of parallelism. |
+| | Executes each step within a Dask task. |
+| | Executes each step within a Celery task. |
+| | Executes each step within an ephemeral Kubernetes pod. |
+| | Executes each step within an ephemeral Kubernetes pod. |
+| | Executes each step within a ephemeral Kubernetes pod, using Celery as a control plane for prioritization and queuing. |
+| | Executes each step within a Docker container, using Celery as a control plane for prioritization and queueing. |
+
+## Custom executors
+
+The executor system is pluggable, meaning it's possible to write your own executor to target a different execution substrate. Note that this is not currently well-documented and the internal APIs continue to be in flux.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/ui-user-settings.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/ui-user-settings.md
new file mode 100644
index 0000000000000..0380daf294e7b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/ui-user-settings.md
@@ -0,0 +1,20 @@
+---
+title: "Managing user settings in the Dagster UI"
+sidebar_label: "User settings in the UI"
+sidebar_position: 30
+---
+
+The **User settings** page in the [Dagster UI](webserver) allows you to define settings like your timezone and theme and enable experimental features.
+
+## Accessing your settings & preferences
+
+To access your settings and preferences:
+
+- **In Dagster Open Source (OSS)**: Click the **gear icon** in the upper right corner of the UI
+- **In Dagster+**: Click the **user menu (your icon) > User Settings** in the upper right corner of the UI
+
+A window will display where you can define settings.
+
+## Experimental feature settings
+
+Use the toggles next to the features in the **Experimental features** section of your **User settings** to enable and disable new features. We'd love your feedback!
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/webserver.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/webserver.md
new file mode 100644
index 0000000000000..9c66c04150ab8
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/operate/webserver.md
@@ -0,0 +1,469 @@
+---
+title: Dagster webserver
+description: "The Dagster UI is a web-based interface for Dagster. You can inspect Dagster objects (ex: assets, jobs, schedules), launch runs, view launched runs, and view assets produced by those runs."
+sidebar_position: 20
+---
+
+The Dagster webserver serves the Dagster UI, a web-based interface for viewing and interacting with Dagster objects. It also responds to GraphQL queries.
+
+In the UI, you can inspect Dagster objects (ex: assets, jobs, schedules), launch runs, view launched runs, and view assets produced by those runs.
+
+## Launching the webserver
+
+The easiest way to launch the webserver from the command line during local development is to run:
+
+```shell
+dagster dev
+```
+
+This command launches both the Dagster webserver and the [Dagster daemon](/guides/deploy/execution/dagster-daemon), allowing you to start a full local deployment of Dagster from the command line.
+
+The command will print out the URL you can access the UI from in the browser, usually on port 3000.
+
+When invoked, the webserver will fetch definitions - such as assets, jobs, schedules, sensors, and resources - from a object in a Python module or package or the code locations configured in an open source deployment's [workspace files](/guides/deploy/code-locations/workspace-yaml). For more information, see the [code locations documentation](/guides/deploy/code-locations/).
+
+You can also launch the webserver by itself from the command line by running:
+
+```shell
+dagster-webserver
+```
+
+Note that several Dagster features, like schedules and sensors, require the Dagster daemon to be running in order to function.
+
+## Overview page
+
+- **Description**: This page, also known as the "factory floor", provides a high-level look at the activity in your Dagster deployment, across all code locations. This includes information about runs, jobs, schedules, sensors, resources, and backfills, all of which can be accessed using the tabs on this page.
+
+- **Accessed by**: Clicking **Overview** in the top navigation bar
+
+![The Overview tab, also known as the Factory Floor, in the Dagster UI](/images/guides/operate/webserver/factory-floor.png)
+
+## Assets
+
+
+
+
+**Asset catalog (OSS)**
+
+- **Description**: The **Asset catalog** page lists all [assets](/guides/build/assets/) in your Dagster deployment, which can be filtered by asset key, compute kind, asset group, [code location](/guides/deploy/code-locations/), and [tags]/guides/build/assets/metadata-and-tags/index.md#tags). Clicking an asset opens the **Asset details** page for that asset. You can also navigate to the **Global asset lineage** page, reload definitions, and materialize assets.
+
+- **Accessed by:** Clicking **Assets** in the top navigation bar
+
+![The Asset Catalog page in the Dagster UI](/images/guides/operate/webserver/asset-catalog.png)
+
+
+
+
+**Asset catalog (Dagster+ Pro)**
+
+:::note
+
+This feature is only available in Dagster+ Pro.
+
+:::
+
+- **Description**: This version of the **Asset catalog** page includes all the information and functionality of the original page, broken out by compute kind, asset group, [code location](/guides/deploy/code-locations/), [tags]/guides/build/assets/metadata-and-tags/index.md#tags), and [owners]/guides/build/assets/metadata-and-tags/index.md#owners), etc. On this page, you can:
+
+ - View all [assets](/guides/build/assets/) in your Dagster deployment
+ - View details about a specific asset by clicking on it
+ - Search assets by asset key, compute kind, asset group, code location, tags, owners, etc.
+ - Access the global asset lineage
+ - Reload definitions
+
+- **Accessed by:** Clicking **Catalog** in the top navigation
+
+![The Asset Catalog page in the Dagster UI](/images/guides/operate/webserver/asset-catalog-cloud-pro.png)
+
+
+
+
+**Catalog views (Dagster+)**
+
+- **Description**: **Catalog views** save a set of filters against the **Asset catalog** to show only the assets you want to see. You can share these views for easy access and faster team collaboration. With **Catalog views**, you can:
+
+ - Filter for a scoped set of [assets](/guides/build/assets) in your Dagster deployment
+ - Create shared views of assets for easier team collaboration
+
+- **Accessed by:**
+
+ - Clicking **Catalog** in the top navigation
+ - **From the Global asset lineage**: Clicking **View global asset lineage**, located near the top right corner of the **Catalog** page
+
+![The Catalog views dropdown in the Dagster+ Pro Catalog UI](/images/guides/operate/webserver/catalog-views.png)
+
+
+
+
+**Global asset lineage**
+
+- **Description**: The **Global asset lineage** page displays dependencies between all of the assets in your Dagster deployment, across all code locations. On this page, you can:
+
+ - Filter assets by group
+ - Filter a subset of assets by using [asset selection syntax](/guides/build/assets/asset-selection-syntax)
+ - Reload definitions
+ - Materialize all or a selection of assets
+ - View run details for the latest materialization of any asset
+
+- **Accessed by**:
+
+ - **From the Asset catalog**: Clicking **View global asset lineage**, located near the top right corner of the page
+ - **From the Asset details page**: Clicking the **Lineage tab**
+
+![The Global asset lineage page in the Dagster UI](/images/guides/operate/webserver/global-asset-lineage.png)
+
+
+
+
+**Asset details**
+
+- **Description**: The **Asset details** page contains details about a single asset. Use the tabs on this page to view detailed information about the asset:
+
+ - **Overview** - Information about the asset such as its description, resources, config, type, etc.
+ - **Partitions** - The asset's partitions, including their materialization status, metadata, and run information
+ - **Events** - The asset's materialization history
+ - **Checks** - The [Asset checks](/guides/test/asset-checks) defined for the asset
+ - **Lineage** - The asset's lineage in the **Global asset lineage** page
+ - **Automation** - The [Declarative Automation conditions](/guides/automate/declarative-automation) associated with the asset
+ - **Insights** - **Dagster+ only.** Historical information about the asset, such as failures and credit usage. Refer to the [Dagster+ Insights](/dagster-plus/features/insights/) documentation for more information.
+
+- **Accessed by**: Clicking an asset in the **Asset catalog**
+
+![The Asset Details page in the Dagster UI](/images/guides/operate/webserver/asset-details.png)
+
+
+
+
+## Runs
+
+
+
+
+**All runs**
+
+- **Description**: The **Runs** page lists all job runs, which can be filtered by job name, run ID, execution status, or tag. Click a run ID to open the **Run details** page and view details for that run.
+
+- **Accessed by**: Clicking **Runs** in the top navigation bar
+
+![UI Runs page](/images/guides/operate/webserver/runs-page.png)
+
+
+
+
+**Run details**
+
+- **Description**: The **Run details** page contains details about a single run, including timing information, errors, and logs. The upper left pane contains a Gantt chart, indicating how long each asset or op took to execute. The bottom pane displays filterable events and logs emitted during execution.
+
+ In this page, you can:
+
+ - **View structured event and raw compute logs.** Refer to the run logs tab for more info.
+ - **Re-execute a run** using the same configuration by clicking the **Re-execute** button. Related runs (e.g., runs created by re-executing the same previous run) are grouped in the right pane for easy reference
+
+- **Accessed by**: Clicking a run in the **Run details** page
+
+![UI Run details page](/images/guides/operate/webserver/run-details.png)
+
+
+
+
+**Run logs**
+
+- **Description**: Located at the bottom of the **Run details** page, the run logs list every event that occurred in a run, the type of event, and detailed information about the event itself. There are two types of logs, which we'll discuss in the next section:
+
+ - Structured event logs
+ - Raw compute logs
+
+- **Accessed by**: Scrolling to the bottom of the **Run details** page
+
+**Structured event logs**
+
+- **Description**: Structured logs are enriched and categorized with metadata. For example, a label of which asset a log is about, links to an asset’s metadata, and what type of event it is available. This structuring also enables easier filtering and searching in the logs.
+
+- **Accessed by**: Clicking the **left side** of the toggle next to the log filter field
+
+![Structured event logs in the Run details page](/images/guides/operate/webserver/run-details-event-logs.png)
+
+**Raw compute logs**
+
+- **Description**: The raw compute logs contain logs for both [`stdout` and `stderr`](https://stackoverflow.com/questions/3385201/confused-about-stdin-stdout-and-stderr), which you can toggle between. To download the logs, click the **arrow icon** near the top right corner of the logs.
+
+- **Accessed by**: Clicking the **right side** of the toggle next to the log filter field
+
+![Raw compute logs in the Run details page](/images/guides/operate/webserver/run-details-compute-logs.png)
+
+
+
+
+## Schedules
+
+
+
+
+**All schedules**
+
+- **Description**: The **Schedules** page lists all [schedules](/guides/automate/schedules) defined in your Dagster deployment, as well as information about upcoming ticks for anticipated scheduled runs. Click a schedule to open the **Schedule details** page.
+
+- **Accessed by**: Clicking **Overview (top nav) > Schedules tab**
+
+![UI Schedules page](/images/guides/operate/webserver/schedules-tab.png)
+
+
+
+
+**Schedule details**
+
+- **Description**: The **Schedule details** page contains details about a single schedule, including its next tick, tick history, and run history. Clicking the **Test schedule** button near the top right corner of the page allows you to test the schedule.
+
+- **Accessed by**: Clicking a schedule in the **Schedules** page.
+
+![UI Schedule details page](/images/guides/operate/webserver/schedule-details.png)
+
+
+
+
+## Sensors
+
+
+
+
+**All sensors**
+
+- **Description**: The **Sensors** page lists all [sensors](/guides/automate/sensors) defined in your Dagster deployment, as well as information about the sensor's frequency and its last tick. Click a sensor to view details about the sensor, including its recent tick history and recent runs.
+
+- **Accessed by**: Clicking **Overview (top nav) > Sensors tab**
+
+![UI Sensors page](/images/guides/operate/webserver/sensors-tab.png)
+
+
+
+
+**Sensor details**
+
+- **Description**: The **Sensor details** page contains details about a single sensor, including its next tick, tick history, and run history. Clicking the **Test sensor** button near the top right corner of the page allows you to test the sensor.
+
+- **Accessed by**: Clicking a sensor in the **Sensors** page
+
+![UI Sensor details page](/images/guides/operate/webserver/sensor-details.png)
+
+
+
+
+## Resources
+
+
+
+
+**All resources**
+
+- **Description**: The **Resources** page lists all [resources](/guides/build/external-resources/) defined in your Dagster deployment, across all code locations. Clicking a resource will open the **Resource details** page.
+
+- **Accessed by**: Clicking **Overview (top nav) > Resources tab**
+
+![UI Resources page](/images/guides/operate/webserver/resources-tab.png)
+
+
+
+
+**Resource details**
+
+- **Description**: The **Resource details** page contains detailed information about a resource, including its configuration, description, and uses. Click the tabs below for more information about the tabs on this page.
+
+- **Accessed by**: Clicking a resource in the **Resources** page.
+
+
+
+
+**Configuration tab**
+
+- **Description**: The **Configuration** tab contains detailed information about a resource's configuration, including the name of each key, type, and value of each config value. If a key's value is an [environment variable](/guides/deploy/using-environment-variables-and-secrets), an `Env var` badge will display next to the value.
+
+- **Accessed by**: On the **Resource details** page, clicking the **Configuration tab**
+
+![UI Resource details - Configuration tab](/images/guides/operate/webserver/resource-details-configuration-tab.png)
+
+
+
+
+**Uses tab**
+
+- **Description**: The **Uses** tab contains information about the other Dagster definitions that use the resource, including [assets](/guides/build/assets/), [jobs](/guides/build/assets/asset-jobs), and [ops](/guides/build/ops). Clicking on any of these definitions will open the details page for that definition type.
+
+- **Accessed by**: On the **Resource details* page, clicking the **Uses tab**
+
+![UI Resource details - Uses tab](/images/guides/operate/webserver/resource-details-uses-tab.png)
+
+
+
+
+
+
+## Backfills
+
+- **Description**: The **Backfills** tab contains information about the backfills in your Dagster deployment, across all code locations. It includes information about when the partition was created, its target, status, run status, and more.
+
+- **Accessed by**: Clicking **Overview (top nav) > Backfills tab**
+
+![UI Backfills tab](/images/guides/operate/webserver/backfills-tab.png)
+
+## Jobs
+
+
+
+
+**All jobs**
+
+- **Description**: The **Jobs** page lists all [jobs](/guides/build/assets/asset-jobs) defined in your Dagster deployment across all code locations. It includes information about the job's schedule or sensor, its latest run time, and its history. Click a job to open the **Job details** page.
+
+- **Accessed by**: Clicking **Overview (top nav) > Jobs tab**
+
+![UI Job Definition](/images/guides/operate/webserver/jobs-tab.png)
+
+
+
+
+**Job details**
+
+- **Description**: The **Job details** page contains detailed information about a job. Click the tabs below for more information about the tabs on this page.
+
+- **Accessed by**: Clicking a job in the **Jobs** page.
+
+
+
+
+**Overview tab**
+
+- **Description**: The **Overview** tab in the **Job details** page shows the graph of assets and/or ops that make up a job.
+
+- **Accessed by:** On the **Job details** page, clicking the **Overview** tab
+
+![UI Job Definition](/images/guides/operate/webserver/job-definition-with-ops.png)
+
+
+
+
+**Launchpad tab**
+
+- **Description**: The **Launchpad tab** provides a configuration editor to let you experiment with configuration and launch runs. **Note**: For assets, this tab will only display if a job requires config. It displays by default for all op jobs.
+
+- **Accessed by:** On the **Job details** page, clicking the **Launchpad** tab
+
+![UI Launchpad](/images/guides/operate/webserver/job-config-with-ops.png)
+
+
+
+
+**Runs tab**
+
+- **Description**: The **Runs** tab displays a list of recent runs for a job. Clicking a run will open the [**Run details** page.
+
+- **Accessed by:** On the **Job details** page, clicking the **Runs** tab
+
+![UI Job runs tab](/images/guides/operate/webserver/jobs-runs-tab.png)
+
+
+
+
+**Partitions tab**
+
+- **Description**: The **Partitions** tab displays information about the [partitions](/guides/build/partitions-and-backfills) associated with the job, including the total number of partitions, the number of missing partitions, and the job's backfill history. **Note**: This tab will display only if the job contains partitions.
+
+- **Accessed by:** On the **Job details** page, clicking the **Partitions** tab
+
+![UI Job Partitions tab](/images/guides/operate/webserver/jobs-partitions-tab.png)
+
+
+
+
+
+
+
+## Deployment
+
+The **Deployment** page includes information about the status of the code locations in your Dagster deployment, daemon (Open Source) or agent (Cloud) health, schedules, sensors, and configuration details.
+
+
+
+
+**Code locations tab**
+
+- **Description**: The **Code locations** tab contains information about the code locations in your Dagster deployment, including their current status, when they were last updated, and high-level details about the definitions they contain. You can reload Dagster definitions by:
+
+ - Clicking **Reload all** to reload all definitions in all code locations
+ - Clicking **Reload** next to a specific code location to reload only that code location's definitions
+
+- **Accessed by**:
+ - Clicking **Deployment** in the top navigation bar
+ - On the **Deployment overview** page, clicking the **Code locations** tab
+
+![UI Deployment overview page](/images/guides/operate/webserver/deployment-code-locations.png)
+
+
+
+
+**Open Source (OSS)**
+
+In addition to the **Code locations** tab, Dagster OSS deployments contain a few additional tabs. Click the tabs below for more information.
+
+
+
+
+**Daemons tab**
+
+- **Description**: The **Daemons** tab contains information about the [daemons](/guides/deploy/execution/dagster-daemon) in an Open Source Dagster deployment, including their current status and when their last heartbeat was detected.
+- **Accessed by**: On the **Deployment overview** page, clicking the **Daemons** tab
+
+![UI Deployment - Daemons tab](/images/guides/operate/webserver/deployment-daemons-tab.png)
+
+
+
+
+**Configuration tab**
+
+- **Description**: The **Configuration** tab displays information about the configuration for a Dagster deployment, which is managed through the [`dagster.yaml`](/guides/deploy/dagster-yaml) file
+- **Accessed by**: On the **Deployment overview** page, clicking the **Configuration** tab
+
+![UI Deployment - Configuration tab](/images/guides/operate/webserver/deployment-configuration-tab.png)
+
+
+
+
+
+
+
+**Dagster+**
+
+In addition to the **Code locations** tab, Dagster+ deployments contain a few additional tabs. Click the tabs below for more information.
+
+
+
+
+**Agents tab**
+
+- **Description**: The **Agents** tab contains information about the agents in a Dagster+ deployment.
+- **Accessed by**: On the **Deployment overview** page, clicking the **Agents** tab
+
+![UI Dagster+ Deployment - Agents tab](/images/guides/operate/webserver/deployment-cloud-agents-tab.png)
+
+
+
+
+**Environment variables tab**
+
+- **Description**: The **Agents** tab contains information about the environment variables configured in a Dagster+ deployment. Refer to the [Dagster+ environment variables documentation](/dagster-plus/deployment/management/environment-variables/) for more info.
+- **Accessed by**: On the **Deployment overview** page, clicking the **Environment variables** tab
+
+![UI Cloud Deployment - Environment variables tab](/images/guides/operate/webserver/deployment-cloud-environment-variables-tab.png)
+
+
+
+
+**Alerts tab**
+
+- **Description**: The **Alerts** tab contains information about the alert policies configured for a Dagster+ deployment. Refer to the [Dagster+ alerts guide](/dagster-plus/features/alerts/) for more info.
+- **Accessed by**: On the **Deployment overview** page, clicking the **Alerts** tab
+
+![UI Dagster+ Deployment - Alerts tab](/images/guides/operate/webserver/deployment-cloud-alerts-tab.png)
+
+
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/asset-checks.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/asset-checks.md
new file mode 100644
index 0000000000000..15bbe955c036c
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/asset-checks.md
@@ -0,0 +1,81 @@
+---
+title: "Testing assets with asset checks"
+sidebar_position: 100
+sidebar_label: "Asset checks"
+---
+
+Asset checks are tests that verify specific properties of your data assets, allowing you to execute data quality checks on your data. For example, you can create checks to:
+
+- Ensure a particular column doesn't contain null values
+- Verify that a tabular asset adheres to a specified schema
+- Check if an asset's data needs refreshing
+
+Each asset check should test only a single asset property to keep tests uncomplicated, reusable, and easy to track over time.
+
+:::note
+
+This article assumes familiarity with [assets](/guides/build/assets/)
+
+:::
+
+## Getting started
+
+To get started with asset checks, follow these general steps:
+
+1. **Define an asset check:** Asset checks are typically defined using the `@asset_check` or `@multi_asset_check` decorator and run either within an asset or separate from the asset.
+2. **Pass the asset checks to the `Definitions` object:** Asset checks must be added to `Definitions` for Dagster to recognize them.
+3. **Choose how to execute asset checks**: By default, all jobs targeting an asset will also run associated checks, although you can run asset checks through the Dagster UI.
+4. **View asset check results in the UI**: Asset check results will appear in the UI and can be customized through the use of metadata and severity levels
+5. **Alert on failed asset check results**: If you are using Dagster+, you can choose to alert on asset checks.
+
+## Defining a single asset check \{#single-check}
+
+:::tip
+Dagster's dbt integration can model existing dbt tests as asset checks. Refer to the [dagster-dbt documentation](/integrations/libraries/dbt) for more information.
+:::
+
+A asset check is defined using the `@asset_check` decorator.
+
+The following example defines an asset check on an asset that fails if the `order_id` column of the asset contains a null value. The asset check will run after the asset has been materialized.
+
+
+
+## Defining multiple asset checks \{#multiple-checks}
+
+In most cases, checking the data quality of an asset will require multiple checks.
+
+The following example defines two asset checks using the `@multi_asset_check` decorator:
+
+- One check that fails if the `order_id` column of the asset contains a null value
+- Another check that fails if the `item_id` column of the asset contains a null value
+
+In this example, both asset checks will run in a single operation after the asset has been materialized.
+
+
+
+## Programmatically generating asset checks \{#factory-pattern}
+
+Defining multiple checks can also be done using a factory pattern. The example below defines the same two asset checks as in the previous example, but this time using a factory pattern and the `@multi_asset_check` decorator.
+
+
+
+## Blocking downstream materialization
+
+By default, if a parent's asset check fails during a run, the run will continue and downstream assets will be materialized. To prevent this behavior, set the `blocking` argument to `True` in the `@asset_check` decorator.
+
+In the example bellow, if the `orders_id_has_no_nulls` check fails, the downstream `augmented_orders` asset won't be materialized.
+
+
+
+## Scheduling and monitoring asset checks
+
+In some cases, running asset checks separately from the job materializing the assets can be useful. For example, running all data quality checks once a day and sending an alert if they fail. This can be achieved using schedules and sensors.
+
+In the example below, two jobs are defined: one for the asset and another for the asset check. Schedules are defined to materialize the asset and execute the asset check independently. A sensor is defined to send an email alert when the asset check job fails.
+
+
+
+## Next steps
+
+- Learn more about [assets](/guides/build/assets/)
+- Learn how to use [Great Expectations with Dagster](https://dagster.io/blog/ensuring-data-quality-with-dagster-and-great-expectations)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/data-freshness-testing.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/data-freshness-testing.md
new file mode 100644
index 0000000000000..783b15a96f6cb
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/data-freshness-testing.md
@@ -0,0 +1,80 @@
+---
+title: "Data freshness checks"
+sidebar_position: 300
+---
+
+Freshness checks provide a way to identify data assets that are overdue for an update. For example, you can use freshness checks to identify stale assets caused by:
+
+- The pipeline hitting an error and failing
+- Runs not being scheduled
+- A backed up run queue
+- Runs taking longer than expected to complete
+
+Freshness checks can also communicate SLAs for their data freshness. For example, downstream asset consumers can determine how often assets are expected to be updated by looking at the defined checks.
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need familiarity with:
+
+- [Assets](/guides/build/assets/defining-assets)
+- [External assets](/guides/build/assets/external-assets)
+- [Asset checks](/guides/test/asset-checks)
+
+
+
+## Getting started
+
+To get started with freshness checks, follow these general steps:
+
+1. **Define a freshness check**: Freshness checks are defined using `build_last_update_freshness_checks`, which utilizes an asset's last updated time to determine freshness.
+
+ **If using Dagster+ Pro**, you can also use [`build_anomaly_detection_freshness_checks`](#anomaly-detection) to define a freshness check that uses an anomaly detection model to determine freshness.
+2. **Define a schedule or sensor**: Defining a schedule or sensor (`build_sensor_for_freshness_checks`) is required to ensure the freshness check executes. If the check only runs after the asset has been materialized, the check won't be able to detect the times materialization fails.
+3. **Pass the freshness check and schedule/sensor to the `Definitions` object**: Freshness checks and the associated schedule or sensor must be added to a `Definitions` object for Dagster to recognize them.
+4. **View the freshness check results in the Dagster UI**: Freshness check results will appear in the UI, allowing you to track the results over time.
+
+## Materializable asset freshness \{#materializable-assets}
+
+Materializable assets are assets materialized by Dagster. To calculate whether a materializable asset is overdue, Dagster uses the asset's last materialization timestamp.
+
+The example below defines a freshness check on an asset that fails if the asset's latest materialization occurred more than one hour before the current time.
+
+
+
+## External asset freshness \{#external-assets}
+
+[External assets](/guides/build/assets/external-assets) are assets orchestrated by systems other than Dagster.
+
+To run freshness checks on external assets, the checks need to know when the external assets were last updated. Emitting these update timestamps as values for the `dagster/last_updated_timestamp` observation metadata key allows Dagster to calculate whether the asset is overdue.
+
+The example below defines a freshness check and adds a schedule to run the check periodically.
+
+
+
+### Testing freshness with anomaly detection \{#anomaly-detection}
+
+:::note
+Anomaly detection is a Dagster+ Pro feature.
+:::
+
+Instead of applying policies on an asset-by-asset basis, Dagster+ Pro users can use `build_anomaly_detection_freshness_checks` to take advantage of a time series anomaly detection model to determine if data arrives later than expected.
+
+
+
+:::note
+If the asset hasn't been updated enough times, the check will pass with a message indicating that more data is needed to detect anomalies.
+:::
+
+## Alerting on overdue assets
+
+:::note
+Freshness check alerts are a Dagster+ feature.
+:::
+
+In Dagster+, you can set up alerts to notify you when assets are overdue for an update. Refer to the [Dagster+ alerting guide](/dagster-plus/features/alerts) for more information.
+
+## Next steps
+
+- Explore more [asset checks](/guides/test/asset-checks)
+- Explore how to [raise alerts when assets are overdue](/dagster-plus/features/alerts) (Dagster+ Pro)
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/index.md
new file mode 100644
index 0000000000000..31ceb625d0813
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/index.md
@@ -0,0 +1,8 @@
+---
+title: "Testing assets"
+sidebar_class_name: hidden
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/running-a-subset-of-asset-checks.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/running-a-subset-of-asset-checks.md
new file mode 100644
index 0000000000000..649f657b070a2
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/running-a-subset-of-asset-checks.md
@@ -0,0 +1,122 @@
+---
+title: "Running a subset of asset checks"
+sidebar_position: 200
+---
+
+In some cases, you may only want to execute some of the [asset checks](asset-checks) defined in a or . For example, you may want to materialize a without executing the checks or only execute a certain set of checks.
+
+In this guide, we'll show you a few approaches to subsetting asset checks in and .
+
+:::note
+
+This article assumes familiarity with [asset checks](asset-checks) and [multi-assets](/guides/build/assets/defining-assets#multi-asset).
+
+:::
+
+## Subsetting checks in @multi_asset_checks
+
+Using the decorator's `specs` and `can_subset` arguments, you can execute a subset of checks in a single op.
+
+{/* TODO link this to proper API doc */}
+Inside the body of the function, we can use `AssetCheckExecutionContext.selected_asset_check_keys` to identify which computations to run. We can also set the decorator's `can_subset` parameter to `True` to execute a subset of the asset checks that the computation contains.
+
+As we don't know in advance which checks will be executed, we explicitly `yield` each asset check result that we're expected to create:
+
+```python file=/concepts/assets/asset_checks/subset_multi_asset_check.py
+from collections.abc import Iterable
+
+from dagster import (
+ AssetCheckExecutionContext,
+ AssetCheckKey,
+ AssetCheckResult,
+ AssetCheckSpec,
+ AssetKey,
+ multi_asset_check,
+)
+
+
+@multi_asset_check(
+ specs=[
+ AssetCheckSpec(name="asset_check_one", asset="asset_one"),
+ AssetCheckSpec(name="asset_check_two", asset="asset_two"),
+ ],
+ can_subset=True,
+)
+def the_check(context: AssetCheckExecutionContext) -> Iterable[AssetCheckResult]:
+ if (
+ AssetCheckKey(AssetKey("asset_one"), "asset_check_one")
+ in context.selected_asset_check_keys
+ ):
+ yield AssetCheckResult(
+ passed=True, metadata={"foo": "bar"}, check_name="asset_check_one"
+ )
+ if (
+ AssetCheckKey(AssetKey("asset_two"), "asset_check_two")
+ in context.selected_asset_check_keys
+ ):
+ yield AssetCheckResult(
+ passed=True, metadata={"foo": "bar"}, check_name="asset_check_two"
+ )
+```
+
+## Subsetting checks in @multi_assets
+
+When using [multi-assets](/guides/build/assets/defining-assets#multi-asset), Dagster assumes that all checks specified on the asset should be executed after it is materialized. This means that attempting to execute some, but not all, of the checks defined by a multi-asset will result in an error.
+
+In the following example, we only want to execute a check when the `multi_asset_piece_1` asset produced by the `multi_asset_1_and_2` multi-asset is materialized:
+
+{/* TODO convert to */}
+```python file=/concepts/assets/asset_checks/subset_check_multi_asset.py
+from dagster import (
+ AssetCheckKey,
+ AssetCheckResult,
+ AssetCheckSpec,
+ AssetExecutionContext,
+ AssetKey,
+ AssetSpec,
+ MaterializeResult,
+ multi_asset,
+)
+
+
+@multi_asset(
+ specs=[
+ AssetSpec("multi_asset_piece_1", group_name="asset_checks", skippable=True),
+ AssetSpec("multi_asset_piece_2", group_name="asset_checks", skippable=True),
+ ],
+ check_specs=[AssetCheckSpec("my_check", asset="multi_asset_piece_1")],
+ can_subset=True,
+)
+def multi_asset_1_and_2(context: AssetExecutionContext):
+ if AssetKey("multi_asset_piece_1") in context.selected_asset_keys:
+ yield MaterializeResult(asset_key="multi_asset_piece_1")
+ # The check will only execute when multi_asset_piece_1 is materialized
+ if (
+ AssetCheckKey(AssetKey("multi_asset_piece_1"), "my_check")
+ in context.selected_asset_check_keys
+ ):
+ yield AssetCheckResult(passed=True, metadata={"foo": "bar"})
+ if AssetKey("multi_asset_piece_2") in context.selected_asset_keys:
+ # No check on multi_asset_piece_2
+ yield MaterializeResult(asset_key="multi_asset_piece_2")
+```
+
+Let's review what we did to accomplish this:
+
+- In the decorator:
+ - For each in the decorator's `specs` argument, set the `skippable` parameter on to `True`. This allows the asset to be skipped when the multi-asset is materialized.
+ - Set the decorator's `can_subset` parameter to `True`, allowing a subset of the computation's assets to be executed
+- Use `AssetExecutionContext.selected_asset_keys` to identify which computations to run
+- For each asset the multi-asset could create,] explicitly `yield` a as we don't know in advance which assets will be executed
+- Use `AssetExecutionContext.selected_asset_check_keys` to determine which asset check to run. In this example, the `my_check` check will only execute when `multi_asset_piece_1` is materialized.
+
+## APIs in this guide
+
+| Name | Description |
+| ------------------------------------------------- | ------------------------------------------------------------------------------------------------------------------------------------- |
+| | A decorator used to define a set of asset checks that execute together in the same op. |
+| | The class returned by asset checks. |
+| | Defines the severity of a given asset check result. |
+| | A class that's passed to asset decorators to define checks that execute in the same op as the asset. |
+| | A decorator used to define [multi-assets](/guides/build/assets/defining-assets#multi-asset). |
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/testing-partitioned-config-and-jobs.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/testing-partitioned-config-and-jobs.md
new file mode 100644
index 0000000000000..02a1b4d0e27f1
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/testing-partitioned-config-and-jobs.md
@@ -0,0 +1,116 @@
+---
+title: "Testing partitioned config and jobs"
+description: Test your partition configuration and jobs.
+sidebar_position: 500
+---
+
+In this article, we'll cover a few ways to test your partitioned config and jobs.
+
+:::note
+
+This article assumes familiarity with [partitioned assets](/guides/build/partitions-and-backfills/partitioning-assets).
+
+:::
+
+## Testing partitioned config
+
+Invoking a object directly invokes the decorated function.
+
+If you want to check whether the generated run config is valid for the config of a job, you can use the function.
+
+{/* TODO convert to */}
+```python file=/concepts/partitions_schedules_sensors/partitioned_config_test.py startafter=start_partition_config endbefore=end_partition_config
+from dagster import validate_run_config, daily_partitioned_config
+from datetime import datetime
+
+
+@daily_partitioned_config(start_date=datetime(2020, 1, 1))
+def my_partitioned_config(start: datetime, _end: datetime):
+ return {
+ "ops": {
+ "process_data_for_date": {"config": {"date": start.strftime("%Y-%m-%d")}}
+ }
+ }
+
+
+def test_my_partitioned_config():
+ # assert that the decorated function returns the expected output
+ run_config = my_partitioned_config(datetime(2020, 1, 3), datetime(2020, 1, 4))
+ assert run_config == {
+ "ops": {"process_data_for_date": {"config": {"date": "2020-01-03"}}}
+ }
+
+ # assert that the output of the decorated function is valid configuration for the
+ # partitioned_op_job job
+ assert validate_run_config(partitioned_op_job, run_config)
+```
+
+If you want to test that a creates the partitions you expect, use the `get_partition_keys` or `get_run_config_for_partition_key` functions:
+
+{/* TODO convert to */}
+```python file=/concepts/partitions_schedules_sensors/partitioned_config_test.py startafter=start_partition_keys endbefore=end_partition_keys
+from dagster import Config, OpExecutionContext
+
+
+@daily_partitioned_config(start_date=datetime(2020, 1, 1), minute_offset=15)
+def my_offset_partitioned_config(start: datetime, _end: datetime):
+ return {
+ "ops": {
+ "process_data": {
+ "config": {
+ "start": start.strftime("%Y-%m-%d-%H:%M"),
+ "end": _end.strftime("%Y-%m-%d-%H:%M"),
+ }
+ }
+ }
+ }
+
+
+class ProcessDataConfig(Config):
+ start: str
+ end: str
+
+
+@op
+def process_data(context: OpExecutionContext, config: ProcessDataConfig):
+ s = config.start
+ e = config.end
+ context.log.info(f"processing data for {s} - {e}")
+
+
+@job(config=my_offset_partitioned_config)
+def do_more_stuff_partitioned():
+ process_data()
+
+
+def test_my_offset_partitioned_config():
+ # test that the partition keys are what you expect
+ keys = my_offset_partitioned_config.get_partition_keys()
+ assert keys[0] == "2020-01-01"
+ assert keys[1] == "2020-01-02"
+
+ # test that the run_config for a partition is valid for partitioned_op_job
+ run_config = my_offset_partitioned_config.get_run_config_for_partition_key(keys[0])
+ assert validate_run_config(do_more_stuff_partitioned, run_config)
+
+ # test that the contents of run_config are what you expect
+ assert run_config == {
+ "ops": {
+ "process_data": {
+ "config": {"start": "2020-01-01-00:15", "end": "2020-01-02-00:15"}
+ }
+ }
+ }
+```
+
+## Testing partitioned jobs
+
+{/* TODO fix the API docs so this can be a PyObject */}
+
+To run a partitioned job in-process on a particular partition, supply a value for the `partition_key` argument of [`dagster.JobDefinition.execute_in_process`](/api/python-api/execution):
+
+{/* TODO convert to */}
+```python file=/concepts/partitions_schedules_sensors/partitioned_job_test.py startafter=start endbefore=end
+def test_partitioned_op_job():
+ assert partitioned_op_job.execute_in_process(partition_key="2020-01-01").success
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/unit-testing-assets-and-ops.md b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/unit-testing-assets-and-ops.md
new file mode 100644
index 0000000000000..855705ddd2510
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/guides/test/unit-testing-assets-and-ops.md
@@ -0,0 +1,66 @@
+---
+title: "Unit testing assets and ops"
+sidebar_position: 400
+---
+
+Unit testing is essential for ensuring that computations function as intended. In the context of data pipelines, this can be particularly challenging. However, Dagster streamlines the process by enabling direct invocation of computations with specified input values and mocked resources, making it easier to verify that data transformations behave as expected.
+
+While unit tests can't fully replace integration tests or manual review, they can catch a variety of errors with a significantly faster feedback loop.
+
+This article covers how to write unit tests for assets with a variety of different input requirements.
+
+:::note
+
+Before you begin implementing unit tests, note that:
+
+- Testing individual assets is generally recommended over unit testing entire jobs.
+- Unit testing isn't recommended in cases where most of the business logic is encoded in an external system, such as an asset which directly invokes an external Databricks job.
+- If you want to test your assets at runtime, you can use [asset checks](asset-checks) to verify the quality of data produced by your pipelines, communicate what the data is expected to do, and more.
+
+:::
+
+## Unit test examples
+
+### Assets and ops without arguments \{#no-arguments}
+
+The simplest assets to test are those with no arguments. In these cases, you can directly invoke definitions.
+
+
+
+### Assets with upstream dependencies \{#upstream-dependencies}
+
+If an asset has an upstream dependency, you can directly pass a value for that dependency when invoking the definition.
+
+
+
+### Assets with config \{#config}
+
+If an asset uses config, you can construct an instance of the required config object and pass it in directly.
+
+
+
+### Assets with resources \{#resources}
+
+If an asset uses a resource, it can be useful to create a mock instance of the resource to avoid interacting with external services.
+
+
+
+### Assets with context \{#context}
+
+If an asset uses a `context` argument, you can use `build_asset_context()` to construct a context object.
+
+
+
+### Assets with multiple parameters \{#multiple-parameters}
+
+If an asset has multiple parameters, we recommended using keyword arguments for clarity.
+
+
+
+## Running the tests
+
+Use `pytest` or your test runner of choice to run your unit tests. Navigate to the top-level project directory (the one that contains the tests directory) and run:
+
+```
+pytest my_project_tests
+```
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/guides/integration-approaches.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/guides/integration-approaches.md
new file mode 100644
index 0000000000000..ca731f7cb6fdd
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/guides/integration-approaches.md
@@ -0,0 +1,127 @@
+---
+title: "Approaches to writing a Dagster integration"
+sidebar_position: 100
+unlisted: true
+---
+
+There are many approaches to writing an integration in Dagster. The choice of approach depends on the specific requirements of the integration, the level of control needed, and the complexity of the external system being integrated. The following are typical approaches that align with Dagster's best practices.
+
+- [Resource providers](#resource-providers)
+- [Factory methods](#factory-methods)
+- [Multi-asset decorators](#multi-asset-decorators)
+- [Pipes protocol](#pipes-protocol)
+
+## Resource providers
+
+One of the most fundamental features that can be implemented in an integration is a resource object to interface with an external service. For example, the `dagster-snowflake` integration provides a custom [SnowflakeResource](https://github.com/dagster-io/dagster/blob/master/python_modules/libraries/dagster-snowflake/dagster_snowflake/resources.py) that is a wrapper around the Snowflake `connector` object.
+
+### Pros
+
+- **Simple** Implementing a resource wrapper is often the first step in iterating on a fully-featured integration.
+- **Reusable** Resources are a core building block in the Dagster ecosystem, and allow one to re-use code across assets.
+
+### Cons
+
+- **Low-level abstraction** While the resource can be re-used throughout the codebase, it does not provide any higher level abstraction to assets or jobs.
+
+### Guide
+
+:::note
+
+A guide for writing a resource based integration is coming soon!
+
+:::
+
+## Factory methods
+
+The factory pattern is used for creating multiple similar objects based on a set of specifications. This is often useful in the data engineering when you have similar processing that will operate on multiple objects with varying parameters.
+
+For example, imagine you would like to perform an operation on a set of tables in a database. You could construct a factory method that takes in a table specification, resulting in a list of assets.
+
+```python
+from dagster import Definitions, asset
+
+parameters = [
+ {"name": "asset1", "table": "users"},
+ {"name": "asset2", "table": "orders"},
+]
+
+
+def process_table(table_name: str) -> None:
+ pass
+
+
+def build_asset(params):
+ @asset(name=params["name"])
+ def _asset():
+ process_table(params["table"])
+
+ return _asset
+
+
+assets = [build_asset(params) for params in parameters]
+
+defs = Definitions(assets=assets)
+```
+
+### Pros
+
+- **Flexibility:** Allows for fine-grained control over the integration logic.
+- **Modularity:** Easy to reuse components across different assets and jobs.
+- **Explicit configuration:** Resources can be explicitly configured, making it clear what dependencies are required.
+
+### Cons
+
+- **Complexity:** Can be more complex to set up compared to other methods.
+- **Boilerplate code:** May require more boilerplate code to define assets, resources, and jobs.
+
+### Guide
+
+:::note
+
+A guide for writing a factory method based integrations is coming soon!
+
+:::
+
+## Multi-asset decorators
+
+In the scenario where a single API call or configuration can result in multiple assets, with a shared runtime or dependencies, one may consider creating a multi-asset decorator. Example implementations of this approach include [dbt](https://github.com/dagster-io/dagster/tree/master/python_modules/libraries/dagster-dbt), [dlt](https://github.com/dagster-io/dagster/tree/master/python_modules/libraries/dagster-dlt), and [Sling](https://github.com/dagster-io/dagster/tree/master/python_modules/libraries/sling).
+
+### Pros
+
+- **Efficiency:** Allows defining multiple assets in a single function, reducing boilerplate code.
+- **Simplicity:** Easier to manage related assets together.
+- **Consistency:** Matches the the developer experience of the Dagster ecosystem by using decorator-based assets.
+
+### Cons
+
+- **Less granular control:** May not provide as much fine-grained control as defining individual assets.
+- **Complexity in debugging:** Debugging issues can be more challenging when multiple assets are defined in a single function.
+
+### Guide
+
+- [Writing a multi-asset decorator integration](multi-asset-integration)
+
+## Pipes protocol
+
+The Pipes protocol is used to integrate with systems that have their own execution environments. It enables running code in these external environments while allowing Dagster to maintain control and visibility.
+
+Example implementations of this approach include:
+- [AWS Lambda](https://github.com/dagster-io/dagster/tree/master/python_modules/libraries/dagster-aws/dagster_aws/pipes)
+- [Databricks](https://github.com/dagster-io/dagster/blob/master/python_modules/libraries/dagster-databricks/dagster_databricks/pipes.py)
+- [Kubernetes](https://github.com/dagster-io/dagster/blob/master/python_modules/libraries/dagster-k8s/dagster_k8s/pipes.py).
+
+### Pros
+
+- **Separation of Environments:** Allows running code in external environments, which can be useful for integrating with systems that have their own execution environments.
+- **Flexibility:** Can integrate with a wide range of external systems and languages.
+- **Streaming logs and metadata:** Provides support for streaming logs and structured metadata back into Dagster.
+
+### Cons
+
+- **Complexity:** Can be complex to set up and configure.
+- **Overhead:** May introduce additional overhead for managing external environments.
+
+### Guide
+
+- [Dagster Pipes details and customization](/guides/build/external-pipelines/dagster-pipes-details-and-customization)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/guides/multi-asset-integration.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/guides/multi-asset-integration.md
new file mode 100644
index 0000000000000..927d6d8278715
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/guides/multi-asset-integration.md
@@ -0,0 +1,270 @@
+---
+title: Creating a multi-asset integration
+description: Create a decorator based multi-asset integration
+sidebar_position: 200
+---
+
+When working in the Dagster ecosystem, you may have noticed that decorators are frequently used. For example, assets, jobs, and ops use decorators. If you have a service that produces many assets, it's possible to define it as a multi-asset decorator-offering a consistent and intuitive developer experience to existing Dagster APIs.
+
+In the context of Dagster, decorators are helpful because they often wrap some form of processing. For example, when writing an asset, you define your processing code and then annotate the function with the `asset` decorator /> decorator. Then, the internal Dagster code can register the asset, assign metadata, pass in context data, or perform any other variety of operations that are required to integrate your asset code with the Dagster platform.
+
+In this guide, you'll learn how to develop a multi-asset integration for a hypothetical replication tool.
+
+:::note
+This guide assumes basic familiarity with Dagster and Python decorators.
+:::
+
+## Step 1: Input
+
+For this guide, let's imagine a tool that replicates data between two databases. It's configured using a `replication.yaml` configuration file, in which a user is able to define source and destination databases, along with the tables that they would like to replicate between these systems.
+
+```yml
+connections:
+ source:
+ type: duckdb
+ connection: example.duckdb
+ destination:
+ type: postgres
+ connection: postgresql://postgres:postgres@localhost/postgres
+
+tables:
+ - name: users
+ primary_key: id
+ - name: products
+ primary_key: id
+ - name: activity
+ primary_key: id
+```
+
+For the integration we're building, we want to provide a multi-asset that encompasses this replication process, and generates an asset for each table being replicated.
+
+We will define a dummy function named `replicate` that will mock the replication process, and return a dictionary with the replication status of each table. In the real world, this could be a function in a library, or a call to a command-line tool.
+
+```python
+import yaml
+
+from pathlib import Path
+from typing import Mapping, Iterator, Any
+
+
+def replicate(replication_configuration_yaml: Path) -> Iterator[Mapping[str, Any]]:
+ data = yaml.safe_load(replication_configuration_yaml.read_text())
+ for table in data.get("tables"):
+ # < perform replication here, and get status >
+ yield {"table": table.get("name"), "status": "success"}
+```
+
+## Step 2: Implementation
+
+First, let's define a `Project` object that takes in the path of our configuration YAML file. This will allow us to encapsulate the logic that gets metadata and table information from our project configuration.
+
+```python
+import yaml
+from pathlib import Path
+
+
+class ReplicationProject():
+ def __init__(self, replication_configuration_yaml: str):
+ self.replication_configuration_yaml = replication_configuration_yaml
+
+ def load(self):
+ return yaml.safe_load(Path(self.replication_configuration_yaml).read_text())
+```
+
+Next, define a function that returns a `multi_asset` function. The `multi_asset` function is a decorator itself, so this allows us to customize the behavior of `multi_asset` and create a new decorator of our own:
+
+```python
+def custom_replication_assets(
+ *,
+ replication_project: ReplicationProject,
+ name: Optional[str] = None,
+ group_name: Optional[str] = None,
+) -> Callable[[Callable[..., Any]], AssetsDefinition]:
+ project = replication_project.load()
+
+ return multi_asset(
+ name=name,
+ group_name=group_name,
+ specs=[
+ AssetSpec(
+ key=table.get("name"),
+ )
+ for table in project.get("tables")
+ ],
+ )
+```
+
+Let's review what this code does:
+
+- Defines a function that returns a `multi_asset` function
+- Loads our replication project and iterates over the tables defined in the input YAML file
+- Uses the tables to create a list of `AssetSpec` objects and passes them to the `specs` parameter, thus defining assets that will be visible in the Dagster UI
+
+Next, we'll show you how to perform the execution of the replication function.
+
+Recall that decorators allow us to wrap a function that performs some operation. In the case of our `multi_asset`, we defined `AssetSpec` objects for our tables, and the actual processing that takes place will be in the body of the decorated function.
+
+In this function, we will perform the replication, and then yield `AssetMaterialization` objects indicating that the replication was successful for a given table.
+
+```python
+from dagster import AssetExecutionContext
+
+
+replication_project_path = "replication.yaml"
+replication_project = ReplicationProject(replication_project_path)
+
+
+@custom_replication_assets(
+ replication_project=replication_project,
+ name="my_custom_replication_assets",
+ group_name="replication",
+)
+def my_assets(context: AssetExecutionContext):
+ results = replicate(Path(replication_project_path))
+ for table in results:
+ if table.get("status") == "SUCCESS":
+ yield AssetMaterialization(asset_key=str(table.get("name")), metadata=table)
+```
+
+There are a few limitations to this approach:
+
+- **We have not encapsulated the logic for replicating tables.** This means that users who use the `custom_replication_assets` decorator would be responsible for yielding asset materializations themselves.
+- **Users can't customize the attributes of the asset**.
+
+For the first limitation, we can resolve this by refactoring the code in the body of our asset function into a Dagster resource.
+
+## Step 3: Moving the replication logic into a resource
+
+Refactoring the replication logic into a resource enables us to support better configuration and re-use of our logic.
+
+To accomplish this, we will extend the `ConfigurableResource` object to create a custom resource. Then, we will define a `run` method that will perform the replication operation:
+
+```python
+from dagster import ConfigurableResource
+from dagster._annotations import public
+
+
+class ReplicationResource(ConfigurableResource):
+ @public
+ def run(
+ self, replication_project: ReplicationProject
+ ) -> Iterator[AssetMaterialization]:
+ results = replicate(Path(replication_project.replication_configuration_yaml))
+ for table in results:
+ if table.get("status") == "SUCCESS":
+ # NOTE: this assumes that the table name is the same as the asset key
+ yield AssetMaterialization(
+ asset_key=str(table.get("name")), metadata=table
+ )
+```
+
+Now, we can refactor our `custom_replication_assets` instance to use this resource:
+
+```python
+@custom_replication_assets(
+ replication_project=replication_project,
+ name="my_custom_replication_assets",
+ group_name="replication",
+)
+def my_assets(replication_resource: ReplicationProject):
+ replication_resource.run(replication_project)
+```
+
+## Step 4: Using translators
+
+At the end of [Step 2](#step-2-implementation), we mentioned that end users were unable to customize asset attributes, like the asset key, generated by our decorator. Translator classes are the recommended way of defining this logic, and they provide users with the option to override the default methods used to convert a concept from your tool (for example, a table name) to the corresponding concept in Dagster (for example, asset key).
+
+To start, we will define a translator method to map the table specification to a Dagster asset key.
+
+:::note
+In a real world integration, you will want to define methods for all common attributes like dependencies, group names, and metadata.
+:::
+
+```python
+from dagster import AssetKey, _check as check
+
+from dataclasses import dataclass
+
+
+@dataclass
+class ReplicationTranslator:
+ @public
+ def get_asset_key(self, table_definition: Mapping[str, str]) -> AssetKey:
+ return AssetKey(str(table_definition.get("name")))
+```
+
+Next, we'll update `custom_replication_assets` to use the translator when defining the `key` on the `AssetSpec`.
+
+:::note
+Note that we took this opportunity to also include the replication project and translator instance on the `AssetSpec` metadata. This is a workaround that we tend to employ in this approach, as it makes it possible to define these objects once and then access them on the context of our asset.
+:::
+
+```python
+def custom_replication_assets(
+ *,
+ replication_project: ReplicationProject,
+ name: Optional[str] = None,
+ group_name: Optional[str] = None,
+ translator: Optional[ReplicationTranslator] = None,
+) -> Callable[[Callable[..., Any]], AssetsDefinition]:
+ project = replication_project.load()
+
+ translator = (
+ check.opt_inst_param(translator, "translator", ReplicationTranslator)
+ or ReplicationTranslator()
+ )
+
+ return multi_asset(
+ name=name,
+ group_name=group_name,
+ specs=[
+ AssetSpec(
+ key=translator.get_asset_key(table),
+ metadata={
+ "replication_project": project,
+ "replication_translator": translator,
+ },
+ )
+ for table in project.get("tables")
+ ],
+ )
+```
+
+Finally, we have to update our resource to use the translator and project provided in the metadata. We are using the `check` method provided by `dagster._check` to ensure that the type of the object is appropriate as we retrieve it from the metadata.
+
+Now, we can use the same `translator.get_asset_key` when yielding the asset materialization, thus ensuring that our asset declarations match our asset materializations:
+
+```python
+class ReplicationResource(ConfigurableResource):
+ @public
+ def run(self, context: AssetExecutionContext) -> Iterator[AssetMaterialization]:
+ metadata_by_key = context.assets_def.metadata_by_key
+ first_asset_metadata = next(iter(metadata_by_key.values()))
+
+ project = check.inst(
+ first_asset_metadata.get("replication_project"),
+ ReplicationProject,
+ )
+
+ translator = check.inst(
+ first_asset_metadata.get("replication_translator"),
+ ReplicationTranslator,
+ )
+
+ results = replicate(Path(project.replication_configuration_yaml))
+ for table in results:
+ if table.get("status") == "SUCCESS":
+ yield AssetMaterialization(
+ asset_key=translator.get_asset_key(table), metadata=table
+ )
+```
+
+## Conclusion
+
+In this guide we walked through how to define a custom multi-asset decorator, a resource for encapsulating tool logic, and a translator for defining the logic to translate a specification to Dagster concepts.
+
+Defining integrations with this approach aligns nicely with the overall development paradigm of Dagster, and is suitable for tools that generate many assets.
+
+The code in its entirety can be seen below:
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airbyte/airbyte-cloud.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airbyte/airbyte-cloud.md
new file mode 100644
index 0000000000000..87602fd6f43cf
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airbyte/airbyte-cloud.md
@@ -0,0 +1,85 @@
+---
+layout: Integration
+status: published
+name: Airbyte Cloud
+title: Using Dagster with Airbyte Cloud
+sidebar_label: Airbyte Cloud
+excerpt: Orchestrate Airbyte Cloud connections and schedule syncs alongside upstream or downstream dependencies.
+date: 2022-11-07
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-airbyte
+docslink: https://docs.dagster.io/integrations/libraries/airbyte/airbyte-cloud
+partnerlink: https://airbyte.com/tutorials/orchestrate-data-ingestion-and-transformation-pipelines
+logo: /integrations/airbyte.svg
+categories:
+ - ETL
+enabledBy:
+enables:
+tags: [dagster-supported, etl]
+sidebar_custom_props:
+ logo: images/integrations/airbyte.svg
+---
+
+This guide provides instructions for using Dagster with Airbyte Cloud using the `dagster-airbyte` library. Your Airbyte Cloud connection tables can be represented as assets in the Dagster asset graph, allowing you to track lineage and dependencies between Airbyte Cloud assets and data assets you are already modeling in Dagster. You can also use Dagster to orchestrate Airbyte Cloud connections, allowing you to trigger syncs for these on a cadence or based on upstream data changes.
+
+## What you'll learn
+
+- How to represent Airbyte Cloud assets in the Dagster asset graph, including lineage to other Dagster assets.
+- How to customize asset definition metadata for these Airbyte Cloud assets.
+- How to materialize Airbyte Cloud connection tables from Dagster.
+- How to customize how Airbyte Cloud connection tables are materialized.
+
+
+ Prerequisites
+
+- The `dagster` and `dagster-airbyte` libraries installed in your environment
+- Familiarity with asset definitions and the Dagster asset graph
+- Familiarity with Dagster resources
+- Familiarity with Airbyte Cloud concepts, like connections and connection tables
+- An Airbyte Cloud workspace
+- An Airbyte Cloud client ID and client secret. For more information, see [Configuring API Access](https://docs.airbyte.com/using-airbyte/configuring-api-access) in the Airbyte Cloud REST API documentation.
+
+
+
+## Set up your environment
+
+To get started, you'll need to install the `dagster` and `dagster-airbyte` Python packages:
+
+```bash
+pip install dagster dagster-airbyte
+```
+
+## Represent Airbyte Cloud assets in the asset graph
+
+To load Airbyte Cloud assets into the Dagster asset graph, you must first construct a resource, which allows Dagster to communicate with your Airbyte Cloud workspace. You'll need to supply your workspace ID, client ID and client secret. See [Configuring API Access](https://docs.airbyte.com/using-airbyte/configuring-api-access) in the Airbyte Cloud REST API documentation for more information on how to create your client ID and client secret.
+
+Dagster can automatically load all connection tables from your Airbyte Cloud workspace as asset specs. Call the function, which returns list of s representing your Airbyte Cloud assets. You can then include these asset specs in your object:
+
+
+
+### Sync and materialize Airbyte Cloud assets
+
+You can use Dagster to sync Airbyte Cloud connections and materialize Airbyte Cloud connection tables. You can use the factory to create all assets definitions for your Airbyte Cloud workspace.
+
+
+
+### Customize the materialization of Airbyte Cloud assets
+
+If you want to customize the sync of your connections, you can use the decorator to do so. This allows you to execute custom code before and after the call to the Airbyte Cloud sync.
+
+
+
+### Customize asset definition metadata for Airbyte Cloud assets
+
+By default, Dagster will generate asset specs for each Airbyte Cloud asset and populate default metadata. You can further customize asset properties by passing an instance of the custom to the function.
+
+
+
+Note that `super()` is called in each of the overridden methods to generate the default asset spec. It is best practice to generate the default asset spec before customizing it.
+
+You can pass an instance of the custom to the decorator or the factory.
+
+### Load Airbyte Cloud assets from multiple workspaces
+
+Definitions from multiple Airbyte Cloud workspaces can be combined by instantiating multiple resources and merging their specs. This lets you view all your Airbyte Cloud assets in a single asset graph:
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airbyte/airbyte-oss.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airbyte/airbyte-oss.md
new file mode 100644
index 0000000000000..54c1acb1de952
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airbyte/airbyte-oss.md
@@ -0,0 +1,35 @@
+---
+layout: Integration
+status: published
+name: Airbyte
+title: Dagster & Airbyte
+sidebar_label: Airbyte
+excerpt: Orchestrate Airbyte connections and schedule syncs alongside upstream or downstream dependencies.
+date: 2022-11-07
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-airbyte
+docslink: https://docs.dagster.io/integrations/libraries/airbyte/airbyte-oss
+partnerlink: https://airbyte.com/tutorials/orchestrate-data-ingestion-and-transformation-pipelines
+categories:
+ - ETL
+enabledBy:
+enables:
+tags: [dagster-supported, etl]
+sidebar_custom_props:
+ logo: images/integrations/airbyte.svg
+---
+
+Using this integration, you can trigger Airbyte syncs and orchestrate your Airbyte connections from within Dagster, making it easy to chain an Airbyte sync with upstream or downstream steps in your workflow.
+
+### Installation
+
+```bash
+pip install dagster-airbyte
+```
+
+### Example
+
+
+
+### About Airbyte
+
+**Airbyte** is an open source data integration engine that helps you consolidate your SaaS application and database data into your data warehouses, lakes and databases.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airbyte/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airbyte/index.md
new file mode 100644
index 0000000000000..66b6d2a9de5a3
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airbyte/index.md
@@ -0,0 +1,23 @@
+---
+layout: Integration
+status: published
+name: Airbyte
+title: Dagster & Airbyte
+sidebar_label: Airbyte
+excerpt: Orchestrate Airbyte connections and schedule syncs alongside upstream or downstream dependencies.
+date: 2022-11-07
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-airbyte
+docslink: https://docs.dagster.io/integrations/libraries/airbyte/
+partnerlink: https://airbyte.com/tutorials/orchestrate-data-ingestion-and-transformation-pipelines
+enabledBy:
+enables:
+tags: [dagster-supported, etl]
+sidebar_custom_props:
+ logo: images/integrations/airbyte.svg
+ categories:
+ - etl
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/decomission.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/decomission.md
new file mode 100644
index 0000000000000..fca1886648cdc
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/decomission.md
@@ -0,0 +1,12 @@
+---
+title: "Decommission the Airflow DAG"
+sidebar_position: 500
+---
+
+Previously, we completed migration of our Airflow DAG to Dagster assets. If you haven't finished that stage yet, please follow along [here](migrate).
+
+Once we are confident in our migrated versions of the tasks, we can decommission the Airflow DAG. First, we can remove the DAG from our Airflow DAG directory.
+
+Next, we can strip the task associations from our Dagster definitions. This can be done by removing the `assets_with_task_mappings` call. We can use this opportunity to attach our assets to a `ScheduleDefinition` so that Dagster's scheduler can manage their execution:
+
+
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/index.md
new file mode 100644
index 0000000000000..2a46bf331f92a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/index.md
@@ -0,0 +1,28 @@
+---
+title: 'Migrating from Airflow to Dagster'
+sidebar_position: 10
+---
+
+This tutorial demonstrates using [`dagster-airlift`](/api/python-api/libraries/dagster-airlift) to migrate an Airflow DAG to Dagster.
+
+Using `dagster-airlift` we can:
+
+- Observe Airflow DAGs and their execution history with no changes to Airflow code
+- Model and observe assets orchestrated by Airflow with no changes to Airflow code
+- Enable a migration process that
+ - Can be done task-by-task in any order with minimal coordination
+ - Has task-by-task rollback to reduce risk
+ - That retains Airflow DAG structure and execution history during the migration
+
+## Process
+
+This is a high level overview of the steps to migrate an Airflow DAG to Dagster:
+
+- **Peer**
+ - Observe an Airflow instance from within a Dagster Deployment via the Airflow REST API.
+ - This loads every Airflow DAG as an asset definition and creates a sensor that polls Airflow for execution history.
+- **Observe**
+ - Add a mapping that maps the Airflow DAG and task id to a basket of definitions that you want to observe. (e.g. render the full lineage the dbt models an Airflow task orchestrates)
+ - The sensor used for peering also polls for task execution history, and adds materializations to an observed asset when its corresponding task successfully executes
+- **Migrate**
+ - Selectively move execution of Airflow tasks to Dagster Software Defined Assets
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/migrate-a-full-dag.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/migrate-a-full-dag.md
new file mode 100644
index 0000000000000..eea7cde116ada
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/migrate-a-full-dag.md
@@ -0,0 +1,49 @@
+---
+title: "Migrate a full DAG"
+sidebar_position: 600
+---
+
+There may be DAGs for which you want to migrate the entire DAG at once rather than on a per-task basis. Some reasons for taking this approach:
+
+- You're making use of "dynamic tasks" in Airflow, which don't conform neatly to the task mapping protocol we've laid out above.
+- You want to make more substantial refactors to the DAG structure that don't conform to the existing task structure
+
+For cases like this, we allow you to map assets to a full DAG.
+
+## Setup
+
+This guide assumes you've only completed the [setup](setup) and [peer](peer) steps of the Airflow migration tutorial. If you've already completed the full migration tutorial, we advise downloading a fresh copy and following along with those steps. This guide will perform the observe and migrate steps at the DAG-level instead of on a task-by-task basis, for the `rebuild_customers_list` DAG.
+
+## Observing DAG-mapped
+
+When migrating an entire DAG at once, we'll want to create assets which map to the entire DAG. Whereas in the [task-by-task observation step](setup), we used the `assets_with_task_mappings` function, we'll instead use the `assets_with_dag_mappings` function.
+
+For our `rebuild_customers_list` DAG, let's take a look at what the new observation code looks like:
+
+
+
+Now, instead of getting a materialization when a particular task completes, each mapped asset will receive a materialization when the entire DAG completes.
+
+## Migrating DAG-mapped assets
+
+Recall that in the [task-by-task migration step](migrate), we "proxy" execution on a task by task basis, which is controlled by a yaml document. For DAG-mapped assets, execution is proxied on a per-DAG basis. Proxying execution to Dagster will require all assets mapped to that DAG be _executable_ within Dagster. Let's take a look at some fully migrated code mapped to DAGs instead of tasks:
+
+
+
+Now that all of our assets are fully executable, we can create a simple yaml file to proxy execution for the whole dag:
+
+
+
+We will similarly use `proxying_to_dagster` at the end of our DAG file (the code is exactly the same here as it was for the per-task migration step)
+
+
+
+Once the `proxied` bit is flipped to True, we can go to the Airflow UI, and we'll see that our tasks have been replaced with a single task.
+
+![Before DAG proxying](/images/integrations/airlift/before_dag_override.png)
+
+![After DAG proxying](/images/integrations/airlift/after_dag_override.png)
+
+When performing dag-level mapping, we don't preserve task structure in the Airflow dags. This single task will materialize all mapped Dagster assets instead of executing the original Airflow task business logic.
+
+We can similarly mark `proxied` back to `False`, and the original task structure and business logic will return unchanged.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/migrate.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/migrate.md
new file mode 100644
index 0000000000000..ab6bace324598
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/migrate.md
@@ -0,0 +1,69 @@
+---
+title: "Migrate assets"
+sidebar_position: 400
+---
+
+Previously, we completed the ["Observe" stage](observe) of the Airflow migration process by encoding the assets that are produced by each task. We also introduced partitioning to those assets.
+
+Up until now, we have not touched the Airflow code base at all. Now it's time to begin the actual migration process.
+
+Once you have created corresponding definitions in Dagster to your Airflow tasks, you can proxy execution to Dagster on a per-task basis while Airflow is still controlling scheduling and orchestration. Once a task has been proxied, Airflow will kick off materializations of corresponding Dagster assets in place of executing the business logic of that task.
+
+To begin proxying tasks in a DAG, first you will need a file to track proxying state. In your Airflow DAG directory, create a `proxied_state` folder, and in it create a yaml file with the same name as your DAG. The included example at `airflow_dags/proxied_state` is used by `make airflow_run`, and can be used as a template for your own proxied state files.
+
+Given our example DAG `rebuild_customers_list` with three tasks, `load_raw_customers`, `run_dbt_model`, and `export_customers`, `proxied_state/rebuild_customers_list.yaml` should look like the following:
+
+
+
+Next, you will need to modify your Airflow DAG to make it aware of the proxied state. This is already done in the example DAG:
+
+
+
+Set `PROXYING` to `True` or eliminate the `if` statement.
+
+The DAG will now display its proxied state in the Airflow UI. (There is some latency as Airflow evaluates the Python file periodically.)
+
+![Migration state rendering in Airflow UI](/images/integrations/airlift/state_in_airflow.png)
+
+### Migrating individual tasks
+
+In order to proxy a task, you must do two things:
+
+1. First, ensure all associated assets are executable in Dagster by providing asset definitions in place of bare asset specs.
+2. The `proxied: False` status in the `proxied_state` YAML folder must be adjusted to `proxied: True`.
+
+Any task marked as proxied will use the `DefaultProxyTaskToDagsterOperator` when executed as part of the DAG. This operator will use the Dagster GraphQL API to initiate a Dagster run of the assets corresponding to the task.
+
+The proxied file acts as the source of truth for proxied state. The information is attached to the DAG and then accessed by Dagster via the REST API.
+
+A task which has been proxied can be easily toggled back to run in Airflow (for example, if a bug in implementation was encountered) simply by editing the file to `proxied: False`.
+
+#### Migrating common operators
+
+For some common operator patterns, like our dbt operator, Dagster supplies factories to build software defined assets for our tasks. In fact, the `@dbt_assets` decorator used earlier already backs its assets with definitions, so we can toggle the proxied state of the `build_dbt_models` task to `proxied: True` in the proxied state file:
+
+
+
+**Important**: It may take up to 30 seconds for the proxied state in the Airflow UI to reflect this change. You must subsequently reload the definitions in Dagster via the UI or by restarting `dagster dev`.
+
+You can now run the `rebuild_customers_list` DAG in Airflow, and the `build_dbt_models` task will be executed in a Dagster run:
+
+![dbt build executing in Dagster](/images/integrations/airlift/proxied_dag.png)
+
+You'll note that we proxied a task in the _middle_ of the Airflow DAG. The Airflow DAG structure and execution history is stable in the Airflow UI, but execution of `build_dbt_models` has moved to Dagster.
+
+#### Migrating the remaining custom operators
+
+For all other operator types, we will need to build our own asset definitions. We recommend creating a factory function whose arguments match the inputs to your Airflow operator. Then, you can use this factory to build definitions for each Airflow task.
+
+For example, our `load_raw_customers` task uses a custom `LoadCSVToDuckDB` operator. We'll define a function `load_csv_to_duckdb_defs` factory to build corresponding software-defined assets. Similarly for `export_customers` we'll define a function `export_duckdb_to_csv_defs` to build SDAs:
+
+
+
+We can then toggle the proxied state of the remaining tasks in the `proxied_state` file:
+
+
+
+## Next steps
+
+Now that we've completed migration of the Airflow DAG, we can decomission it. Follow along [here](decomission).
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/observe.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/observe.md
new file mode 100644
index 0000000000000..435d3c7df03b2
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/observe.md
@@ -0,0 +1,86 @@
+---
+title: "Observe assets"
+sidebar_position: 300
+---
+
+Previously, we completed the ["Peering" stage](peer) of the Airflow migration process by peering the Airflow instance with a Dagster code location.
+
+The next step is to represent our Airflow workflows more richly by observing the data assets that are produced by our tasks. Similar to the peering step, this stage does not require _any changes_ to Airflow code.
+
+In order to do this, we must define the relevant assets in the Dagster code location.
+
+In our example, we have three sequential tasks:
+
+1. `load_raw_customers` loads a CSV file of raw customer data into duckdb.
+2. `run_dbt_model` builds a series of dbt models (from [jaffle shop](https://github.com/dbt-labs/jaffle_shop_duckdb)) combining customer, order, and payment data.
+3. `export_customers` exports a CSV representation of the final customer file from duckdb to disk.
+
+We will first create a set of asset specs that correspond to the assets produced by these tasks. We will then annotate these asset specs so that Dagster can associate them with the Airflow tasks that produce them.
+
+The first and third tasks involve a single table each. We can manually construct specs for these two tasks. Dagster provides the `assets_with_task_mappings` utility to annotate our asset specs with the tasks that produce them. Assets which are properly annotated will be materialized by the Airlift sensor once the corresponding task completes: These annotated specs are then provided to the `defs` argument to `build_defs_from_airflow_instance`.
+
+We will also create a set of dbt asset definitions for the `build_dbt_models` task. We can use the `dagster-dbt`-supplied decorator `@dbt_assets` to generate these definitions using Dagster's dbt integration.
+
+First, you need to install the extra that has the dbt factory:
+
+```bash
+uv pip install 'dagster-airlift[dbt]'
+```
+
+Then, we will construct our assets:
+
+
+
+### Viewing observed assets
+
+Once your assets are set up, you should be able to reload your Dagster definitions and see a full representation of the dbt project and other data assets in your code.
+
+
+
+Kicking off a run of the DAG in Airflow, you should see the newly created assets materialize in Dagster as each task completes.
+
+_Note: There will be some delay between task completion and assets materializing in Dagster, managed by the sensor. This sensor runs every 30 seconds by default (you can reduce down to one second via the `minimum_interval_seconds` argument to `sensor`)._
+
+### Moving the asset check
+
+Now that we've introduced an asset explicitly for the `customers.csv` file output by the DAG, we should move the asset check constructed during the Peering step to instead be on the `customers_csv` asset. Simply change the `asset` targeted by the `@asset_check` decorator to be `AssetKey(["customers_csv"])`. Doing this ensures that even when we delete the DAG, the asset check will live on.
+
+When done, our code will look like this.
+
+
+
+### Adding partitions
+
+If your Airflow tasks produce time-partitioned assets, Airlift can automatically associate your materializations to the relevant partitions. In the case of `rebuild_customers_list`, data is daily partitioned in each created table, and and the Airflow DAG runs on a `@daily` cron schedule. We can likewise add a `DailyPartitionsDefinition` to each of our assets.
+
+
+
+Now, every time the sensor synthesizes a materialization for an asset, it will automatically have a partition associated with it.
+
+Let's try this out by kicking off an airflow backfill for today:
+
+```bash
+airflow dags backfill rebuild_customers_list --start-date $(date +"%Y-%m-%d")
+```
+
+After this dag run completes, you should see a partitioned materialization appear in Dagster.
+
+![Partitioned materialization in Dagster](/images/integrations/airlift/partitioned_mat.png)
+
+Let's clear our Airflow runs so that we can kick off this backfill again for testing in the future.
+
+```bash
+airflow db clean
+```
+
+In order for partitioned assets to work out of the box with `dagster-airlift`, the following things need to be true:
+
+- The asset can only be time-window partitioned. This means static, dynamic, and multi partitioned definitions will require custom functionality.
+- The partitioning scheme must match up with the [logical_date / execution_date](https://airflow.apache.org/docs/apache-airflow/stable/faq.html#what-does-execution-date-mean) of corresponding Airflow runs. That is, each logical_date should correspond _exactly_ to a partition in Dagster.
+
+## Next steps
+
+Next, it's time to begin migrating our Airflow DAG code to Dagster. Follow along with the Migrate step [here](migrate).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/peer.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/peer.md
new file mode 100644
index 0000000000000..952377a021d04
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/peer.md
@@ -0,0 +1,94 @@
+---
+title: "Peer your Airflow instance with a Dagster code location"
+sidebar_position: 200
+---
+
+At this point, we should have finished the [setup](setup) step, and now we have the example code setup with a fresh virtual environment, and Airflow running locally. Now, we can start writing Dagster code.
+
+We call the first stage of migration from Airflow to Dagster the "Peering" stage, at which we will "peer" the Airflow instance with a Dagster code location, which will create an asset representation of each Airflow DAG that you can view in Dagster. This process does not require any changes to your Airflow instance.
+
+First, you will want a new shell and navigate to the same directory. You will need to set up the `dagster-airlift` package in your Dagster environment:
+
+```bash
+source .venv/bin/activate
+uv pip install 'dagster-airlift[core]' dagster-webserver dagster
+```
+
+Next, create a `Definitions` object using `build_defs_from_airflow_instance`. You can use the empty `tutorial_example/dagster_defs/definitions.py` file as a starting point:
+
+
+
+This function creates:
+
+- An external asset representing each DAG. This asset is marked as materialized whenever a DAG run completes.
+- A sensor that polls the Airflow instance for operational information. This sensor is responsible for creating materializations when a DAG executes. The sensor must remain on in order to properly update execution status.
+
+Let's set up some environment variables, and then point Dagster to see the asset created from our Airflow DAG:
+
+```bash
+# Set up environment variables to point to the airlift-migration-tutorial directory on your machine
+export TUTORIAL_EXAMPLE_DIR=$(pwd)
+export TUTORIAL_DBT_PROJECT_DIR="$TUTORIAL_EXAMPLE_DIR/tutorial_example/shared/dbt"
+export AIRFLOW_HOME="$TUTORIAL_EXAMPLE_DIR/.airflow_home"
+dagster dev -f tutorial_example/dagster_defs/definitions.py
+```
+
+
+
+Let's kick off a run of the `reubild_customers_list` DAG in Airflow.
+
+```bash
+airflow dags backfill rebuild_customers_list --start-date $(shell date +"%Y-%m-%d")
+```
+
+When this run has completed in Airflow, we should be able to navigate to the Dagster UI, and see that the Dagster has registered a materialization corresponding to that successful run.
+
+
+
+Run the following command to clean the Airflow and Dagster run history (we just do this so we can run the same example backfill in the future). Under the hood, this just deletes runs from Airflow and asset materializations from Dagster.
+
+```bash
+make clean
+```
+
+_Note: When the code location loads, Dagster will query the Airflow REST API in order to build a representation of your DAGs. In order for Dagster to reflect changes to your DAGs, you will need to reload your code location._
+
+## Asset checks as User Acceptance Tests
+
+Once you have peered your Airflow DAGs in Dagster, regardless of migration progress, you can begin to add asset checks to your Dagster code. In Dagster, Asset checks can be used to validate the quality of your data assets, and can provide additional observability and value on top of your Airflow DAG even before migration starts.
+
+Asset checks can both act as useful _user acceptance tests_ to ensure that any migration steps taken are successful, as well as _outlive_ the migration itself.
+
+For example, we're going to add an asset check to ensure that the final `customers` CSV output exists, and has a nonzero number of rows.
+
+
+
+Once we reload the code location, we'll see a tab `checks` indicating the presence of an asset check on our `rebuild_customers_list` asset.
+
+![Asset check on peer DAG](/images/integrations/airlift/asset_check_peered_dag.png)
+
+Let's run the backfill again:
+
+```bash
+airflow dags backfill rebuild_customers_list --start-date $(shell date +"%Y-%m-%d")
+```
+
+And we'll see that the asset check executed successfully in Dagster (indicated by the green check mark).
+
+![Asset check success](/images/integrations/airlift/peer_check_success.png)
+
+Let's again wipe materializations and runs for tutorial purposes.
+
+```bash
+make clean
+```
+
+## Next steps
+
+The next step is to start observing the asset dependencies within your DAG. Follow along at the Observe stage of the tutorial [here](observe)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/reference.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/reference.md
new file mode 100644
index 0000000000000..42004b62457e5
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/reference.md
@@ -0,0 +1,94 @@
+---
+title: "dagster-airlift integration reference"
+description: "dagster-airlift is a toolkit for observing and migrating Airflow DAGs within Dagster."
+sidebar_position: 700
+---
+
+`dagster-airlift` is a toolkit for observing and migrating Airflow DAGs within Dagster. This reference page provides additional information for working with `dagster-airlift` that is not provided within the tutorial. You should start by reading the [dagster-airlift tutorial](index.md) before using this reference page.
+
+- [Migration best practices](#migration-best-practices)
+- [Supporting custom authorization](#supporting-custom-authorization)
+- [Dagster Plus Authorization](#dagster-authorization)
+- [Dealing with changing Airflow](#dealing-with-changing-airflow)
+- [Automating changes to code locations](#automating-changes-to-code-locations)
+- [Peering to multiple Airflow instances](#peering-to-multiple-airflow-instances)
+- [Customizing DAG proxying operator](#customizing-dag-proxying-operator)
+
+## Migration best practices
+
+When migrating Airflow DAGs to Dagster, we recommend a few best practices:
+
+- **Create separate packages for the Airflow and Dagster deployments.** Airflow has complex dependencies and can be difficult to install in the same environment as Dagster.
+- **Create user acceptance tests in Dagster before migrating.** This will help you catch issues easily during migration.
+- **Understand the rollback procedure for your migration.** When proxying execution to Dagster from Airflow, you can always rollback with a single line-of-code change in the Airflow DAG.
+
+### Supporting custom authorization
+
+If your Dagster deployment lives behind a custom auth backend, you can customize the Airflow-to-Dagster proxying behavior to authenticate to your backend. `proxying_to_dagster` can take a parameter `dagster_operator_klass`, which allows you to define a custom `BaseProxyTasktoDagsterOperator` class. This allows you to override how a session is created. Let's say for example, your Dagster installation requires an access key to be set whenever a request is made, and that access key is set in an Airflow `Variable` called `my_api_key`. We can create a custom `BaseProxyTasktoDagsterOperator` subclass which will retrieve that variable value and set it on the session, so that any requests to Dagster's graphql API will be made using that api key.
+
+
+
+### Dagster+ authorization
+
+You can use a custom proxy operator to establish a connection to a Dagster plus deployment. The below example proxies to Dagster Plus using organization name, deployment name, and user token set as Airflow Variables. To set a Dagster+ user token, see "[Managing user tokens in Dagster+](/dagster-plus/deployment/management/tokens/user-tokens)".
+
+
+
+### Dealing with changing Airflow
+
+In order to make spin-up more efficient, `dagster-airlift` caches the state of the Airflow instance in the dagster database, so that repeat fetches of the code location don't require additional calls to Airflow's rest API. However, this means that the Dagster definitions can potentially fall out of sync with Airflow. Here are a few different ways this can manifest:
+
+- A new Airflow DAG is added. The lineage information does not show up for this dag, and materializations are not recorded.
+- A DAGis removed. The polling sensor begins failing, because there exist assets which expect that DAGto exist.
+- The task dependency structure within a DAGchanges. This may result in `unsynced` statuses in Dagster, or missing materializations. This is not an exhaustive list of problems, but most of the time the tell is that materializations are missing, or assets are missing. When you find yourself in this state, you can force `dagster-airlift` to reload Airflow state by reloading the code location. To do this, go to the `Deployment` tab on the top nav, and click `Redeploy` on the code location relevant to your asset. After some time, the code location should be reloaded with refreshed state from Airflow.
+
+### Automating changes to code locations
+
+If changes to your Airflow instance are controlled via a ci/cd process, you can add a step to automatically induce a redeploy of the relevant code location. To learn how to use the Dagster GraphQL client to do this, see the [Dagster GraphQL client docs](/guides/operate/graphql/graphql-client).
+
+### Peering to multiple Airflow instances
+
+Airlift supports peering to multiple Airflow instances, as you can invoke `build_defs_from_airflow_instance` multiple times and combine them with `Definitions.merge`:
+
+```python
+from dagster import Definitions
+
+from dagster_airlift.core import AirflowInstance, build_defs_from_airflow_instance
+
+defs = Definitions.merge(
+ build_defs_from_airflow_instance(
+ airflow_instance=AirflowInstance(
+ auth_backend=BasicAuthBackend(
+ webserver_url="http://yourcompany.com/instance_one",
+ username="admin",
+ password="admin",
+ ),
+ name="airflow_instance_one",
+ )
+ ),
+ build_defs_from_airflow_instance(
+ airflow_instance=AirflowInstance(
+ auth_backend=BasicAuthBackend(
+ webserver_url="http://yourcompany.com/instance_two",
+ username="admin",
+ password="admin",
+ ),
+ name="airflow_instance_two",
+ )
+ ),
+)
+```
+
+### Customizing DAG proxying operator
+
+Similar to how we can customize the operator we construct on a per-DAGbasis, we can customize the operator we construct on a per-DAGbasis. We can use the `build_from_dag_fn` argument of `proxying_to_dagster` to provide a custom operator in place of the default.
+
+For example, in the following example we can see that the operator is customized to provide an authorization header which authenticates Dagster.
+
+
+
+`BaseProxyDAGToDagsterOperator` has three abstract methods which must be implemented:
+
+- `get_dagster_session`, which controls the creation of a valid session to access the Dagster graphql API.
+- `get_dagster_url`, which retrieves the domain at which the dagster webserver lives.
+- `build_from_dag`, which controls how the proxying task is constructed from the provided DAG.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/setup.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/setup.md
new file mode 100644
index 0000000000000..9bd4e12b77042
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/airflow-to-dagster/setup.md
@@ -0,0 +1,82 @@
+---
+title: "Set up a local environment"
+sidebar_position: 100
+---
+
+In this step, we'll
+
+- Install the example code
+- Set up a local environment
+- Ensure we can run Airflow locally.
+
+## Installation & project structure
+
+First, we'll create a fresh virtual environment using `uv`.
+
+```bash
+pip install uv
+uv venv
+source .venv/bin/activate
+```
+
+Next, we'll install Dagster, and verify that the dagster CLI is available.
+
+```bash
+uv pip install dagster
+dagster --version
+```
+
+First, we'll create a fresh virtual environment using `uv`.
+
+```bash
+dagster project from-example --name airlift-migration-tutorial --example airlift-migration-tutorial
+```
+
+### Project structure
+
+The following explains the structure of the repo.
+
+```plaintext
+tutorial_example
+├── shared: Contains shared Python & SQL code used Airflow and proxied Dagster code
+│
+├── dagster_defs: Contains Dagster definitions
+│ ├── stages: Contains reference implementations of each stage of the migration process
+│ ├── definitions.py: Empty starter file for following along with the tutorial
+│
+├── airflow_dags: Contains the Airflow DAG and associated files
+│ ├── proxied_state: Contains migration state files for each DAG, see migration step below
+│ ├── dags.py: The Airflow DAG definition
+```
+
+## Running Airflow locally
+
+The tutorial example involves running a local Airflow instance. This can be done by running the following commands from the root of the `airlift-migration-tutorial` directory.
+
+First, install the required python packages:
+
+```bash
+make airflow_install
+```
+
+Next, scaffold the Airflow instance, and initialize the dbt project:
+
+```bash
+make airflow_setup
+```
+
+Finally, run the Airflow instance with environment variables set:
+
+```bash
+make airflow_run
+```
+
+This will run the Airflow Web UI in a shell. You should now be able to access the Airflow UI at `http://localhost:8080`, with the default username and password set to `admin`.
+
+You should be able to see the `rebuild_customers_list` DAG in the Airflow UI, made up of three tasks: `load_raw_customers`, `run_dbt_model`, and `export_customers`.
+
+![Rebuild customers list DAG](/images/integrations/airlift/rebuild_customers_dag.png)
+
+## Next steps
+
+The next step is to peer a Dagster installation with the Airflow Instance. [Click here](peer) to follow along for part 2.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/federated-execution.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/federated-execution.md
new file mode 100644
index 0000000000000..79cb6f7c2b8e9
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/federated-execution.md
@@ -0,0 +1,46 @@
+---
+title: "Federating execution across Airflow instances"
+sidebar_position: 300
+---
+
+At this point, we should be [observing our DAGs within Dagster](observe), and now we have cross-instance lineage for our DAGs. Now, we'll federate the execution of our DAGs across both Airflow instances by using Dagster's Declarative Automation system.
+
+## Making `customer_metrics` executable
+
+The `load_airflow_dag_asset_specs` function creates asset representations (called `AssetSpec`) of Airflow DAGs, but these assets are not executable. We need to define an execution function in Dagster in order to make them executable.
+
+In order to federate execution of `customer_metrics`, we first need to make it executable within Dagster. We can do this by using the `@multi_asset` decorator to define how the `customer_metrics` asset should be executed. We'll use the `AirflowInstance` defined earlier to trigger a run of the `customer_metrics` DAG. We then wait for the run to complete, and if it is successful, we'll successfully materialize the asset. If the run fails, we'll raise an exception.
+
+
+
+Now, we'll replace the `customer_metrics_dag_asset` in our `Definitions` object with the `run_customer_metrics` function:
+
+
+
+We should be able to go to the Dagster UI and see that the `customer_metrics` asset can now be materialized.
+
+## Federating execution
+
+Ultimately, we would like to kick off a run of `customer_metrics` whenever `load_customers` completes successfully. We're already retrieving a materialization when `load_customers` completes, so we can use this to trigger a run of `customer_metrics` by using Declarative Automation. First, we'll add an `AutomationCondition.eager()` to our `customer_metrics_dag_asset`. This will tell Dagster to run the `run_customer_metrics` function whenever the `load_customers` asset is materialized.
+
+
+
+Now, we can set up Declarative Automation by adding an `AutomationConditionSensorDefinition`.
+
+
+
+We'll add this sensor to our `Definitions` object.
+
+
+
+Now the `run_customer_metrics` function will be executed whenever the `load_customers` asset is materialized. Let's test this out by triggering a run of the `load_customers` DAG in Airflow. When the run completes, we should see a materialization of the `customer_metrics` asset kick off in the Dagster UI, and eventually a run of the `customer_metrics` DAG in the metrics Airflow instance.
+
+## Complete code
+
+When all the above steps are complete, your code should look something like this.
+
+
+
+## Conclusion
+
+That concludes the tutorial! We've federated the execution of our DAGs across two Airflow instances using Dagster's Declarative Automation system. We've also set up cross-instance lineage for our DAGs, and can now observe the lineage and execution of our DAGs in the Dagster UI.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/index.md
new file mode 100644
index 0000000000000..9c63925721075
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/index.md
@@ -0,0 +1,29 @@
+---
+title: "Federating execution between Airflow instances with Dagster"
+sidebar_position: 20
+---
+
+This tutorial demonstrates using `dagster-airlift` to observe DAGs from multiple Airflow instances, and federate execution between them using Dagster as a centralized control plane.
+
+Using `dagster-airlift` we can
+
+- Observe Airflow DAGs and their execution history
+- Directly trigger Airflow DAGs from Dagster
+- Set up federated execution _across_ Airflow instances
+
+All of this can be done with no changes to Airflow code.
+
+## Overview
+
+This tutorial will take you through an imaginary data platform team that has the following scenario:
+
+- An Airflow instance `warehouse`, run by another team, that is responsible for loading data into a data warehouse.
+- An Airflow instance `metrics`, run by the data platform team, that deploys all the metrics constructed by data scientists on top of the data warehouse.
+
+Two DAGs have been causing a lot of pain lately for the team: `warehouse.load_customers` and `metrics.customer_metrics`. The `warehouse.load_customers` DAG is responsible for loading customer data into the data warehouse, and the `metrics.customer_metrics` DAG is responsible for computing metrics on top of the customer data. There's a cross-instance dependency relationship between these two DAGs, but it's not observable or controllable. The data platform team would ideally _only_ like to rebuild the `metrics.customer_metrics` DAG when the `warehouse.load_customers` DAG has new data. In this guide, we'll use `dagster-airlift` to observe the `warehouse` and `metrics` Airflow instances, and set up a federated execution controlled by Dagster that only triggers the `metrics.customer_metrics` DAG when the `warehouse.load_customers` DAG has new data. This process won't require any changes to the Airflow code.
+
+## Pages
+
+* [Setup](setup)
+* [Observing multiple Airflow instances](observe)
+* [Federating execution across Airflow instances](federated-execution)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/observe.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/observe.md
new file mode 100644
index 0000000000000..8ad690dd3ae98
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/observe.md
@@ -0,0 +1,90 @@
+---
+title: "Observing multiple Airflow instances"
+sidebar_position: 200
+---
+
+At this point, you should have finished the [setup](setup) step, and now the example code set up with a fresh virtual environment, and two Airflow instances running locally. Now, we can start writing Dagster code.
+
+## Observing the Airflow instances
+
+We'll start by creating asset representations of our DAGs in Dagster.
+
+Create a new shell and navigate to the root of the tutorial directory. You will need to set up the `dagster-airlift` package in your Dagster environment:
+
+```bash
+source .venv/bin/activate
+uv pip install 'dagster-airlift[core]' dagster-webserver dagster
+```
+
+### Observing the `warehouse` Airflow instance
+
+Next, we'll declare a reference to our `warehouse` Airflow instance, which is running at `http://localhost:8081`.
+
+
+
+Now, we can use the `load_airflow_dag_asset_specs` function to create asset representations of the DAGs in the `warehouse` Airflow instance:
+
+
+
+Now, let's add these assets to a `Definitions` object:
+
+
+
+Let's set up some environment variables, and then point Dagster to see the asset created from our Airflow instance:
+
+```bash
+# Set up environment variables to point to the airlift-federation-tutorial directory on your machine
+export TUTORIAL_EXAMPLE_DIR=$(pwd)
+export DAGSTER_HOME="$TUTORIAL_EXAMPLE_DIR/.dagster_home"
+dagster dev -f airlift_federation_tutorial/dagster_defs/definitions.py
+```
+
+If we navigate to the Dagster UI (running at `http://localhost:3000`), we should see the assets created from the `warehouse` Airflow instance.
+
+![Assets from the warehouse Airflow instance in the Dagster UI](/images/integrations/airlift/observe_warehouse.png)
+
+There's a lot of DAGs in this instance, and we only want to focus on the `load_customers` DAG. Let's filter the assets to only include the `load_customers` DAG:
+
+
+
+Let's instead add this asset to our `Definitions` object:
+
+
+
+Now, our Dagster environment only includes the `load_customers` DAG from the `warehouse` Airflow instance.
+
+![Assets from the warehouse Airflow instance in the Dagster UI](/images/integrations/airlift/only_load_customers.png)
+
+Finally, we'll use a sensor to poll the `warehouse` Airflow instance for new runs. This way, whenever we get a successful run of the `load_customers` DAG, we'll see a materialization in the Dagster UI:
+
+
+
+Now, we can add this sensor to our `Definitions` object:
+
+
+
+You can test this by navigating to the Airflow UI at localhost:8081, and triggering a run of the `load_customers` DAG. When the run completes, you should see a materialization in the Dagster UI.
+
+![Materialization of the load_customers DAG in the Dagster UI](/images/integrations/airlift/load_customers_mat.png)
+
+### Observing the `metrics` Airflow instance
+
+We can repeat the same process for the `customer_metrics` DAG in the `metrics` Airflow instance, which runs at `http://localhost:8082`. We'll leave this as an exercise to test your understanding.
+
+When complete, your code should look like this:
+
+
+
+### Adding lineage between `load_customers` and `customer_metrics`
+
+Now that we have both DAGs loaded into Dagster, we can observe the cross-dag lineage between them. To do this, we'll use the `replace_attributes` function to add a dependency from the `load_customers` asset to the `customer_metrics` asset:
+
+
+
+Now, after adding the updated `customer_metrics_dag_asset` to our `Definitions` object, we should see the lineage between the two DAGs in the Dagster UI.
+
+![Lineage between load_customers and customer_metrics in the Dagster UI](/images/integrations/airlift/dag_lineage.png)
+
+## Next steps
+
+Next, we'll federate the execution of our DAGs across both Airflow instances. Follow along [here](federated-execution).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/setup.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/setup.md
new file mode 100644
index 0000000000000..fafcae7d2712e
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/federation-tutorial/setup.md
@@ -0,0 +1,93 @@
+---
+title: "Setup"
+sidebar_position: 100
+---
+
+In this step, we'll
+
+- Install the example code
+- Set up a local environment
+- Ensure we can run Airflow locally.
+
+## Installation & project structure
+
+First, we'll create a fresh virtual environment using `uv` and activate it.
+
+```bash
+pip install uv
+uv venv
+source .venv/bin/activate
+```
+
+Next, we'll install Dagster, and verify that the dagster CLI is available.
+
+```bash
+uv pip install dagster
+dagster --version
+```
+
+Finally, we'll install the tutorial example code.
+
+```bash
+dagster project from-example --name airlift-federation-tutorial --example airlift-federation-tutorial
+```
+
+### Project structure
+
+The following explains the structure of the repo.
+
+```plaintext
+airlift_federation_tutorial
+├── constants.py: Contains constant values used throughout both Airflow and Dagster
+├── dagster_defs: Contains Dagster definitions
+│ ├── definitions.py: Empty starter file for following along with the tutorial
+│ └── stages: Contains reference implementations for each stage of the migration process.
+├── metrics_airflow_dags: Contains the Airflow DAGs for the "downstream" airflow instance
+└── warehouse_airflow_dags: Contains the Airflow DAGs for the "upstream" airflow instance
+```
+
+## Running Airflow locally
+
+The tutorial example involves running a local Airflow instance. This can be done by running the following commands from the root of the `airlift-migration-tutorial` directory.
+
+First, install the required python packages:
+
+```bash
+make airflow_install
+```
+
+Next, scaffold the two Airflow instances we'll be using for this tutorial:
+
+```bash
+make airflow_setup
+```
+
+Finally, let's run the two Airflow instances with environment variables set:
+
+In one shell run:
+
+```bash
+make warehouse_airflow_run
+```
+
+In a separate shell, run:
+
+```bash
+make metrics_airflow_run
+```
+
+This will run two Airflow Web UIs, one for each Airflow instance. You should now be able to access the warehouse Airflow UI at `http://localhost:8081`, with the default username and password set to `admin`.
+
+You should be able to see the `load_customers` DAG in the Airflow UI.
+
+![load_customers DAG](/images/integrations/airlift/load_customers.png)
+
+Similarly, you should be able to access the metrics Airflow UI at `http://localhost:8082`, with the default username and password set to `admin`.
+
+You should be able to see the `customer_metrics` DAG in the Airflow UI.
+
+![customer_metrics DAG](/images/integrations/airlift/customer_metrics.png)
+
+## Next steps
+
+In the next section, we'll add asset representations of our DAGs, and set up lineage across both Airflow instances. Follow along [here](observe).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/index.md
new file mode 100644
index 0000000000000..2b4f72187b420
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/index.md
@@ -0,0 +1,57 @@
+---
+layout: Integration
+status: published
+name: Airlift
+title: Dagster & Airlift
+sidebar_label: Airlift
+excerpt: Easily integrate Dagster and Airflow.
+date:
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-airlift
+docslink: https://docs.dagster.io/integrations/libraries/airlift/
+partnerlink:
+logo:
+categories:
+ - ETL
+enabledBy:
+enables:
+tags: [dagster-supported, other]
+sidebar_custom_props:
+ logo: images/integrations/airflow.svg
+---
+
+Airlift is a toolkit for integrating Dagster and Airflow. Using `dagster-airflift`, you can:
+
+- Observe Airflow instances from within Dagster
+- Accelerate the migration of Airflow DAGs to Dagster assets with opinionated tooling
+
+## Compatibility
+
+### REST API Availability
+
+Airlift depends on the availability of Airflow’s REST API. Airflow’s REST API was made stable in its 2.0 release (Dec 2020) and was introduced experimentally in 1.10 in August 2018. Currently Airflow requires the availability of the REST API.
+
+- **OSS:** Stable as of 2.00
+- **MWAA**
+ - Note: only available in Airflow 2.4.3 or later on MWAA.
+- **Cloud Composer:** No limitations as far as we know.
+- **Astronomer:** No limitations as far as we know.
+
+## Migrating from Airflow to Dagster
+
+You can use Airlift to migrate an Airflow DAG to Dagster assets. Airlift enables a migration process that
+
+- Can be done task-by-task in any order with minimal coordination
+- Has task-by-task rollback to reduce risk
+- Retains Airflow DAG structure and execution history during the migration
+
+To get started, see "[Migrating from Airflow to Dagster](airflow-to-dagster/)".
+
+## Federating execution between Airflow instances with Dagster
+
+You can use Airflit to observe DAGs from multiple Airflow instances, and federate execution between them using Dagster as a centralized control plane.
+
+To get started, see "[Federating execution between Airflow instances with Dagster](federation-tutorial/)".
+
+## Migrating common Airflow operators to Dagster
+
+You can easily migrate common Airflow operators to Dagster. For more information, see the "[Migrating common Airflow operators to Dagster](operator-migration/)".
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/bash-operator-dbt.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/bash-operator-dbt.md
new file mode 100644
index 0000000000000..eb47d4408bc83
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/bash-operator-dbt.md
@@ -0,0 +1,41 @@
+---
+title: "Migrating an Airflow BashOperator (dbt) to Dagster"
+sidebar_position: 200
+---
+
+In this page, we'll explain migrating an Airflow `BashOperator` that runs a `dbt` command to Dagster.
+
+## About the Airflow BashOperator
+
+In Airflow, you might have a `BashOperator` that runs a `dbt` command. For example, you might have a task that runs `dbt run` to build your dbt models.
+
+
+
+## Dagster equivalent
+
+The Dagster equivalent is to instead use the `dagster-dbt` library to run commands against your dbt project. Here would be the equivalent code in Dagster:
+
+
+
+## Migrating the operator
+
+Migrating the operator breaks down into a few steps:
+
+1. Making the dbt project available to both your Airflow and Dagster deployments.
+2. Writing a @dbt_asset-decorated function which runs your dbt commands.
+3. Using `dagster-airlift` to proxy execution of the original task to Dagster.
+
+### Step 1: Making the dbt project available & building manifest
+
+First, you'll need to make the dbt project available to the Dagster runtime and build the manifest.
+
+- If you're building your Dagster deployment in a monorepo alongside your dbt and Airflow projects, you can follow this guide: [Monorepo setup](/integrations/libraries/dbt/reference#deploying-a-dagster-project-with-a-dbt-project).
+- If you're deploying within a separate repository, you can follow this guide: [Separate repository setup](/integrations/libraries/dbt/reference#deploying-a-dbt-project-from-a-separate-git-repository). */}
+
+### Step 2: Writing a @dbt_asset-decorated function
+
+Once your dbt project is available, you can write a function that runs your dbt commands using the decorator and . Most dbt CLI commands and flags are supported - to learn more about using `@dbt_assets`, check out the [dagster-dbt quickstart](/integrations/libraries/dbt/transform-dbt) and [reference](/integrations/libraries/dbt/reference).
+
+### Step 3: Using dagster-airlift to proxy execution
+
+Finally, you can use `dagster-airlift` to proxy the execution of the original task to Dagster. For more information, see "[Migrating from Airflow to Dagster](/integrations/libraries/airlift/airflow-to-dagster).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/bash-operator-general.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/bash-operator-general.md
new file mode 100644
index 0000000000000..d6d406c981a05
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/bash-operator-general.md
@@ -0,0 +1,63 @@
+---
+title: "Migrating an Airflow BashOperator to Dagster"
+sidebar_position: 100
+---
+
+In this page, we'll explain migrating an Airflow `BashOperator` to Dagster.
+
+:::note
+
+If using the `BashOperator` to execute dbt commands, see "[Migrating an Airflow BashOperator (dbt) to Dagster](bash-operator-dbt)".
+
+:::
+
+## About the Airflow BashOperator
+
+The Airflow `BashOperator` is a common operator used to execute bash commands as part of a data pipeline.
+
+
+
+The `BashOperator`'s functionality is very general since it can be used to run any bash command, and there exist richer integrations in Dagster for many common BashOperator use cases. We'll explain how 1-1 migration of the BashOperator to execute a bash command in Dagster, and how to use the `dagster-airlift` library to proxy the execution of the original task to Dagster. We'll also provide a reference for richer integrations in Dagster for common BashOperator use cases.
+
+## Dagster equivalent
+
+The direct Dagster equivalent to the `BashOperator` is to use the to execute a bash command in a subprocess.
+
+## Migrating the operator
+
+Migrating the operator breaks down into a few steps:
+
+1. Ensure that the resources necessary for your bash command are available to both your Airflow and Dagster deployments.
+2. Write an that executes the bash command using the .
+3. Use `dagster-airlift` to proxy execution of the original task to Dagster.
+4. (Optional) Implement a richer integration for common BashOperator use cases.
+
+### Step 1: Ensure shared bash command access
+
+First, you'll need to ensure that the bash command you're running is available for use in both your Airflow and Dagster deployments. What this entails will vary depending on the command you're running. For example, if you're running a Python script, it's as simple as ensuring the Python script exists in a shared location accessible to both Airflow and Dagster, and all necessary env vars are set in both environments.
+
+### Step 2: Writing an `@asset`-decorated function
+
+You can write a Dagster -decorated function that runs your bash command. This is quite straightforward using the .
+
+
+
+### Step 3: Using dagster-airlift to proxy execution
+
+Finally, you can use `dagster-airlift` to proxy the execution of the original task to Dagster. For more information, see "[Migrating from Airflow to Dagster](/integrations/libraries/airlift/airflow-to-dagster).
+
+### Step 4: Implementing richer integrations
+
+For many of the use cases that you might be using the BashOperator for, Dagster might have better options. We'll detail some of those here.
+
+#### Running a Python script
+
+As mentioned above, you can use the to run a Python script in a subprocess. But you can also modify this script to send additional information and logging back to Dagster. See the [Dagster Pipes tutorial](/guides/build/external-pipelines/) for more information.
+
+#### Running a dbt command
+
+We have a whole guide for switching from the `BashOperator` to the `dbt` integration in Dagster. For more information, see "[Migrating an Airflow BashOperator (dbt) to Dagster](bash-operator-dbt)".
+
+#### Running S3 Sync or other AWS CLI commands
+
+Dagster has a rich set of integrations for AWS services. For example, you can use the to interact with S3 directly.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/index.md
new file mode 100644
index 0000000000000..8d5d822cd530a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/index.md
@@ -0,0 +1,11 @@
+---
+title: "Migrating common Airflow operators to Dagster"
+sidebar_position: 30
+---
+
+You can easily migrate usage of common Airflow operator types to Dagster.
+
+import DocCardList from '@theme/DocCardList';
+
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/kubernetes-pod-operator.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/kubernetes-pod-operator.md
new file mode 100644
index 0000000000000..43bf8272f07f9
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/kubernetes-pod-operator.md
@@ -0,0 +1,100 @@
+---
+title: "Migrating an Airflow KubernetesPodOperator to Dagster"
+sidebar_position: 400
+---
+
+In this page, we'll explain migrating an Airflow `KubernetesPodOperator` to Dagster.
+
+## About the Airflow KubernetesPodOperator
+
+The KubernetesPodOperator in Apache Airflow enables users to execute containerized tasks within Kubernetes pods as part of their data pipelines.
+
+
+
+## Dagster equivalent
+
+The Dagster equivalent is to use the to execute a task within a Kubernetes pod.
+
+
+
+## Migrating the operator
+
+Migrating the operator breaks down into a few steps:
+
+1. Ensure that your Dagster deployment has access to the Kubernetes cluster.
+2. Write an that executes the task within a Kubernetes pod using the .
+3. Use `dagster-airlift` to proxy execution of the original task to Dagster.
+
+### Step 1: Ensure access to the Kubernetes cluster
+
+First, you need to ensure that your Dagster deployment has access to the Kubernetes cluster where you want to run your tasks. The accepts `kubeconfig` and `kubecontext`, and `env` arguments to configure the Kubernetes client.
+
+Here's an example of what this might look like when configuring the client to access an EKS cluster:
+
+
+
+### Step 2: Writing an asset that executes the task within a Kubernetes pod
+
+Once you have access to the Kubernetes cluster, you can write an asset that executes the task within a Kubernetes pod using the . In comparison to the KubernetesPodOperator, the PipesK8sClient allows you to define the pod spec directly in your Python code.
+
+In the [parameter comparison](#parameter-comparison) section of this doc, you'll find a detailed comparison describing how to map the KubernetesPodOperator parameters to the PipesK8sClient parameters.
+
+
+
+This is just a snippet of what the PipesK8sClient can do. Take a look at our full guide on the [dagster-k8s PipesK8sClient](/guides/build/external-pipelines/kubernetes-pipeline) for more information.
+
+### Step 3: Using dagster-airlift to proxy execution
+
+Finally, you can use `dagster-airlift` to proxy the execution of the original task to Dagster. For more information, see "[Migrating from Airflow to Dagster](/integrations/libraries/airlift/airflow-to-dagster).
+
+## Parameter comparison
+
+Here's a comparison of the parameters between the KubernetesPodOperator and the PipesK8sClient: Directly supported arguments:
+
+- in_cluster (named load_incluster_config in the PipesK8sClient)
+- cluster_context (named kube_context in the PipesK8sClient)
+- config_file (named kubeconfig_file in the PipesK8sClient)
+
+Many arguments are supported indirectly via the `base_pod_spec` argument.
+
+- volumes: Volumes to be used by the Pod (key `volumes`)
+- affinity: Node affinity/anti-affinity rules for the Pod (key `affinity`)
+- node_selector: Node selection constraints for the Pod (key `nodeSelector`)
+- hostnetwork: Enable host networking for the Pod (key `hostNetwork`)
+- dns_config: DNS settings for the Pod (key `dnsConfig`)
+- dnspolicy: DNS policy for the Pod (key `dnsPolicy`)
+- hostname: Hostname of the Pod (key `hostname`)
+- subdomain: Subdomain for the Pod (key `subdomain`)
+- schedulername: Scheduler to be used for the Pod (key `schedulerName`)
+- service_account_name: Service account to be used by the Pod (key `serviceAccountName`)
+- priority_class_name: Priority class for the Pod (key `priorityClassName`)
+- security_context: Security context for the entire Pod (key `securityContext`)
+- tolerations: Tolerations for the Pod (key `tolerations`)
+- image_pull_secrets: Secrets for pulling container images (key `imagePullSecrets`)
+- termination_grace_period: Grace period for Pod termination (key `terminationGracePeriodSeconds`)
+- active_deadline_seconds: Deadline for the Pod's execution (key `activeDeadlineSeconds`)
+- host_aliases: Additional entries for the Pod's /etc/hosts (key `hostAliases`)
+- init_containers: Initialization containers for the Pod (key `initContainers`)
+
+The following arguments are supported under the nested `containers` key of the `base_pod_spec` argument of the PipesK8sClient:
+
+- image: Docker image for the container (key 'image')
+- cmds: Entrypoint command for the container (key `command`)
+- arguments: Arguments for the entrypoint command (key `args`)
+- ports: List of ports to expose from the container (key `ports`)
+- volume_mounts: List of volume mounts for the container (key `volumeMounts`)
+- env_vars: Environment variables for the container (key `env`)
+- env_from: List of sources to populate environment variables (key `envFrom`)
+- image_pull_policy: Policy for pulling the container image (key `imagePullPolicy`)
+- container_resources: Resource requirements for the container (key `resources`)
+- container_security_context: Security context for the container (key `securityContext`)
+- termination_message_policy: Policy for the termination message (key `terminationMessagePolicy`)
+
+For a full list, see the [kubernetes container spec documentation](https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.26/#container-v1-core). The following arguments are supported under the `base_pod_meta` argument, which configures the metadata of the pod:
+
+- name: `name`
+- namespace: `namespace`
+- labels: `labels`
+- annotations: `annotations`
+
+For a full list, see the [kubernetes objectmeta spec documentation](https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.26/#objectmeta-v1-meta).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/python-operator.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/python-operator.md
new file mode 100644
index 0000000000000..4111e25caafba
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/airlift/operator-migration/python-operator.md
@@ -0,0 +1,77 @@
+---
+title: "Migrating an Airflow PythonOperator to Dagster"
+sidebar_position: 300
+---
+
+In this page, we'll explain migrating an Airflow `PythonOperator` to Dagster.
+
+## About the Airflow PythonOperator
+
+In Airflow, the `PythonOperator` runs arbitrary Python functions. For example, you might have a task that runs a function `write_to_db`, which combs a directory for files, and writes each one to a db table.
+
+
+
+## Dagster equivalent
+
+The Dagster equivalent is instead to construct a or -decorated function, which materializes assets corresponding to what your python function is doing.
+
+
+
+## Migrating the operator
+
+Migrating the operator breaks down into a few steps:
+
+1. Make a shared library available to both Airflow and Dagster with your python function.
+2. Writing an `@asset`-decorated function which runs the python function shared between both modules.
+3. Using `dagster-airlift` to proxy execution of the original task to Dagster.
+
+### Step 1: Building a shared library
+
+We recommend a monorepo setup for migration; this allows you to keep all your code in one place and easily share code between Airflow and Dagster, without complex CI/CD coordination.
+
+First, we recommend factoring out a shared package to be available to both the Dagster runtime and the Airflow runtime which contains your python function. The process is as follows:
+
+1. Scaffold out a new python project which will contain your shared infrastructure.
+2. Ensure that the shared library is available to both your Airflow and Dagster deployments. This can be done by adding an editable requirement to your `setup.py` or `pyproject.toml` file in your Airflow/Dagster package.
+3. Include the python dependencies relevant to your particular function in your new package. Write your python function in the shared package, and change your Airflow code to import the function from the shared library.
+
+To illustrate what this might look like a bit more; let's say you originally have this project structure in Airflow:
+
+```plaintext
+airflow_repo/
+├── airflow_package/
+│ └── dags/
+│ └── my_dag.py # Contains your Python function
+```
+
+With dag code that looks this:
+
+
+
+You might create a new top-level package to contain the shared code:
+
+```plaintext
+airflow_repo/
+├── airflow_package/
+│ └── dags/
+│ └── my_dag.py # Imports the python function from shared module.
+├── shared-package/
+│ └── shared_package/
+│ └── shared_module.py # Contains your Python function
+```
+
+And then import the function from the shared package in Airflow:
+
+
+
+The reason we recommend using a separate `shared` package is to help ensure that there aren't dependency conflicts between Airflow and Dagster as you migrate. Airflow has very complex dependency management, and migrating to Dagster gives you an opportunity to clean up and isolate your dependencies. You can do this with a series of shared packages in the monorepo, which will eventually be isolated code locations in Dagster.
+
+### Step 2: Writing an `@asset`-decorated function
+
+Next, you can write a Dagster or -decorated function that runs your python function. This will generally be pretty straightforward for a `PythonOperator` migration, as you can generally just invoke the shared function into the `asset` function.
+
+
+
+### Step 3: Using `dagster-airlift` to proxy execution
+
+Finally, you can use `dagster-airlift` to proxy the execution of the original task to Dagster. For more information, see "[Migrating from Airflow to Dagster](/integrations/libraries/airlift/airflow-to-dagster).
\ No newline at end of file
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/anthropic.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/anthropic.md
new file mode 100644
index 0000000000000..e8647e1dd4347
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/anthropic.md
@@ -0,0 +1,34 @@
+---
+layout: Integration
+status: published
+name: Anthropic
+title: Dagster & Anthropic
+sidebar_label: Anthropic
+excerpt: Integrate Anthropic calls into your Dagster pipelines, without breaking the bank.
+partnerlink: https://docs.anthropic.com/en/api/getting-started
+categories:
+ - Other
+enabledBy:
+enables:
+tags: [dagster-supported]
+sidebar_custom_props:
+ logo: images/integrations/anthropic.svg
+---
+
+The `dagster-anthropic` library allows you to easily interact with the Anthropic REST API using the Anthropic Python API to build AI steps into your Dagster pipelines. You can also log Anthropic API usage metadata in Dagster Insights, giving you detailed observability on API call credit consumption.
+
+When paired with Dagster assets, the resource automatically logs Anthropic usage metadata in asset metadata.
+
+### Installation
+
+```bash
+pip install dagster dagster-anthropic
+```
+
+### Example
+
+
+
+### About Anthropic
+
+Anthropic is an AI research company focused on developing safe and ethical AI systems. Their flagship product, Claude, is a language model known for its strong capabilities in analysis, writing, and coding tasks while maintaining high standards of truthfulness and safety.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/athena.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/athena.md
new file mode 100644
index 0000000000000..2867613455184
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/athena.md
@@ -0,0 +1,35 @@
+---
+layout: Integration
+status: published
+name: Athena
+title: Dagster & AWS Athena
+sidebar_label: Athena
+excerpt: This integration allows you to connect to AWS Athena and analyze data in Amazon S3 using standard SQL within your Dagster pipelines.
+date: 2024-06-21
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-aws
+docslink:
+partnerlink: https://aws.amazon.com/
+categories:
+ - Storage
+enabledBy:
+enables:
+tags: [dagster-supported, storage]
+sidebar_custom_props:
+ logo: images/integrations/aws-athena.svg
+---
+
+This integration allows you to connect to AWS Athena, a serverless interactive query service that makes it easy to analyze data in Amazon S3 using standard SQL. Using this integration, you can issue queries to Athena, fetch results, and handle query execution states within your Dagster pipelines.
+
+### Installation
+
+```bash
+pip install dagster-aws
+```
+
+### Examples
+
+
+
+### About AWS Athena
+
+AWS Athena is a serverless, interactive query service that allows you to analyze data directly in Amazon S3 using standard SQL. Athena is easy to use; point to your data in Amazon S3, define the schema, and start querying using standard SQL. Most results are delivered within seconds. With Athena, there are no infrastructure setups, and you pay only for the queries you run. It scales automatically—executing queries in parallel—so results are fast, even with large datasets and complex queries.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/cloudwatch.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/cloudwatch.md
new file mode 100644
index 0000000000000..fa5613d94888d
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/cloudwatch.md
@@ -0,0 +1,45 @@
+---
+layout: Integration
+status: published
+name: CloudWatch
+title: Dagster & AWS CloudWatch
+sidebar_label: CloudWatch
+excerpt: This integration allows you to send Dagster logs to AWS CloudWatch, enabling centralized logging and monitoring of your Dagster jobs.
+date: 2024-06-21
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-aws
+docslink:
+partnerlink: https://aws.amazon.com/
+categories:
+ - Monitoring
+enabledBy:
+enables:
+tags: [dagster-supported, monitoring]
+sidebar_custom_props:
+ logo: images/integrations/aws-cloudwatch.svg
+---
+
+This integration allows you to send Dagster logs to AWS CloudWatch, enabling centralized logging and monitoring of your Dagster jobs. By using AWS CloudWatch, you can take advantage of its powerful log management features, such as real-time log monitoring, log retention policies, and alerting capabilities.
+
+Using this integration, you can configure your Dagster jobs to log directly to AWS CloudWatch, making it easier to track and debug your workflows. This is particularly useful for production environments where centralized logging is essential for maintaining observability and operational efficiency.
+
+### Installation
+
+```bash
+pip install dagster-aws
+```
+
+### Examples
+
+
+
+### About AWS CloudWatch
+
+AWS CloudWatch is a monitoring and observability service provided by Amazon Web Services (AWS). It allows you to collect, access, and analyze performance and operational data from a variety of AWS resources, applications, and services. With AWS CloudWatch, you can set up alarms, visualize logs and metrics, and gain insights into your infrastructure and applications to ensure they're running smoothly.
+
+AWS CloudWatch provides features such as:
+
+- Real-time monitoring: Track the performance of your applications and infrastructure in real-time.
+- Log management: Collect, store, and analyze log data from various sources.
+- Alarms and notifications: Set up alarms to automatically notify you of potential issues.
+- Dashboards: Create custom dashboards to visualize metrics and logs.
+- Integration with other AWS services: Seamlessly integrate with other AWS services for a comprehensive monitoring solution.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/ecr.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/ecr.md
new file mode 100644
index 0000000000000..ce017ef28ba93
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/ecr.md
@@ -0,0 +1,37 @@
+---
+layout: Integration
+status: published
+name: ECR
+title: Dagster & AWS ECR
+sidebar_label: ECR
+excerpt: This integration allows you to connect to AWS Elastic Container Registry (ECR), enabling you to manage your container images more effectively in your Dagster pipelines.
+date: 2024-06-21
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-aws
+docslink:
+partnerlink: https://aws.amazon.com/
+categories:
+ - Other
+enabledBy:
+enables:
+tags: [dagster-supported]
+sidebar_custom_props:
+ logo: images/integrations/aws-ecr.svg
+---
+
+This integration allows you to connect to AWS Elastic Container Registry (ECR). It provides resources to interact with AWS ECR, enabling you to manage your container images.
+
+Using this integration, you can seamlessly integrate AWS ECR into your Dagster pipelines, making it easier to manage and deploy containerized applications.
+
+### Installation
+
+```bash
+pip install dagster-aws
+```
+
+### Examples
+
+
+
+### About AWS ECR
+
+AWS Elastic Container Registry (ECR) is a fully managed Docker container registry that makes it easy for developers to store, manage, and deploy Docker container images. AWS ECR is integrated with Amazon Elastic Kubernetes Service (EKS), simplifying your development to production workflow. With ECR, you can securely store and manage your container images and easily integrate with your existing CI/CD pipelines. AWS ECR provides high availability and scalability, ensuring that your container images are always available when you need them.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/emr.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/emr.md
new file mode 100644
index 0000000000000..6ce46c198e8a7
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/emr.md
@@ -0,0 +1,42 @@
+---
+layout: Integration
+status: published
+name: EMR
+title: Dagster & AWS EMR
+sidebar_label: EMR
+excerpt: The AWS EMR integration allows you to seamlessly integrate AWS EMR into your Dagster pipelines for petabyte-scale data processing using open source tools like Apache Spark, Hive, Presto, and more.
+date: 2024-06-21
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-aws
+docslink:
+partnerlink: https://aws.amazon.com/
+categories:
+ - Compute
+enabledBy:
+enables:
+tags: [dagster-supported, compute]
+sidebar_custom_props:
+ logo: images/integrations/aws-emr.svg
+---
+
+The `dagster-aws` integration provides ways orchestrating data pipelines that leverage AWS services, including AWS EMR (Elastic MapReduce). This integration allows you to run and scale big data workloads using open source tools such as Apache Spark, Hive, Presto, and more.
+
+Using this integration, you can:
+
+- Seamlessly integrate AWS EMR into your Dagster pipelines.
+- Utilize EMR for petabyte-scale data processing.
+- Easily manage and monitor EMR clusters and jobs from within Dagster.
+- Leverage Dagster's orchestration capabilities to handle complex data workflows involving EMR.
+
+### Installation
+
+```bash
+pip install dagster-aws
+```
+
+### Examples
+
+
+
+### About AWS EMR
+
+**AWS EMR** (Elastic MapReduce) is a cloud big data platform for processing vast amounts of data using open source tools such as Apache Spark, Apache Hive, Apache HBase, Apache Flink, Apache Hudi, and Presto. It simplifies running big data frameworks, allowing you to process and analyze large datasets quickly and cost-effectively. AWS EMR provides the scalability, flexibility, and reliability needed to handle complex data processing tasks, making it an ideal choice for data engineers and scientists.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/glue.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/glue.md
new file mode 100644
index 0000000000000..1310c7c56b615
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/glue.md
@@ -0,0 +1,36 @@
+---
+layout: Integration
+status: published
+name: Glue
+title: Dagster & AWS Glue
+sidebar_label: Glue
+excerpt: The AWS Glue integration enables you to initiate AWS Glue jobs directly from Dagster, seamlessly pass parameters to your code, and stream logs and structured messages back into Dagster.
+date: 2024-08-20
+apireflink: https://docs.dagster.io/integrations/libraries/aws/glue
+docslink:
+partnerlink: https://aws.amazon.com/
+logo: /integrations/aws-glue.svg
+categories:
+ - Compute
+enabledBy:
+enables:
+tags: [dagster-supported, compute]
+sidebar_custom_props:
+ logo: images/integrations/aws-glue.svg
+---
+
+The `dagster-aws` integration library provides the `PipesGlueClient` resource, enabling you to launch AWS Glue jobs directly from Dagster assets and ops. This integration allows you to pass parameters to Glue code while Dagster receives real-time events, such as logs, asset checks, and asset materializations, from the initiated jobs. With minimal code changes required on the job side, this integration is both efficient and easy to implement.
+
+### Installation
+
+```bash
+pip install dagster-aws
+```
+
+### Examples
+
+
+
+### About AWS Glue
+
+**AWS Glue** is a fully managed cloud service designed to simplify and automate the process of discovering, preparing, and integrating data for analytics, machine learning, and application development. It supports a wide range of data sources and formats, offering seamless integration with other AWS services. AWS Glue provides the tools to create, run, and manage ETL (Extract, Transform, Load) jobs, making it easier to handle complex data workflows. Its serverless architecture allows for scalability and flexibility, making it a preferred choice for data engineers and analysts who need to process and prepare data efficiently.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/index.md
new file mode 100644
index 0000000000000..46c2458c73ef3
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/index.md
@@ -0,0 +1,9 @@
+---
+title: AWS
+sidebar_custom_props:
+ logo: images/integrations/aws.svg
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/lambda.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/lambda.md
new file mode 100644
index 0000000000000..63fa4445bda46
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/lambda.md
@@ -0,0 +1,36 @@
+---
+layout: Integration
+status: published
+name: Lambda
+title: Dagster & AWS Lambda
+sidebar_label: Lambda
+excerpt: Using the AWS Lambda integration with Dagster, you can leverage serverless functions to execute external code in your pipelines.
+date: 2024-06-21
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-aws
+docslink:
+partnerlink: https://aws.amazon.com/
+logo: /integrations/aws-lambda.svg
+categories:
+ - Compute
+enabledBy:
+enables:
+tags: [dagster-supported, compute]
+sidebar_custom_props:
+ logo: images/integrations/aws-lambda.svg
+---
+
+Using this integration, you can leverage AWS Lambda to execute external code as part of your Dagster pipelines. This is particularly useful for running serverless functions that can scale automatically and handle various workloads without the need for managing infrastructure. The `PipesLambdaClient` class allows you to invoke AWS Lambda functions and stream logs and structured metadata back to Dagster's UI and tools.
+
+### Installation
+
+```bash
+pip install dagster-aws
+```
+
+### Examples
+
+
+
+### About AWS Lambda
+
+**AWS Lambda** is a serverless compute service provided by Amazon Web Services (AWS). It allows you to run code without provisioning or managing servers. AWS Lambda automatically scales your application by running code in response to each trigger, such as changes to data in an Amazon S3 bucket or an update to a DynamoDB table. You can use AWS Lambda to extend other AWS services with custom logic, or create your own backend services that operate at AWS scale, performance, and security.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/redshift.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/redshift.md
new file mode 100644
index 0000000000000..fee20442e8847
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/redshift.md
@@ -0,0 +1,36 @@
+---
+layout: Integration
+status: published
+name: Redshift
+title: Dagster & AWS Redshift
+sidebar_label: Redshift
+excerpt: 'Using this integration, you can seamlessly integrate AWS Redshift into your Dagster workflows, leveraging Redshifts data warehousing capabilities for your data pipelines.'
+date: 2024-06-21
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-aws
+docslink:
+partnerlink: https://aws.amazon.com/
+logo: /integrations/aws-redshift.svg
+categories:
+ - Storage
+enabledBy:
+enables:
+tags: [dagster-supported, storage]
+sidebar_custom_props:
+ logo: images/integrations/aws-redshift.svg
+---
+
+Using this integration, you can connect to an AWS Redshift cluster and issue queries against it directly from your Dagster assets. This allows you to seamlessly integrate Redshift into your data pipelines, leveraging the power of Redshift's data warehousing capabilities within your Dagster workflows.
+
+### Installation
+
+```bash
+pip install dagster-aws
+```
+
+### Examples
+
+
+
+### About AWS Redshift
+
+**AWS Redshift** is a fully managed, petabyte-scale data warehouse service in the cloud. You can start with just a few hundred gigabytes of data and scale to a petabyte or more. This enables you to use your data to acquire new insights for your business and customers. Redshift offers fast query performance using SQL-based tools and business intelligence applications, making it a powerful tool for data warehousing and analytics.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/s3.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/s3.md
new file mode 100644
index 0000000000000..2c359045abbc6
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/s3.md
@@ -0,0 +1,38 @@
+---
+layout: Integration
+status: published
+name: S3
+title: Dagster & AWS S3
+sidebar_label: S3
+excerpt: The AWS S3 integration allows data engineers to easily read and write objects to the durable AWS S3 storage, enabling engineers to have a resilient storage layer when constructing their pipelines.
+date: 2024-06-21
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-aws
+docslink:
+partnerlink: https://aws.amazon.com/
+logo: /integrations/aws-s3.svg
+categories:
+ - Storage
+enabledBy:
+enables:
+tags: [dagster-supported, storage]
+sidebar_custom_props:
+ logo: images/integrations/aws-s3.svg
+---
+
+The AWS S3 integration allows data engineers to easily read, and write objects to the durable AWS S3 storage -- enabling engineers to a resilient storage layer when constructing their pipelines.
+
+### Installation
+
+```bash
+pip install dagster-aws
+```
+
+### Examples
+
+Here is an example of how to use the `S3Resource` in a Dagster job to interact with AWS S3:
+
+
+
+### About AWS S3
+
+**AWS S3** is an object storage service that offers industry-leading scalability, data availability, security, and performance. This means customers of all sizes and industries can use it to store and protect any amount of data for a range of use cases, such as data lakes, websites, mobile applications, backup and restore, archive, enterprise applications, IoT devices, and big data analytics. Amazon S3 provides easy-to-use management features so you can organize your data and configure finely tuned access controls to meet your specific business, organizational, and compliance requirements.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/secretsmanager.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/secretsmanager.md
new file mode 100644
index 0000000000000..6d98f9fa42c7c
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/secretsmanager.md
@@ -0,0 +1,35 @@
+---
+layout: Integration
+status: published
+name: Secrets Manager
+title: Dagster & AWS Secrets Manager
+sidebar_label: Secrets Manager
+excerpt: This integration allows you to manage, retrieve, and rotate credentials, API keys, and other secrets using AWS Secrets Manager.
+date: 2024-06-21
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-aws
+docslink:
+partnerlink: https://aws.amazon.com/
+categories:
+ - Other
+enabledBy:
+enables:
+tags: [dagster-supported]
+sidebar_custom_props:
+ logo: images/integrations/aws-secretsmanager.svg
+---
+
+This integration allows you to manage, retrieve, and rotate credentials, API keys, and other secrets using [AWS Secrets Manager](https://aws.amazon.com/secrets-manager/).
+
+### Installation
+
+```bash
+pip install dagster-aws
+```
+
+### Examples
+
+
+
+### About AWS Secrets Manager
+
+**AWS Secrets Manager** helps you protect access to your applications, services, and IT resources without the upfront cost and complexity of managing your own hardware security module infrastructure. With Secrets Manager, you can rotate, manage, and retrieve database credentials, API keys, and other secrets throughout their lifecycle. Users and applications retrieve secrets with a call to Secrets Manager APIs, eliminating the need to hardcode sensitive information in plain text.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/ssm.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/ssm.md
new file mode 100644
index 0000000000000..f272544567800
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/aws/ssm.md
@@ -0,0 +1,37 @@
+---
+layout: Integration
+status: published
+name: Systems Parameter Store
+title: Dagster & AWS Systems Parameter Store
+sidebar_label: Systems Parameter Store
+excerpt: The Dagster AWS Systems Manager (SSM) Parameter Store integration allows you to manage and retrieve parameters stored in AWS SSM Parameter Store directly within your Dagster pipelines.
+date: 2024-06-21
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-aws
+docslink:
+partnerlink: https://aws.amazon.com/
+categories:
+ - Other
+enabledBy:
+enables:
+tags: [dagster-supported]
+sidebar_custom_props:
+ logo: images/integrations/aws-ssm.svg
+---
+
+The Dagster AWS Systems Manager (SSM) Parameter Store integration allows you to manage and retrieve parameters stored in AWS SSM Parameter Store directly within your Dagster pipelines. This integration provides resources to fetch parameters by name, tags, or paths, and optionally set them as environment variables for your operations.
+
+### Installation
+
+```bash
+pip install dagster-aws
+```
+
+### Examples
+
+
+
+### About AWS Systems Parameter Store
+
+**AWS Systems Manager Parameter Store** is a secure storage service for configuration data management and secrets management. It allows you to store data such as passwords, database strings, and license codes as parameter values. You can then reference these parameters in your applications or scripts, ensuring that sensitive information isn't hard-coded or exposed in your codebase.
+
+AWS Systems Manager Parameter Store integrates with AWS Identity and Access Management (IAM) to control access to parameters, and it supports encryption using AWS Key Management Service (KMS) to protect sensitive data. This service is essential for maintaining secure and manageable configurations across your AWS environment.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/azure-adls2.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/azure-adls2.md
new file mode 100644
index 0000000000000..0b3b095fac1b8
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/azure-adls2.md
@@ -0,0 +1,37 @@
+---
+layout: Integration
+status: published
+title: Dagster & Azure Data Lake Storage Gen 2
+sidebar_label: Azure Data Lake Storage Gen 2
+excerpt: Get utilities for ADLS2 and Blob Storage.
+date: 2022-11-07
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-azure
+docslink:
+partnerlink: https://azure.microsoft.com/
+logo: /integrations/Azure.svg
+categories:
+ - Storage
+enabledBy:
+enables:
+tags: [dagster-supported, storage]
+sidebar_custom_props:
+ logo: images/integrations/azure.svg
+---
+
+Dagster helps you use Azure Storage Accounts as part of your data pipeline. Azure Data Lake Storage Gen 2 (ADLS2) is our primary focus but we also provide utilities for Azure Blob Storage.
+
+### Installation
+
+```bash
+pip install dagster-azure
+```
+
+### Example
+
+
+
+In this updated code, we use `ADLS2Resource` directly instead of `adls2_resource`. The configuration is passed to `ADLS2Resource` during its instantiation.
+
+### About Azure Data Lake Storage Gen 2 (ADLS2)
+
+**Azure Data Lake Storage Gen 2 (ADLS2)** is a set of capabilities dedicated to big data analytics, built on Azure Blob Storage. ADLS2 combines the scalability, cost-effectiveness, security, and rich capabilities of Azure Blob Storage with a high-performance file system that's built for analytics and is compatible with the Hadoop Distributed File System (HDFS). This makes it an ideal choice for data lakes and big data analytics.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/census.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/census.md
new file mode 100644
index 0000000000000..2c2826c0c6219
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/census.md
@@ -0,0 +1,37 @@
+---
+layout: Integration
+status: published
+name: Census
+title: Dagster & Census
+sidebar_label: Census
+excerpt: Trigger Census synchs from within your Dagster pipelines.
+date: 2022-11-07
+apireflink: http://docs.dagster.io/api/python-api/libraries/dagster-census
+partnerlink: https://www.getcensus.com/
+communityIntegration: true
+logo: /integrations/Census.svg
+categories:
+ - ETL
+enabledBy:
+enables:
+tags: [community-supported, etl]
+sidebar_custom_props:
+ logo: images/integrations/census.svg
+ community: true
+---
+
+With the `dagster-census` integration you can execute a Census sync and poll until that sync completes, raising an error if it's unsuccessful.
+
+### Installation
+
+```bash
+pip install dagster-census
+```
+
+### Example
+
+
+
+### About Census
+
+**Census** syncs data from your cloud warehouse to the SaaS tools your organization uses. It allows everyone in your organization to take action with good data, no custom scripts or API integrations required.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/chroma.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/chroma.md
new file mode 100644
index 0000000000000..c28e0736a9302
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/chroma.md
@@ -0,0 +1,33 @@
+---
+layout: Integration
+status: published
+name: Chroma
+title: Dagster & Chroma
+sidebar_label: Chroma
+excerpt: 'Integrate Chroma vector database capabilities into your AI pipelines powered by Dagster.'
+partnerlink: https://docs.trychroma.com/
+logo: /integrations/chroma.svg
+categories:
+ - Storage
+enabledBy:
+enables:
+tags: [dagster-supported, storage]
+sidebar_custom_props:
+ logo: images/integrations/chroma.png
+---
+
+The `dagster-chroma` library allows you to easily interact with Chroma's vector database capabilities to build AI-powered data pipelines in Dagster. You can perform vector similarity searches, manage schemas, and handle data operations directly from your Dagster assets.
+
+### Installation
+
+```bash
+pip install dagster dagster-chroma
+```
+
+### Example
+
+
+
+### About Chroma
+
+**Chroma** is the open-source AI application database. Chroma makes it easy to build LLM apps by making knowledge, facts, and skills pluggable for LLMs. It provides a simple API for storing and querying embeddings, documents, and metadata. Chroma can be used to build semantic search, question answering, and other AI-powered applications. The database can run embedded in your application or as a separate service.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/cube.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/cube.md
new file mode 100644
index 0000000000000..e5a7c6be0e9f2
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/cube.md
@@ -0,0 +1,37 @@
+---
+layout: Integration
+status: published
+name: Cube
+title: Dagster & Cube
+sidebar_label: Cube
+excerpt: 'Push changes from upstream data sources to Cubes semantic layer.'
+date: 2023-08-30
+apireflink: https://cube.dev/docs/orchestration-api/dagster
+partnerlink: https://cube.dev/
+communityIntegration: true
+logo: /integrations/cube.svg
+categories:
+ - Other
+enabledBy:
+enables:
+tags: [community-supported]
+sidebar_custom_props:
+ logo: images/integrations/cube.svg
+ community: true
+---
+
+With the `dagster_cube` integration you can setup Cube and Dagster to work together so that Dagster can push changes from upstream data sources to Cube using its integration API.
+
+### Installation
+
+```bash
+pip install dagster_cube
+```
+
+### Example
+
+
+
+### About Cube
+
+**Cube.js** is the semantic layer for building data applications. It helps data engineers and application developers access data from modern data stores, organize it into consistent definitions, and deliver it to every application.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/databricks.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/databricks.md
new file mode 100644
index 0000000000000..6135b49c79857
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/databricks.md
@@ -0,0 +1,38 @@
+---
+layout: Integration
+status: published
+name: Databricks
+title: Dagster & Databricks
+sidebar_label: Databricks
+excerpt: The Databricks integration enables you to initiate Databricks jobs directly from Dagster, seamlessly pass parameters to your code, and stream logs and structured messages back into Dagster.
+date: 2024-08-20
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-databricks
+docslink: https://docs.dagster.io/integrations/libraries/databricks
+partnerlink: https://databricks.com/
+logo: /integrations/databricks.svg
+categories:
+ - Compute
+enabledBy:
+enables:
+tags: [dagster-supported, compute]
+sidebar_custom_props:
+ logo: images/integrations/databricks.svg
+---
+
+The `dagster-databricks` integration library provides the `PipesDatabricksClient` resource, enabling you to launch Databricks jobs directly from Dagster assets and ops. This integration allows you to pass parameters to Databricks code while Dagster receives real-time events, such as logs, asset checks, and asset materializations, from the initiated jobs. With minimal code changes required on the job side, this integration is both efficient and easy to implement.
+
+### Installation
+
+```bash
+pip install dagster-databricks
+```
+
+### Example
+
+
+
+
+
+### About Databricks
+
+**Databricks** is a unified data analytics platform that simplifies and accelerates the process of building big data and AI solutions. It integrates seamlessly with Apache Spark and offers support for various data sources and formats. Databricks provides powerful tools to create, run, and manage data pipelines, making it easier to handle complex data engineering tasks. Its collaborative and scalable environment is ideal for data engineers, scientists, and analysts who need to process and analyze large datasets efficiently.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/datadog.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/datadog.md
new file mode 100644
index 0000000000000..126a564e2ec3c
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/datadog.md
@@ -0,0 +1,36 @@
+---
+layout: Integration
+status: published
+name: Datadog
+title: Dagster & Datadog
+sidebar_label: Datadog
+excerpt: Publish metrics to Datadog from within Dagster ops and entralize your monitoring metrics.
+date: 2022-11-07
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-datadog
+docslink:
+partnerlink: https://www.datadoghq.com/
+logo: /integrations/Datadog.svg
+categories:
+ - Monitoring
+enabledBy:
+enables:
+tags: [dagster-supported, monitoring]
+sidebar_custom_props:
+ logo: images/integrations/datadog.svg
+---
+
+While Dagster provides comprehensive monitoring and observability of the pipelines it orchestrates, many teams look to centralize all their monitoring across apps, processes and infrastructure using Datadog's 'Cloud Monitoring as a Service'. The `dagster-datadog` integration allows you to publish metrics to Datadog from within Dagster ops.
+
+### Installation
+
+```bash
+pip install dagster-datadog
+```
+
+### Example
+
+
+
+### About Datadog
+
+**Datadog** is an observability service for cloud-scale applications, providing monitoring of servers, databases, tools, and services, through a SaaS-based data analytics platform.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/dbt-cloud.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/dbt-cloud.md
new file mode 100644
index 0000000000000..023cf39f6969f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/dbt-cloud.md
@@ -0,0 +1,37 @@
+---
+layout: Integration
+status: published
+name: dbt Cloud
+title: Dagster & dbt Cloud
+sidebar_label: dbt Cloud
+excerpt: Run dbt Cloud™ jobs as part of your data pipeline.
+date: 2022-11-07
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-dbt#assets-dbt-cloud
+docslink: https://docs.dagster.io/integration/libraries/dbt/dbt_cloud
+partnerlink:
+categories:
+ - ETL
+enabledBy:
+enables:
+tags: [dagster-supported, etl]
+sidebar_custom_props:
+ logo: images/integrations/dbt/dbt.svg
+---
+
+Dagster allows you to run dbt Cloud jobs alongside other technologies. You can schedule them to run as a step in a larger pipeline and manage them as a data asset.
+
+### Installation
+
+```bash
+pip install dagster-dbt
+```
+
+### Example
+
+
+
+### About dbt Cloud
+
+**dbt Cloud** is a hosted service for running dbt jobs. It helps data analysts and engineers productionize dbt deployments. Beyond dbt open source, dbt Cloud provides scheduling , CI/CD, serving documentation, and monitoring & alerting.
+
+If you're currently using dbt Cloud™, you can also use Dagster to run `dbt-core` in its place. You can read more about [how to do that here](https://dagster.io/blog/migrate-off-dbt-cloud).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/dbt-core.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/dbt-core.md
new file mode 100644
index 0000000000000..57fcf0510a2ed
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/dbt-core.md
@@ -0,0 +1,47 @@
+---
+layout: Integration
+status: published
+name: dbt
+title: Dagster & dbt
+sidebar_label: dbt
+excerpt: Put your dbt transformations to work, directly from within Dagster.
+date: 2022-11-07
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-dbt
+docslink: https://docs.dagster.io/api/python-api/libraries/dagster-dbt#dbt-core
+partnerlink: https://www.getdbt.com/
+categories:
+ - ETL
+enabledBy:
+enables:
+tags: [dagster-supported, etl]
+sidebar_custom_props:
+ logo: images/integrations/dbt/dbt.svg
+---
+
+Dagster orchestrates dbt alongside other technologies, so you can schedule dbt with Spark, Python, etc. in a single data pipeline.
+
+Dagster assets understand dbt at the level of individual dbt models. This means that you can:
+
+- Use Dagster's UI or APIs to run subsets of your dbt models, seeds, and snapshots.
+- Track failures, logs, and run history for individual dbt models, seeds, and snapshots.
+- Define dependencies between individual dbt models and other data assets. For example, put dbt models after the Fivetran-ingested table that they read from, or put a machine learning after the dbt models that it's trained from.
+
+### Installation
+
+```bash
+pip install dagster-dbt
+```
+
+### Example
+
+
+
+### About dbt
+
+**dbt** is a SQL-first transformation workflow that lets teams quickly and collaboratively deploy analytics code following software engineering best practices like modularity, portability, CI/CD, and documentation.
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/index.md
new file mode 100644
index 0000000000000..39bf04e4a4ece
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/index.md
@@ -0,0 +1,15 @@
+---
+layout: Integration
+status: published
+name: dbt
+title: Dagster & dbt
+sidebar_label: dbt
+sidebar_custom_props:
+ logo: images/integrations/dbt/dbt.svg
+ categories:
+ - etl
+---
+
+import DocCardList from '@theme/DocCardList';
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/reference.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/reference.md
new file mode 100644
index 0000000000000..140ae040fc549
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/reference.md
@@ -0,0 +1,573 @@
+---
+title: "dagster-dbt integration reference"
+description: Dagster can orchestrate dbt alongside other technologies.
+---
+
+:::note
+
+Using dbt Cloud? Check out the [dbt Cloud with Dagster guide](dbt-cloud).
+
+:::
+
+This reference provides a high-level look at working with dbt models through Dagster's [software-defined assets](/guides/build/assets/) framework using the [`dagster-dbt` integration library](/api/python-api/libraries/dagster-dbt).
+
+For a step-by-step implementation walkthrough, refer to the [Using dbt with Dagster asset definitions tutorial](using-dbt-with-dagster).
+
+## Relevant APIs
+
+| Name | Description |
+| ---------------------------------------------------------------------------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------------------- |
+| [`dagster-dbt project scaffold`](/api/python-api/libraries/dagster-dbt#scaffold) | A CLI command to initialize a new Dagster project for an existing dbt project. |
+| | A decorator used to define Dagster assets for dbt models defined in a dbt manifest. |
+| | A class that defines a Dagster resource used to execute dbt CLI commands. |
+| | A class that defines the representation of an invoked dbt command. |
+| | A class that defines the representation of a dbt project and related settings that assist with managing dependencies and `manifest.json` preparation. |
+| | A class that can be overridden to customize how Dagster asset metadata is derived from a dbt manifest. |
+| | A class with settings to enable Dagster features for a dbt project. |
+| | A class that defines a selection of assets from a dbt manifest and a dbt selection string. |
+| | A helper method that builds a from a dbt manifest and dbt selection string. |
+| | A helper method that builds a from a dbt manifest, dbt selection string, and cron string. |
+| | A helper method that retrieves the for a dbt model. |
+| | A helper method that retrieves the for a dbt source with a singular table. |
+| | A helper method that retrieves the for a dbt source with multiple tables. |
+
+## dbt models and Dagster asset definitions
+
+Dagster’s [asset definitions](/guides/build/assets/) bear several similarities to dbt models. An asset definition contains an asset key, a set of upstream asset keys, and an operation that is responsible for computing the asset from its upstream dependencies. Models defined in a dbt project can be interpreted as Dagster asset definitions:
+
+- The asset key for a dbt model is (by default) the name of the model.
+- The upstream dependencies of a dbt model are defined with `ref` or `source` calls within the model's definition.
+- The computation required to compute the asset from its upstream dependencies is the SQL within the model's definition.
+
+These similarities make it natural to interact with dbt models as asset definitions. Let’s take a look at a dbt model and an asset definition, in code:
+
+![Comparison of a dbt model and Dagster asset in code](/images/integrations/dbt/using-dbt-with-dagster/asset-dbt-model-comparison.png)
+
+Here's what's happening in this example:
+
+- The first code block is a **dbt model**
+ - As dbt models are named using file names, this model is named `orders`
+ - The data for this model comes from a dependency named `raw_orders`
+- The second code block is a **Dagster asset**
+ - The asset key corresponds to the name of the dbt model, `orders`
+ - `raw_orders` is provided as an argument to the asset, defining it as a dependency
+
+## Scaffolding a Dagster project from a dbt project
+
+:::note
+
+Check out [part two of the dbt & Dagster tutorial](using-dbt-with-dagster/load-dbt-models) to see this concept in context.
+
+:::
+
+You can create a Dagster project that wraps your dbt project by using the [`dagster-dbt project scaffold`](/api/python-api/libraries/dagster-dbt#scaffold) command line interface.
+
+```shell
+dagster-dbt project scaffold --project-name project_dagster --dbt-project-dir path/to/dbt/project
+```
+
+This creates a directory called `project_dagster/` inside the current directory. The `project_dagster/` directory contains a set of files that define a Dagster project that loads the dbt project at the path defined by `--dbt-project-dir`. The path to the dbt project must contain a `dbt_project.yml`.
+
+## Loading dbt models from a dbt project
+
+:::note
+
+Check out [part two of the dbt & Dagster tutorial](using-dbt-with-dagster/load-dbt-models) to see this concept in context.
+
+:::
+
+The `dagster-dbt` library offers to define Dagster assets for dbt models. It requires a [dbt manifest](https://docs.getdbt.com/reference/artifacts/manifest-json), or `manifest.json`, to be created from your dbt project to parse your dbt project's representation.
+
+The manifest can be created in two ways:
+
+1. **At run time**: A dbt manifest is generated when your Dagster definitions are loaded, or
+2. **At build time**: A dbt manifest is generated before loading your Dagster definitions and is included as part of your Python package.
+
+When deploying your Dagster project to production, **we recommend generating the manifest at build time** to avoid the overhead of recompiling your dbt project every time your Dagster code is executed. A `manifest.json` should be precompiled and included in the Python package for your Dagster code.
+
+The easiest way to handle the creation of your manifest file is to use .
+
+In the Dagster project created by the [`dagster-dbt project scaffold`](/api/python-api/libraries/dagster-dbt#scaffold) command, the creation of your manifest is handled at run time during development:
+
+
+
+The manifest path can then be accessed with `my_dbt_project.manifest_path`.
+
+When developing locally, you can run the following command to generate the manifest at run time for your dbt and Dagster project:
+
+```shell
+dagster dev
+```
+
+In production, a precompiled manifest should be used. Using , the manifest can be created at build time by running the [`dagster-dbt project prepare-and-package`](/api/python-api/libraries/dagster-dbt#prepare-and-package) command in your CI/CD workflow. For more information, see the [Deploying a Dagster project with a dbt project](#deploying-a-dagster-project-with-a-dbt-project) section.
+
+## Deploying a Dagster project with a dbt project
+
+:::note
+
+**Got questions about our recommendations or something to add?**
+
+Join our [GitHub discussion](https://github.com/dagster-io/dagster/discussions/13767) to share how you deploy your Dagster code with your dbt project.
+
+:::
+
+When deploying your Dagster project to production, your dbt project must be present alongside your Dagster project so that dbt commands can be executed. As a result, we recommend that you set up your continuous integration and continuous deployment (CI/CD) workflows to package the dbt project with your Dagster project.
+
+### Deploying a dbt project from a separate git repository
+
+If you are managing your Dagster project in a separate git repository from your dbt project, you should include the following steps in your CI/CD workflows.
+
+In your CI/CD workflows for your Dagster project:
+
+1. Include any secrets that are required by your dbt project in your CI/CD environment.
+2. Clone the dbt project repository as a subdirectory of your Dagster project.
+3. Run `dagster-dbt project prepare-and-package --file path/to/project.py` to
+ - Build your dbt project's dependencies,
+ - Create a dbt manifest for your Dagster project, and
+ - Package your dbt project
+
+In the CI/CD workflows for your dbt project, set up a dispatch action to trigger a deployment of your Dagster project when your dbt project changes.
+
+### Deploying a dbt project from a monorepo
+
+:::note
+
+With [Dagster+](https://dagster.io/cloud), we streamline this option. As part of our Dagster+ onboarding for dbt users, we can automatically create a Dagster project in an existing dbt project repository.
+
+:::
+
+If you are managing your Dagster project in the same git repository as your dbt project, you should include the following steps in your CI/CD workflows.
+
+In your CI/CD workflows for your Dagster and dbt project:
+
+1. Include any secrets that are required by your dbt project in your CI/CD environment.
+2. Run `dagster-dbt project prepare-and-package --file path/to/project.py` to
+ - Build your dbt project's dependencies,
+ - Create a dbt manifest for your Dagster project, and
+ - Package your dbt project
+
+## Leveraging dbt defer with branch deployments
+
+:::note
+
+This feature requires the `DAGSTER_BUILD_STATEDIR` environment variable to be set in your CI/CD. Learn more about required environment variables in CI/CD for Dagster+ [here](/dagster-plus/features/ci-cd/configuring-ci-cd).
+
+:::
+
+It is possible to leverage [dbt defer](https://docs.getdbt.com/reference/node-selection/defer) by passing a `state_path` to . This is useful for testing recent changes made in development against the state of the dbt project in production. Using `dbt defer`, you can run a subset of models or tests, those that have been changed between development and production, without having to build their upstream parents first.
+
+In practice, this is most useful when combined with branch deployments in Dagster+, to test changes made in your branches. This can be done by updating your CI/CD files and your Dagster code.
+
+First, let's take a look at your CI/CD files. You might have one or two CI/CD files to manage your production and branch deployments. In these files, find the steps related to your dbt project - refer to the [Deploying a Dagster project with a dbt project](#deploying-a-dagster-project-with-a-dbt-project) section for more information.
+
+Once your dbt steps are located, add the following step to manage the state of your dbt project.
+
+```bash
+dagster-cloud ci dagster-dbt project manage-state --file path/to/project.py
+```
+
+The `dagster-cloud ci dagster-dbt project manage-state` CLI command fetches the `manifest.json` file from your production branch and saves it to a state directory, in order to power the `dbt defer` command.
+
+In practice, this command fetches the `manifest.json` file from your production branch and add it to the state directory set to the `state_path` of the DbtProject found in `path/to/project.py`. The production `manifest.json` file can then be used as the deferred dbt artifacts.
+
+Now that your CI/CD files are updated to manage the state of your dbt project using the dagster-cloud CLI, we need to update the Dagster code to pass a state directory to the DbtProject.
+
+Update your Dagster code to pass a `state_path` to your `DbtProject` object. Note that value passed to `state_path` must be a path, relative to the dbt project directory, to a state directory of dbt artifacts. In the code below, we set the `state_path` to 'state/'. If this directory does not exist in your project structure, it will be created by Dagster.
+
+Also, update the dbt command in your `@dbt_assets` definition to pass the defer args using `get_defer_args`.
+
+
+
+## Using config with `@dbt_assets`
+
+{/* Similar to Dagster software-defined assets, `@dbt_assets` can use a config system to enable [run configuration](/concepts/configuration/config-schema). This allows to provide parameters to jobs at the time they're executed. */}
+
+Similar to Dagster software-defined assets, `@dbt_assets` can use a config system to enable [run configuration](/guides/operate/configuration/run-configuration). This allows to provide parameters to jobs at the time they're executed.
+
+In the context of dbt, this can be useful if you want to run commands or flags for specific use cases. For instance, you may want to add [the --full-refresh flag](https://docs.getdbt.com/reference/resource-configs/full_refresh) to your dbt commands in some cases. Using a config system, the `@dbt_assets` object can be easily modified to support this use case.
+
+
+
+Now that the `@dbt_assets` object is updated, the run configuration can be passed to a job.
+
+
+
+In the example above, the job is configured to use the `--full-refresh` flag with the dbt build command when materializing the assets.
+
+## Scheduling dbt jobs
+
+Once you have your dbt assets, you can define a job to materialize a selection of these assets on a schedule.
+
+### Scheduling jobs that contain only dbt assets
+
+In this example, we use the function to create a job, `daily_dbt_models`, as well as a schedule which will execute this job once a day. We define the set of models we'd like to execute using [dbt's selection syntax](https://docs.getdbt.com/reference/node-selection/syntax#how-does-selection-work), in this case selecting only the models with the tag `daily`.
+
+
+
+
+### Scheduling jobs that contain dbt assets and non-dbt assets
+
+In many cases, it's useful to be able to schedule dbt assets alongside non-dbt assets. In this example, we build an of dbt assets using , then select all assets (dbt-related or not) which are downstream of these dbt models. From there, we create a job that targets that selection of assets and schedule it to run daily.
+
+
+
+Refer to the [Schedule documentation](/guides/automate/schedules/) for more info on running jobs on a schedule.
+
+## Understanding asset definition attributes
+
+In Dagster, each asset definition has attributes. Dagster automatically generates these attributes for each asset definition loaded from the dbt project. These attributes can optionally be overridden by the user.
+
+- [Customizing asset keys](#customizing-asset-keys)
+- [Customizing group names](#customizing-group-names)
+- [Customizing owners](#customizing-owners)
+- [Customizing descriptions](#customizing-descriptions)
+- [Customizing metadata](#customizing-metadata)
+- [Customizing tags](#customizing-tags)
+- [Customizing automation conditions](#customizing-automation-conditions)
+
+### Customizing asset keys
+
+For dbt models, seeds, and snapshots, the default asset key will be the configured schema for that node, concatenated with the name of the node.
+
+| dbt node type | Schema | Model name | Resulting asset key |
+| --------------------- | --------- | ------------ | ------------------- |
+| model, seed, snapshot | `None` | `MODEL_NAME` | `MODEL_NAME` |
+| | `SCHEMA` | `MODEL_NAME` | `SCHEMA/MODEL_NAME` |
+| | `None` | my_model | some_model |
+| | marketing | my_model | marketing/my_model |
+
+For dbt sources, the default asset key will be the name of the source concatenated with the name of the source table.
+
+| dbt node type | Source name | Table name | Resulting asset key |
+| ------------- | ------------- | ------------ | ------------------------ |
+| source | `SOURCE_NAME` | `TABLE_NAME` | `SOURCE_NAME/TABLE_NAME` |
+| | jaffle_shop | orders | jaffle_shop/orders |
+
+There are two ways to customize the asset keys generated by Dagster for dbt assets:
+
+1. Defining [meta config](https://docs.getdbt.com/reference/resource-configs/meta) on your dbt node, or
+2. Overriding Dagster's asset key generation by implementing a custom .
+
+To override an asset key generated by Dagster for a dbt node, you can define a `meta` key on your dbt node's `.yml` file. The following example overrides the asset key for a source and table as `snowflake/jaffle_shop/orders`:
+
+```yaml
+sources:
+ - name: jaffle_shop
+ tables:
+ - name: orders
+ meta:
+ dagster:
+ asset_key: ["snowflake", "jaffle_shop", "orders"]
+```
+
+Alternatively, to override the asset key generation for all dbt nodes in your dbt project, you can create a custom and implement . The following example adds a `snowflake` prefix to the default generated asset key:
+
+
+
+### Customizing group names
+
+For dbt models, seeds, and snapshots, the default Dagster group name will be the [dbt group](https://docs.getdbt.com/docs/build/groups) defined for that node.
+
+| dbt node type | dbt group name | Resulting Dagster group name |
+| --------------------- | -------------- | ---------------------------- |
+| model, seed, snapshot | `GROUP_NAME` | `GROUP_NAME` |
+| | `None` | `None` |
+| | finance | finance |
+
+There are two ways to customize the group names generated by Dagster for dbt assets:
+
+1. Defining [meta config](https://docs.getdbt.com/reference/resource-configs/meta) on your dbt node, or
+2. Overriding Dagster's group name generation by implementing a custom
+
+To override the group name generated by Dagster for a dbt node, you can define a `meta` key in your dbt project file, on your dbt node's property file, or on the node's in-file config block. The following example overrides the Dagster group name for the following model as `marketing`:
+
+```yaml
+models:
+ - name: customers
+ config:
+ meta:
+ dagster:
+ group: marketing
+```
+
+Alternatively, to override the Dagster group name generation for all dbt nodes in your dbt project, you can create a custom and implement . The following example defines `snowflake` as the group name for all dbt nodes:
+
+
+
+### Customizing owners
+
+For dbt models, seeds, and snapshots, the default Dagster owner will be the email associated with the [dbt group](https://docs.getdbt.com/docs/build/groups) defined for that node.
+
+| dbt node type | dbt group name | dbt group's email | Resulting Dagster owner |
+| --------------------- | -------------- | ------------------- | ----------------------- |
+| model, seed, snapshot | `GROUP_NAME` | `OWNER@DOMAIN.COM` | `OWNER@DOMAIN.COM` |
+| | `GROUP_NAME` | `None` | `None` |
+| | `None` | `None` | `None` |
+| | finance | `owner@company.com` | `owner@company.com` |
+| | finance | `None` | `None` |
+
+There are two ways to customize the asset keys generated by Dagster for dbt assets:
+
+1. Defining [meta config](https://docs.getdbt.com/reference/resource-configs/meta) on your dbt node, or
+2. Overriding Dagster's generation of owners by implementing a custom
+
+To override the owners generated by Dagster for a dbt node, you can define a `meta` key in your dbt project file, on your dbt node's property file, or on the node's in-file config block. The following example overrides the Dagster owners for the following model as `owner@company.com` and `team:data@company.com`:
+
+```yaml
+models:
+ - name: customers
+ config:
+ meta:
+ dagster:
+ owners: ["owner@company.com", "team:data@company.com"]
+```
+
+Alternatively, to override the Dagster generation of owners for all dbt nodes in your dbt project, you can create a custom and implement . The following example defines `owner@company.com` and `team:data@company.com` as the owners for all dbt nodes:
+
+
+
+### Customizing descriptions
+
+For dbt models, seeds, and snapshots, the default Dagster description will be the dbt node's description.
+
+To override the Dagster description for all dbt nodes in your dbt project, you can create a custom and implement . The following example defines the raw SQL of the dbt node as the Dagster description:
+
+
+
+### Customizing metadata
+
+For dbt models, seeds, and snapshots, the default Dagster definition metadata will be the dbt node's declared column schema.
+
+To override the Dagster definition metadata for all dbt nodes in your dbt project, you can create a custom and implement . The following example defines the metadata of the dbt node as the Dagster metadata, using :
+
+
+
+Dagster also supports fetching additional metadata at dbt execution time to attach to asset materializations. For more information, see the [Customizing asset materialization metadata](#customizing-asset-materialization-metadata) section.
+
+#### Attaching code reference metadata
+
+{/* Dagster's dbt integration can automatically attach [code reference](/guides/dagster/code-references) metadata to the SQL files backing your dbt assets. */}
+Dagster's dbt integration can automatically attach [code reference](/guides/build/assets/metadata-and-tags/index.md#source-code) metadata to the SQL files backing your dbt assets. To enable this feature, set the `enable_code_references` parameter to `True` in the passed to your :
+
+
+
+### Customizing tags
+
+:::note
+
+In Dagster, tags are key-value pairs. However, in dbt, tags are strings. To bridge this divide, the dbt tag string is used as the Dagster tag key, and the Dagster tag value is set to the empty string, `""`. Any dbt tags that don't match Dagster's supported tag key format (e.g. they contain unsupported characters) will be ignored by default.
+
+:::
+
+For dbt models, seeds, and snapshots, the default Dagster tags will be the dbt node's configured tags.
+
+Any dbt tags that don't match Dagster's supported tag key format (e.g. they contain unsupported characters) will be ignored.
+
+To override the Dagster tags for all dbt nodes in your dbt project, you can create a custom and implement . The following converts dbt tags of the form `foo=bar` to key/value pairs:
+
+
+
+### Customizing automation conditions
+
+To override the generated for each dbt node in your dbt project, you can create a custom and implement . The following example defines as the condition for all dbt nodes:
+
+
+
+## dbt models, code versions, and "Unsynced"
+
+Note that Dagster allows the optional specification of a [`code_version`](/guides/build/assets/defining-assets#asset-code-versions) for each asset definition, which are used to track changes. The `code_version` for an asset arising from a dbt model is defined automatically as the hash of the SQL defining the DBT model. This allows the asset graph in the UI to use the "Unsynced" status to indicate which dbt models have new SQL since they were last materialized.
+
+## Loading dbt tests as asset checks
+
+:::note
+
+**Asset checks for dbt have been enabled by default, starting in `dagster-dbt` 0.23.0.**
+
+`dbt-core` 1.6 or later is required for full functionality.
+
+:::
+
+Dagster loads your dbt tests as [asset checks](/guides/test/asset-checks).
+
+### Indirect selection
+
+Dagster uses [dbt indirect selection](https://docs.getdbt.com/reference/global-configs/indirect-selection) to select dbt tests. By default, Dagster won't set `DBT_INDIRECT_SELECTION` so that the set of tests selected by Dagster is the same as the selected by dbt. When required, Dagster will override `DBT_INDIRECT_SELECTION` to `empty` in order to explicitly select dbt tests. For example:
+
+- Materializing dbt assets and excluding their asset checks
+- Executing dbt asset checks without materializing their assets
+
+### Singular tests
+
+Dagster will load both generic and singular tests as asset checks. In the event that your singular test depends on multiple dbt models, you can use dbt metadata to specify which Dagster asset it should target. These fields can be filled in as they would be for the dbt [ref function](https://docs.getdbt.com/reference/dbt-jinja-functions/ref). The configuration can be supplied in a [config block](https://docs.getdbt.com/reference/data-test-configs) for the singular test.
+
+```sql
+{{
+ config(
+ meta={
+ 'dagster': {
+ 'ref': {
+ 'name': 'customers',
+ 'package_name': 'my_dbt_assets'
+ 'version': 1,
+ },
+ }
+ }
+ )
+}}
+```
+
+`dbt-core` version 1.6 or later is required for Dagster to read this metadata.
+
+If this metadata isn't provided, Dagster won't ingest the test as an asset check. It will still run the test and emit a events with the test results.
+
+### Disabling asset checks
+
+You can disable modeling your dbt tests as asset checks. The tests will still run and will be emitted as events. To do so you'll need to define a with that have asset checks disabled. The following example disables asset checks when using :
+
+
+
+## Customizing asset materialization metadata
+
+Dagster supports fetching additional metadata at dbt execution time to attach as [materialization metadata](/guides/build/assets/metadata-and-tags/), which is recorded each time your models are rebuilt and displayed in the Dagster UI.
+
+### Fetching row count data
+
+:::note
+
+**Emitting row count data for dbt models is currently an experimental feature.**
+
+To use this feature, you'll need to be on at least `dagster>=0.17.6` and `dagster-dbt>=0.23.6`.
+
+:::
+
+Dagster can automatically fetch [row counts](/guides/build/assets/metadata-and-tags/) for dbt-generated tables and emit them as [materialization metadata](/guides/build/assets/metadata-and-tags/) to be displayed in the Dagster UI.
+
+Row counts are fetched in parallel to the execution of your dbt models. To enable this feature, call on the returned by the `stream()` dbt CLI call:
+
+
+
+Once your dbt models have been materialized, you can view the row count data in the metadata table.
+
+### Fetching column-level metadata
+
+:::note
+
+**Emitting column-level metadata for dbt models is currently an experimental feature.**
+
+To use this feature, you'll need to be on at least `dagster>=1.8.0` and `dagster-dbt>=0.24.0`.
+
+:::
+
+Dagster allows you to emit column-level metadata, like [column schema](/guides/build/assets/metadata-and-tags/) and [column lineage](/guides/build/assets/metadata-and-tags/), as [materialization metadata](/guides/build/assets/metadata-and-tags/).
+
+With this metadata, you can view documentation in Dagster for all columns, not just columns described in your dbt project.
+
+Column-level metadata is fetched in parallel to the execution of your dbt models. To enable this feature, call on the returned by the `stream()` dbt CLI call:
+
+
+
+### Composing metadata fetching methods
+
+Metadata fetching methods such as can be chained with other metadata fetching methods like :
+
+
+
+## Defining dependencies
+
+- [Upstream dependencies](#upstream-dependencies)
+- [Downstream dependencies](#downstream-dependencies)
+
+### Upstream dependencies
+
+#### Defining an asset as an upstream dependency of a dbt model
+
+Dagster allows you to define existing assets as upstream dependencies of dbt models. For example, say you have the following asset with asset key `upstream`:
+
+
+
+Then, in the downstream model, you can select from this source data. This defines a dependency relationship between your upstream asset and dbt model:
+
+```sql
+select *
+ from {{ source("dagster", "upstream") }}
+ where foo=1
+```
+
+#### Defining a dbt source as a Dagster asset
+
+Dagster parses information about assets that are upstream of specific dbt models from the dbt project itself. Whenever a model is downstream of a [dbt source](https://docs.getdbt.com/docs/building-a-dbt-project/using-sources), that source will be parsed as an upstream asset.
+
+For example, if you defined a source in your `sources.yml` file like this:
+
+```yaml
+sources:
+ - name: jaffle_shop
+ tables:
+ - name: orders
+```
+
+and use it in a model:
+
+```sql
+select *
+ from {{ source("jaffle_shop", "orders") }}
+ where foo=1
+```
+
+Then this model has an upstream source with the `jaffle_shop/orders` asset key.
+
+In order to manage this upstream asset with Dagster, you can define it by passing the key into an asset definition via :
+
+
+
+This allows you to change asset keys within your dbt project without having to update the corresponding Dagster definitions.
+
+The method is used when a source has only one table. However, if a source contains multiple tables, like this example:
+
+```yaml
+sources:
+ - name: clients_data
+ tables:
+ - name: names
+ - name: history
+```
+
+You can use define a with keys from instead:
+
+
+
+### Downstream dependencies
+
+Dagster allows you to define assets that are downstream of specific dbt models via . The below example defines `my_downstream_asset` as a downstream dependency of `my_dbt_model`:
+
+
+
+In the downstream asset, you may want direct access to the contents of the dbt model. To do so, you can customize the code within your `@asset`-decorated function to load upstream data.
+
+Dagster alternatively allows you to delegate loading data to an I/O manager. For example, if you wanted to consume a dbt model with the asset key `my_dbt_model` as a Pandas dataframe, that would look something like the following:
+
+
+
+## Building incremental models using partitions
+
+You can define a Dagster alongside dbt in order to build incremental models.
+
+Partitioned assets will be able to access the 's start and end dates, and these can be passed to dbt's CLI as variables which can be used to filter incremental models.
+
+When a partition definition to passed to the decorator, all assets are defined to operate on the same partitions. With this in mind, we can retrieve any time window from property in order to get the current start and end partitions.
+
+
+
+With the variables defined, we can now reference `min_date` and `max_date` in our SQL and configure the dbt model as incremental. Here, we define an incremental run to operate on rows with `order_date` that is between our `min_date` and `max_date`.
+
+```sql
+-- Configure the model as incremental, use a unique_key and the delete+insert strategy to ensure the pipeline is idempotent.
+{{ config(materialized='incremental', unique_key='order_date', incremental_strategy="delete+insert") }}
+
+select * from {{ ref('my_model') }}
+
+-- Use the Dagster partition variables to filter rows on an incremental run
+{% if is_incremental() %}
+where order_date >= '{{ var('min_date') }}' and order_date <= '{{ var('max_date') }}'
+{% endif %}
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/transform-dbt.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/transform-dbt.md
new file mode 100644
index 0000000000000..9079d11701966
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/transform-dbt.md
@@ -0,0 +1,73 @@
+---
+title: Getting started
+sidebar_position: 200
+---
+
+Dagster orchestrates dbt alongside other technologies, so you can schedule dbt with Spark, Python, etc. in a single data pipeline. Dagster's asset-oriented approach allows Dagster to understand dbt at the level of individual dbt models.
+
+
+ Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- A basic understanding of dbt, DuckDB, and Dagster concepts such as [assets](/guides/build/assets/) and [resources](/guides/build/external-resources/)
+- To install the [dbt](https://docs.getdbt.com/docs/core/installation-overview) and [DuckDB CLIs](https://duckdb.org/docs/api/cli/overview.html)
+- To install the following packages:
+
+ ```shell
+ pip install dagster duckdb plotly pandas dagster-dbt dbt-duckdb
+ ```
+
+
+## Setting up a basic dbt project
+
+Start by downloading this basic dbt project, which includes a few models and a DuckDB backend:
+
+```bash
+git clone https://github.com/dagster-io/basic-dbt-project
+```
+
+The project structure should look like this:
+
+```
+├── README.md
+├── dbt_project.yml
+├── profiles.yml
+├── models
+│ └── example
+│ ├── my_first_dbt_model.sql
+│ ├── my_second_dbt_model.sql
+│ └── schema.yml
+```
+
+First, you need to point Dagster at the dbt project and ensure Dagster has what it needs to build an asset graph. Create a `definitions.py` in the same directory as the dbt project:
+
+
+
+## Adding upstream dependencies
+
+Oftentimes, you'll want Dagster to generate data that will be used by downstream dbt models. To do this, add an upstream asset that the dbt project will as a source:
+
+
+
+Next, you'll add a dbt model that will source the `raw_customers` asset and define the dependency for Dagster. Create the dbt model:
+
+
+
+Next, create a `_source.yml` file that points dbt to the upstream `raw_customers` asset:
+
+
+
+![Screenshot of dbt lineage](/images/integrations/dbt/dbt-lineage.png)
+
+## Adding downstream dependencies
+
+You may also have assets that depend on the output of dbt models. Next, create an asset that depends on the result of the new `customers` model. This asset will create a histogram of the first names of the customers:
+
+
+
+## Scheduling dbt models
+
+You can schedule your dbt models by using the `dagster-dbt`'s `build_schedule_from_dbt_selection` function:
+
+
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster-plus/hybrid.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster-plus/hybrid.md
new file mode 100644
index 0000000000000..c3aa608651c8f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster-plus/hybrid.md
@@ -0,0 +1,70 @@
+---
+title: "Using dbt with Hybrid deployments in Dagster+"
+description: Deploy your dbt & Dagster project with Hybrid deployments in Dagster+.
+---
+
+If you have a Hybrid deployment, you must make the dbt project accessible to the Dagster code executed by your agent.
+
+- When using Amazon Elastic Container Service (ECS), Kubernetes, or Docker agent, you must include the dbt project in the Docker Image containing your Dagster code.
+- When using a local agent, you must make your dbt project accessible to your Dagster code on the same machine as your agent.
+
+For the dbt project to be used by Dagster, it must contain an up-to-date [manifest file](https://docs.getdbt.com/reference/artifacts/manifest-json) and [project dependencies](https://docs.getdbt.com/docs/collaborate/govern/project-dependencies).
+
+In this guide, we'll demonstrate how to prepare your dbt project for use in your Hybrid deployment in Dagster+.
+
+## Prerequisites
+
+To follow the steps in this guide, you'll need **an existing dbt project** that contains the following files in the repository root:
+
+- [`dbt_project.yml`](https://docs.getdbt.com/reference/dbt_project.yml)
+- [`profiles.yml`](https://docs.getdbt.com/docs/core/connect-data-platform/profiles.yml)
+
+## Using an Amazon Elastic Container Service (ECS), Kubernetes, or Docker agent
+
+If you are using an Amazon Elastic Container Service (ECS), Kubernetes, or Docker agent for your Hybrid deployments in Dagster+, your Dagster code must be packaged in a Docker image and pushed to a registry your agent can access. In this scenario, to use a dbt project with Dagster, you'll need to include it with your code in the Docker image.
+
+Before including the dbt project in the Docker image, you'll need to make sure it contains an up-to-date [manifest file](https://docs.getdbt.com/reference/artifacts/manifest-json) and [project dependencies](https://docs.getdbt.com/docs/collaborate/govern/project-dependencies).
+
+This can be done by running the [`dagster-dbt project prepare-and-package`](/api/python-api/libraries/dagster-dbt#prepare-and-package) command. In the workflow building and pushing your Docker image, make sure this command runs before building your Docker image to ensure all required dbt files are included. Note that this command runs `dbt deps` and `dbt parse` to create your manifest file.
+
+### Using CI/CD files
+
+If you are using [CI/CD files](/dagster-plus/features/ci-cd/ci-cd-file-reference) in a Git repository to build and push your Docker image, you'll need to add a few steps to allow the dbt project to deploy successfully.
+
+Our example updates the CI/CD files of a project from a GitHub repository, but this could be achieved in other platform like GitLab.
+
+1. In your Dagster project, locate the `.github/workflows` directory.
+
+2. Open the `deploy.yml` file.
+
+3. Locate the step in which which you build and push your docker image.
+
+4. Before this step, add the following:
+
+ ```yaml
+ - name: Prepare DBT project for deployment
+ run: |
+ python -m pip install pip --upgrade
+ pip install . --upgrade --upgrade-strategy eager ## Install the Python dependencies from the setup.py file, ex: dbt-core and dbt-duckdb
+ dagster-dbt project prepare-and-package --file /project.py ## Replace with the project.py location in the Dagster project folder
+ shell: bash
+ ```
+
+ When you add this step, you'll need to:
+
+ - **Add any [adapters](https://docs.getdbt.com/docs/connect-adapters) and libraries used by dbt to your `setup.py` file**.
+ - **Add the location of your Dagster project directory** to the `dagster-dbt project prepare-and-package` command.
+
+5. Save the changes.
+
+6. Open the `branch_deployments.yml` file and repeat steps 3 - 5.
+
+7. Commit the changes to the repository.
+
+Once the new step is pushed to the remote, your workflow will be updated to prepare your dbt project before building and pushing your docker image.
+
+## Using a local agent
+
+When using a local agent for your Hybrid deployments in Dagster+, your Dagster code and dbt project must be in a Python environment that can be accessed on the same machine as your agent.
+
+When updating the dbt project, it is important to refresh the [manifest file](https://docs.getdbt.com/reference/artifacts/manifest-json) and [project dependencies](https://docs.getdbt.com/docs/collaborate/govern/project-dependencies) to ensure that they are up-to-date when used with your Dagster code. This can be done by running the [`dagster-dbt project prepare-and-package`](/api/python-api/libraries/dagster-dbt#prepare-and-package) command. Note that this command runs `dbt deps` and `dbt parse` to refresh your manifest file.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster-plus/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster-plus/index.md
new file mode 100644
index 0000000000000..447107f1cb01c
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster-plus/index.md
@@ -0,0 +1,24 @@
+---
+title: "Using dbt with Dagster+"
+description: Deploy your dbt & Dagster project in Dagster+.
+---
+
+Using a dbt project in Dagster+ allows you to automatically load your dbt models as Dagster assets. This can be be done with both deployment options in Dagster+: Serverless and Hybrid.
+
+[Learn more about deployment options in Dagster+](/dagster-plus/deployment/deployment-types/).
+
+
+## Serverless deployments
+
+If you have a Serverless deployment, you can directly import an existing dbt project in Dagster+ when adding a new code location.
+
+For more information, see "[Using dbt with Serverless deployments in Dagster+](serverless)".
+
+## Hybrid deployments
+
+If you have a Hybrid deployment, you must make the dbt project accessible to the Dagster code executed by your agent.
+
+- When using Amazon Elastic Container Service (ECS), Kubernetes, or Docker agent, you must include the dbt project in the Docker Image containing your Dagster code.
+- When using a local agent, you must make your dbt project accessible to your Dagster code on the same machine as your agent.
+
+For more information, see "[Using dbt with Hybrid deployments in Dagster+](hybrid)".
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster-plus/serverless.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster-plus/serverless.md
new file mode 100644
index 0000000000000..6bb15045dd328
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster-plus/serverless.md
@@ -0,0 +1,199 @@
+---
+title: "Using dbt with Serverless deployments in Dagster+"
+description: Deploy your dbt & Dagster project with Serverless deployments in Dagster+.
+---
+
+Importing an existing dbt project in Dagster+ allows you to automatically load your dbt models as Dagster assets. This can be be done with:
+
+- An existing dbt project that is not already using Dagster, or
+- A Dagster project in which your dbt project is included
+
+In this guide, we'll demonstrate by using an existing dbt project that doesn't use Dagster.
+
+## Prerequisites
+
+To follow the steps in this guide, you'll need **Dagster+ Organization Admin**, **Admin**, or **Editor** permissions. This is required to create a code location.
+
+**For dbt-only projects**, you'll also need **an existing dbt project** that contains the following files in the repository root:
+
+- [`dbt_project.yml`](https://docs.getdbt.com/reference/dbt_project.yml)
+- [`profiles.yml`](https://docs.getdbt.com/docs/core/connect-data-platform/profiles.yml)
+
+**For dbt and Dagster projects**, Dagster+ requires several files to be present in your project. Refer to the [dbt & Dagster tutorial](/integrations/libraries/dbt/using-dbt-with-dagster/) to learn more about the structure and files required in a dbt and Dagster project.
+
+## Step 1: Import your project in Dagster+
+
+In this section, we'll demonstrate how to import an existing project to Dagster+. Our example imports the project from a GitHub repository, but Dagster+ also supports Gitlab.
+
+1. Sign in to your Dagster+ account.
+
+2. Navigate to **Deployment > Code locations**.
+
+3. Click **Add code location**.
+
+4. Depending on the type of project you imported, this step will vary:
+
+ - **For dbt-only projects**, click **Import a dbt project**, then **Continue**.
+ - **For dbt and Dagster projects,** click **Import a Dagster project**.
+
+5. At this point, you'll be prompted to select either GitHub or Gitlab. For this guide, we'll select **GitHub**.
+
+6. If prompted, sign into your GitHub account and complete the authorization process for the Dagster+ application. **Note**: The profile or organization you're using to authorize Dagster+ must have read and write access to the repository containing the project. After the authorization is complete, you'll be redirected back to Dagster+.
+
+7. In Dagster+, locate and select the repository containing the project by using the dropdowns. **Note**: dbt projects must have `dbt_profiles.yml` and `profiles.yml` files in the repository root or an error will display.
+
+8. Click **Continue** to begin the import process.
+
+9. The last step of the import process adds a few files, which we'll discuss in the next section, to the project. Depending on the type of project you imported, this step will vary:
+
+ - **For dbt-only projects**, Dagster+ will open a pull request to update the repository. You'll need to review and merge the pull request to complete the process.
+ - **For dbt and Dagster projects,** Dagster+ will directly commit the files to the repository.
+
+Once Dagster+ finishes importing the project, move onto the next step.
+
+## Step 2: Review the repository changes
+
+The file structure of the repository will change the first time a project is deployed using Dagster+. For dbt projects, a few things will happen:
+
+- **A [`dagster_cloud.yaml` file](/dagster-plus/deployment/code-locations/dagster-cloud-yaml) will be created.** This file defines the project as a Dagster+ code location.
+- **A few `.yml` files, used for CI/CD, will be created in `.github/workflows`.** [These files](/dagster-plus/features/ci-cd/ci-cd-file-reference), named `branch_deployments.yml` and `deploy.yml`, manage the deployments of the repository.
+- **For dbt-only projects being deployed for the first time**, Dagster+ will create a new Dagster project in the repository using the [`dagster-dbt scaffold`](/integrations/libraries/dbt/reference#scaffolding-a-dagster-project-from-a-dbt-project) command. This will result in a Dagster project that matches the dbt project. For example, a dbt project named `my_dbt_project` will contain a Dagster project in `my_dbt_project/my_dbt_project` after the process completes.
+
+**Use the following tabs** to see how the repository will change for a dbt-only project and a dbt and Dagster project being deployed for the first time.
+
+
+
+
+Before the Dagster+ changes, a typical dbt project would include files like `dbt_project.yml`, `profiles.yml`, dbt models in `.sql` format, and sbt seeds in `.csv` format. As this is a git repository, other files like `.gitignore`, `LICENSE` and `README.md` may also be included:
+
+```shell
+## dbt-only project
+## before Dagster+ deployment
+
+my_dbt_project
+├── models
+│ ├── my_model.sql
+├── seeds
+│ ├── my_seeds.csv
+├── .gitignore
+├── LICENSE
+├── README.md
+├── dbt_project.yml
+└── profiles.yml
+```
+
+When the Dagster+ deployment process completes, the repository will now look like the following:
+
+```shell
+## dbt-only project
+## after Dagster+ deployment
+
+my_dbt_project
+├── .github ## CI/CD files
+│ ├── workflows
+│ │ ├── branch_deployments.yml
+│ │ ├── deploy.yml
+├── models
+│ ├── my_model.sql
+├── my_dbt_project ## New Dagster project
+│ ├── my_dbt_project
+│ │ ├── __init__.py
+│ │ ├── assets.py
+│ │ ├── definitions.py
+│ │ ├── project.py
+│ │ ├── schedules.py
+│ ├── pyproject.toml
+│ ├── setup.py
+├── seeds
+│ ├── my_seeds.csv
+├── .gitignore
+├── LICENSE
+├── README.md
+├── dagster_cloud.yaml ## Dagster+ code location file
+├── dbt_project.yml
+└── profiles.yml
+```
+
+
+
+
+After the Dagster+ changes, a dbt and Dagster project will include the files required for dbt and Dagster, some files related to git, and the newly-added Dagster+ files:
+
+```shell
+## dbt and Dagster project
+## after Dagster+ deployment
+
+my_dbt_and_dagster_project
+├── .github ## CI/CD files
+│ ├── workflows
+│ │ ├── branch_deployments.yml
+│ │ ├── deploy.yml
+├── dbt
+│ ├── models
+│ │ ├── my_model.sql
+│ ├── seeds
+│ │ ├── my_seeds.csv
+│ ├── dbt_project.yml
+│ ├── profiles.yml
+├── my_dbt_and_dagster_project
+│ ├── __init__.py
+│ ├── assets.py
+│ ├── definitions.py
+│ ├── project.py
+│ ├── schedules.py
+├── .gitignore
+├── LICENSE
+├── README.md
+├── dagster_cloud.yaml ## Dagster+ code location file
+├── pyproject.toml
+└── setup.py
+```
+
+
+
+
+## Step 3: Update the CI/CD files
+
+:::note
+
+This step only applies to dbt and Dagster projects. Skip this step if you imported a dbt-only project.
+
+:::
+
+The last step is to update the [CI/CD files](/dagster-plus/features/ci-cd/ci-cd-file-reference) in the repository. When you import a dbt project into Dagster+ using the **Import a Dagster project** option, you'll need to add a few steps to allow the dbt project to deploy successfully.
+
+1. In your Dagster project, locate the `.github/workflows` directory.
+
+2. Open the `deploy.yml` file.
+
+3. Locate the `Checkout for Python Executable Deploy` step, which should be on or near line 38.
+
+4. After this step, add the following:
+
+ ```yaml
+ - name: Prepare DBT project for deployment
+ if: steps.prerun.outputs.result == 'pex-deploy'
+ run: |
+ python -m pip install pip --upgrade
+ cd project-repo
+ pip install . --upgrade --upgrade-strategy eager ## Install the Python dependencies from the setup.py file, ex: dbt-core and dbt-duckdb
+ dagster-dbt project prepare-and-package --file /project.py ## Replace with the project.py location in the Dagster project folder
+ shell: bash
+ ```
+
+ When you add this step, you'll need to:
+
+ - **Add any [adapters](https://docs.getdbt.com/docs/connect-adapters) and libraries used by dbt to your `setup.py` file**. In this example, we're using `dbt-core` and `dbt-duckdb`.
+ - **Add the location of your Dagster project directory** to the `dagster-dbt project prepare-and-package` command. In this example, our project is in the `/my_dbt_and_dagster_project` directory.
+
+5. Save the changes.
+
+6. Open the `branch_deployments.yml` file and repeat steps 3 - 5.
+
+7. Commit the changes to the repository.
+
+Once the new step is pushed to the remote, GitHub will automatically try to run a new job using the updated workflow.
+
+## What's next?
+
+For an end-to-end example, from the project creation to the deployment to Dagster+, check out the Dagster & dbt course in [Dagster University](https://courses.dagster.io).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/downstream-assets.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/downstream-assets.md
new file mode 100644
index 0000000000000..c257da10cc8fe
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/downstream-assets.md
@@ -0,0 +1,74 @@
+---
+title: "Add a downstream asset"
+description: Dagster can orchestrate dbt alongside other technologies.
+sidebar_position: 400
+---
+
+By this point, you've [set up a dbt project](set-up-dbt-project), [loaded dbt models into Dagster as assets](load-dbt-models), and [defined assets upstream of your dbt models](upstream-assets).
+
+In this step, you'll:
+
+- [Install the plotly library](#step-1-install-the-plotly-library)
+- [Define a downstream asset that computes a chart using plotly](#step-2-define-the-order_count_chart-asset)
+- [Materialize the `order_count_chart` asset](#step-3-materialize-the-order_count_chart-asset)
+
+## Step 1: Install the plotly library
+
+```shell
+pip install plotly
+```
+
+## Step 2: Define the order_count_chart asset
+
+You've added upstream assets to your data pipeline, but nothing downstream - until now. In this step, you'll define a Dagster asset called `order_count_chart` that uses the data in the `customers` dbt model to computes a plotly chart of the number of orders per customer.
+
+Like the `raw_customers` asset that we added in the [previous section](upstream-assets#step-2-define-an-upstream-dagster-asset), we'll put this asset in our `definitions.py` file, inside the `jaffle_dagster` directory.
+
+To add the `order_count_chart` asset:
+
+1. Replace the imports section with the following:
+
+
+
+ This adds an import for plotly, as well as and , which we'll use in our asset.
+
+2. After your definition of `jaffle_shop_dbt_assets`, add the definition for the `order_count_chart` asset:
+
+
+
+ This asset definition looks similar the asset we defined in the previous section. In this case, instead of fetching data from an external source and writing it to DuckDB, it reads data from DuckDB, and then uses it to make a plot.
+
+ The line `deps=get_asset_key_for_model([jaffle_shop_dbt_assets], "customers")` tells Dagster that this asset is downstream of the `customers` dbt model. This dependency will be displayed as such in Dagster's UI. If you launch a run to materialize both of them, Dagster won't run `order_count_chart` until `customers` completes.
+
+3. Add the `order_count_chart` to the `Definitions`:
+
+
+
+## Step 3: Materialize the order_count_chart asset
+
+If the Dagster UI is still running from the previous section, click the "Reload Definitions" button in the upper right corner. If you shut it down, then you can launch it again with the same command from the previous section:
+
+```shell
+dagster dev
+```
+
+The UI will look like this:
+
+![Asset group with dbt models and Python asset](/images/integrations/dbt/using-dbt-with-dagster/downstream-assets/asset-graph.png)
+
+A new asset named `order_count_chart` is at the bottom, downstream of the `customers` asset. Click on `order_count_chart` and click **Materialize selected**.
+
+That's it! When the run successfully completes, the following chart will automatically open in your browser:
+
+![plotly chart asset displayed in Chrome](/images/integrations/dbt/using-dbt-with-dagster/downstream-assets/order-count-chart.png)
+
+## What's next?
+
+That's the end of this tutorial - congratulations! By now, you should have a working dbt and Dagster integration and a handful of materialized Dagster assets.
+
+What's next? From here, you can:
+
+- Learn more about [asset definitions](/guides/build/assets/)
+- Learn how to [build jobs that materialize dbt assets](/integrations/libraries/dbt/reference#scheduling-dbt-jobs)
+- Get a [deeper understanding of Dagster's dbt integration](/integrations/libraries/dbt/reference)
+- Check out the [`dagster-dbt` API docs](/api/python-api/libraries/dagster-dbt)
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/index.md
new file mode 100644
index 0000000000000..9c40754037396
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/index.md
@@ -0,0 +1,41 @@
+---
+title: "Using dbt with Dagster"
+description: Dagster can orchestrate dbt alongside other technologies.
+---
+
+:::note
+
+Using dbt Cloud? Check out the [Dagster & dbt Cloud guide](/integrations/libraries/dbt/dbt-cloud).
+
+:::note
+
+In this tutorial, we'll walk you through integrating dbt with Dagster using a smaller version of dbt's example [jaffle shop project](https://github.com/dbt-labs/jaffle_shop), the [dagster-dbt library](/api/python-api/libraries/dagster-dbt), and a data warehouse, such as [DuckDB](https://duckdb.org/).
+
+By the end of this tutorial, you'll have your dbt models represented in Dagster along with other [Dagster asset definitions](/integrations/libraries/dbt/reference#dbt-models-and-dagster-asset-definitions) upstream and downstream of them:
+
+![Asset group with dbt models and Python asset](/images/integrations/dbt/using-dbt-with-dagster/downstream-assets/asset-graph-materialized.png)
+
+To get there, you will:
+
+- [Set up a dbt project](set-up-dbt-project)
+- [Load the dbt models into Dagster as assets](load-dbt-models)
+- [Create and materialize upstream Dagster assets](upstream-assets)
+- [Create and materialize a downstream asset](downstream-assets) that outputs a plotly chart
+
+## Prerequisites
+
+To complete this tutorial, you'll need:
+
+- **To have [git](https://en.wikipedia.org/wiki/Git) installed**. If it's not installed already (find out by typing `git` in your terminal), you can install it using the [instructions on the git website](https://git-scm.com/book/en/v2/Getting-Started-Installing-Git).
+
+- **To install dbt, Dagster, and the Dagster webserver/UI**. Run the following to install everything using pip:
+
+ ```shell
+ pip install dagster-dbt dagster-webserver dbt-duckdb
+ ```
+
+ The `dagster-dbt` library installs both `dbt-core` and `dagster` as dependencies. `dbt-duckdb` is installed as you'll be using [DuckDB](https://duckdb.org/) as a database during this tutorial. Refer to the [dbt](https://docs.getdbt.com/dbt-cli/install/overview) and [Dagster](/getting-started/installation) installation docs for more info.
+
+## Ready to get started?
+
+When you've fulfilled all the prerequisites for the tutorial, you can get started by [setting up the dbt project](set-up-dbt-project).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/load-dbt-models.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/load-dbt-models.md
new file mode 100644
index 0000000000000..e8a05a3fa7401
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/load-dbt-models.md
@@ -0,0 +1,101 @@
+---
+title: "Load dbt models as Dagster assets"
+description: Dagster can orchestrate dbt alongside other technologies.
+sidebar_position: 200
+---
+
+At this point, you should have a [fully-configured dbt project](set-up-dbt-project) that's ready to work with Dagster.
+
+In this section, you'll finally begin integrating dbt with Dagster. To do so, you'll:
+
+- [Create a Dagster project that wraps your dbt project](#step-1-create-a-dagster-project-that-wraps-your-dbt-project)
+- [Inspect your Dagster project in Dagster's UI](#step-2-inspect-your-dagster-project-in-dagsters-ui)
+- [Build your dbt models in Dagster](#step-3-build-your-dbt-models-in-dagster)
+- [Understand the Python code in your Dagster project](#step-4-understand-the-python-code-in-your-dagster-project)
+
+## Step 1: Create a Dagster project that wraps your dbt project
+
+You can create a Dagster project that wraps your dbt project by using the `dagster-dbt` command line interface. Make sure you're in the directory where your `dbt_project.yml` is. If you're continuing from the previous section, then you'll already be in this directory. Then, run:
+
+```shell
+dagster-dbt project scaffold --project-name jaffle_dagster
+```
+
+This creates a directory called `jaffle_dagster/` inside the current directory. The `jaffle_dagster/` directory contains a set of files that define a Dagster project.
+
+In general, it's up to you where to put your Dagster project. It's most common to put your Dagster project at the root of your git repository. Therefore, in this case, because the `dbt_project.yml` was at the root of the `jaffle_shop` git repository, we created our Dagster project there.
+
+**Note**: The `dagster-dbt project scaffold` command creates the Dagster project in whatever directory you run it from. If that's a different directory from where your `dbt_project.yml` lives, then you'll need to provide a value for the `--dbt-project-dir` option so that Dagster knows where to look for your dbt project.
+
+## Step 2: Inspect your Dagster project in Dagster's UI
+
+Now that you have a Dagster project, you can run Dagster's UI to take a look at it.
+
+1. Change directories to the Dagster project directory:
+
+ ```shell
+ cd jaffle_dagster/
+ ```
+
+2. To start Dagster's UI, run the following:
+
+ ```shell
+ dagster dev
+ ```
+
+ Which will result in output similar to:
+
+ ```shell
+ Serving dagster-webserver on http://127.0.0.1:3000 in process 70635
+ ```
+
+3. In your browser, navigate to [http://127.0.0.1:3000](http://127.0.0.1:3000) The page will display the assets:
+
+![Asset graph in Dagster's UI, containing dbt models loaded as Dagster assets](/images/integrations/dbt/using-dbt-with-dagster/load-dbt-models/asset-graph.png)
+
+## Step 3: Build your dbt models in Dagster
+
+You can do more than view your dbt models in Dagster – you can also run them. In Dagster, running a dbt model corresponds to _materializing_ an asset. Materializing an asset means running some computation to update its contents in persistent storage. In this tutorial, that persistent storage is our local DuckDB database.
+
+To build your dbt project, i.e. materialize your assets, click the **Materialize all** button near the top right corner of the page. This will launch a run to materialize the assets. When finished, the **Materialized** and **Latest Run** attributes in the asset will be populated:
+
+![Asset graph in Dagster's UI, showing materialized assets](/images/integrations/dbt/using-dbt-with-dagster/load-dbt-models/asset-graph-materialized.png)
+
+After the run completes, you can:
+
+- Click the **asset** to open a sidebar containing info about the asset, including its last materialization stats and a link to view the **Asset details** page
+- Click the ID of the **Latest Run** in an asset to view the **Run details** page. This page contains detailed info about the run, including timing information, errors, and logs.
+
+## Step 4: Understand the Python code in your Dagster project
+
+You saw how you can create a Dagster project that loads a dbt project. How does this work? Understanding how Dagster loads a dbt project will give you a foundation for customizing how Dagster runs your dbt project, as well as for connecting it to other data assets outside of dbt.
+
+The most important file is the Python file that contains the set of definitions for Dagster to load: `jaffle_shop/definitions.py`. Dagster executes the code in this file to find out what assets it should be aware of, as well as details about those assets. For example, when you ran `dagster dev` in the previous step, Dagster executed the code in this file to determine what assets to display in the UI.
+
+In our `definitions.py` Python file, we import from `assets.py`, which contains the code to model our dbt models as Dagster assets. To return a Dagster asset for each dbt model, the code in this `assets.py` file needs to know what dbt models you have. It finds out what models you have by reading a file called a `manifest.json`, which is a file that dbt can generate for any dbt project and contains information about every model, seed, snapshot, test, etc. in the project.
+
+To retrieve the `manifest.json`, `assets.py` imports from `project.py`, which defines an internal representation of your dbt project. Then, in `assets.py`, the path to the `manifest.json` file can be accessed with `dbt_project.manifest_path`:
+
+
+
+Generating the `manifest.json` file for a dbt project is time-consuming, so it's best to avoid doing so every time this Python module is imported. Thus, in production deployments of Dagster, you'll typically have the CI/CD system that packages up your code generate your `manifest.json`.
+
+However, in development, you typically want changes made to files in your dbt project to be immediately reflected in the Dagster UI without needing to regenerate the manifest.
+
+`dbt_project.prepare_if_dev()` helps with this – it re-generates your `manifest.json` at the time Dagster imports your code, _but_ only if it's being imported by the `dagster dev` command.
+
+Once you've got a `manifest.json` file, it's time to define your Dagster assets using it. The following code, in your project's `assets.py`, does this:
+
+
+
+This code might look a bit fancy, because it uses a decorator. Here's a breakdown of what's going on:
+
+- It creates a variable named `jaffle_shop_dbt_assets` that holds an object that represents a set of Dagster assets.
+- These Dagster assets reflect the dbt models described in the manifest file. The manifest file is passed in using the `manifest` argument.
+- The decorated function defines what should happen when you materialize one of these Dagster assets, e.g. by clicking the **Materialize** button in the UI or materializing it automatically by putting it on a schedule. In this case, it will invoke the `dbt build` command on the selected assets. The `context` parameter that's provided along with `dbt build` carries the selection.
+
+If you later want to customize how your dbt models are translated into Dagster assets, you'll do so by editing its definition in `assets.py`.
+
+## What's next?
+
+At this point, you've loaded your dbt models into Dagster as assets, viewed them in Dagster's asset graph UI, and materialized them. Next, you'll learn how to [add upstream Dagster assets](upstream-assets).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/set-up-dbt-project.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/set-up-dbt-project.md
new file mode 100644
index 0000000000000..d5bfad64cb0f4
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/set-up-dbt-project.md
@@ -0,0 +1,77 @@
+---
+title: "Set up the dbt project"
+description: Dagster can orchestrate dbt alongside other technologies.
+sidebar_position: 100
+---
+
+In this part of the tutorial, you will:
+
+- [Download a dbt project](#step-1-download-the-sample-dbt-project)
+- [Configure your dbt project to run with DuckDB](#step-2-configure-your-dbt-project-to-run-with-duckdb)
+- [Build your dbt project](#step-3-build-your-dbt-project)
+
+## Step 1: Download the sample dbt project
+
+Let's get started by downloading a sample dbt project. We'll use the standard dbt [Jaffle Shop](https://github.com/dbt-labs/jaffle_shop) example.
+
+1. First, create a folder that will ultimately contain both your dbt project and Dagster code.
+
+ ```shell
+ mkdir tutorial-dbt-dagster
+ ```
+
+2. Then, navigate into that folder:
+
+ ```shell
+ cd tutorial-dbt-dagster
+ ```
+
+3. Finally, download the sample dbt project into that folder.
+
+ ```shell
+ git clone https://github.com/dbt-labs/jaffle_shop.git
+ ```
+
+## Step 2: Configure your dbt project to run with DuckDB
+
+Running dbt requires a data warehouse to store the tables that are created from the dbt models. For our data warehouse, we'll use DuckDB, because setting it up doesn't require any long-running services or external infrastructure.
+
+You'll set up dbt to work with DuckDB by configuring a dbt [profile](https://docs.getdbt.com/docs/core/connect-data-platform/connection-profiles):
+
+1. Navigate into the `jaffle_shop` folder, which was created when you downloaded the project, inside your `tutorial-dbt-dagster` folder:
+
+ ```shell
+ cd jaffle_shop
+ ```
+
+2. In this folder, with your text editor of choice, create a file named `profiles.yml` and add the following code to it:
+
+ ```yaml
+ jaffle_shop:
+ target: dev
+ outputs:
+ dev:
+ type: duckdb
+ path: tutorial.duckdb
+ threads: 24
+ ```
+
+## Step 3: Build your dbt project
+
+With the profile configured above, your dbt project should now be usable. To test it out, run:
+
+```shell
+dbt build
+```
+
+This will run all the models, seeds, and snapshots in the project and store a set of tables in your DuckDB database.
+
+:::note
+
+For other dbt projects, you may need to run additional commands before building the project. For instance, a project with [dependencies](https://docs.getdbt.com/docs/collaborate/govern/project-dependencies) will require you to run [`dbt deps`](https://docs.getdbt.com/reference/commands/deps) before building the project. For more information, see the [official dbt Command reference page](https://docs.getdbt.com/reference/dbt-commands).
+
+:::note
+
+## What's next?
+
+At this point, you should have a fully-configured dbt project that's ready to work with Dagster. The next step is to [load the dbt models into Dagster as assets](load-dbt-models).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/upstream-assets.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/upstream-assets.md
new file mode 100644
index 0000000000000..3e123643fa1c3
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dbt/using-dbt-with-dagster/upstream-assets.md
@@ -0,0 +1,125 @@
+---
+title: "Define assets upstream of your dbt models"
+description: Dagster can orchestrate dbt alongside other technologies.
+sidebar_position: 300
+---
+
+By this point, you've [set up a dbt project](set-up-dbt-project) and [loaded dbt models into Dagster as assets](load-dbt-models).
+
+However, the tables at the root of the pipeline are static: they're [dbt seeds](https://docs.getdbt.com/docs/build/seeds), CSVs that are hardcoded into the dbt project. In a more realistic data pipeline, these tables would typically be ingested from some external data source, for example by using a tool like Airbyte or Fivetran, or by Python code.
+
+These ingestion steps in the pipeline often don't make sense to define inside dbt, but they often still do make sense to define as Dagster assets. You can think of a Dagster asset definition as a more general version of a dbt model. A dbt model is one kind of asset, but another kind is one that's defined in Python, using Dagster's Python API. The dbt integration reference page includes a [section](/integrations/libraries/dbt/reference#dbt-models-and-dagster-asset-definitions) that outlines the parallels between dbt models and Dagster asset definitions.
+
+In this section, you'll replace the `raw_customers` dbt seed with a Dagster asset that represents it. You'll write Python code that populates this table by fetching data from the web. This will allow you to launch runs that first execute Python code to populate the `raw_customers` table and then invoke dbt to populate the downstream tables.
+
+You'll:
+
+- [Install the Pandas and DuckDB Python libraries](#step-1-install-the-pandas-and-duckdb-python-libraries)
+- [Define an upstream Dagster asset](#step-2-define-an-upstream-dagster-asset)
+- [In the dbt project, replace a seed with a source](#step-3-in-the-dbt-project-replace-a-seed-with-a-source)
+- [Materialize the assets using the Dagster UI](#step-4-materialize-the-assets-using-the-dagster-ui)
+
+## Step 1: Install the Pandas and DuckDB Python libraries
+
+The Dagster asset that you write will fetch data using [Pandas](https://pandas.pydata.org/) and write it out to your DuckDB warehouse using [DuckDB's Python API](https://duckdb.org/docs/api/python/overview.html). To use these, you'll need to install them:
+
+```shell
+pip install pandas duckdb pyarrow
+```
+
+## Step 2: Define an upstream Dagster asset
+
+To fetch the data the dbt models require, we'll write a Dagster asset for `raw_customers`. We'll put this asset in our `assets.py` file, inside the `jaffle_dagster` directory. This is the file that contains the code that defines our dbt models, which we reviewed at the end of the [last section](load-dbt-models#step-4-understand-the-python-code-in-your-dagster-project). Copy and paste this code to overwrite the existing contents of that file:
+
+
+
+Let's review the changes we made:
+
+1. At the top, we added imports for `pandas` and `duckdb`, which we use for fetching data into a `DataFrame` and writing it to DuckDB.
+
+2. We added a `duckdb_database_path` variable, which holds the location of our DuckDB database. Remember that DuckDB databases are just regular files on the local filesystem. The path is the same path that we used when we configured our `profiles.yml` file. This variable is used in the implementations of the `raw_customers` asset.
+
+3. We added a definition for the `raw_customers` table by writing a function named `raw_customers` and decorating it with the decorator. We labeled it with `compute_kind="python"` to indicate in the Dagster UI that this is an asset defined in Python. The implementation inside the function fetches data from the internet and writes it to a table in our DuckDB database. Similar to how running a dbt model executes a select statement, materializing this asset will execute this Python code.
+
+Finally, let's update the `assets` argument of our `Definitions` object, in `definitions.py`, to include the new asset we just defined:
+
+
+
+## Step 3: In the dbt project, replace a seed with a source
+
+1. Because we're replacing it with a Dagster asset, we no longer need the dbt seed for `raw_customers`, so we can delete it:
+
+ ```shell
+ cd ..
+ rm seeds/raw_customers.csv
+ ```
+
+2. Instead, we want to tell dbt that `raw_customers` is a table that is defined outside of the dbt project. We can do that by defining it inside a [dbt source](https://docs.getdbt.com/docs/build/sources).
+
+ Create a file called `sources.yml` inside the `models/` directory, and put this inside it:
+
+ ```yaml
+ version: 2
+
+ sources:
+ - name: jaffle_shop
+ tables:
+ - name: raw_customers
+ meta:
+ dagster:
+ asset_key: ["raw_customers"] # This metadata specifies the corresponding Dagster asset for this dbt source.
+ ```
+
+This is a standard dbt source definition, with one addition: it includes metadata, under the `meta` property, that specifies the Dagster asset that it corresponds to. When Dagster reads the contents of the dbt project, it reads this metadata and infers the correspondence. For any dbt model that depends on this dbt source, Dagster then knows that the Dagster asset corresponding to the dbt model should depend on the Dagster asset corresponding to the source.
+
+3. Then, update the model that depends on the `raw_customers` seed to instead depend on the source. Replace the contents of `model/staging/stg_customers.sql` with this:
+
+ ```sql
+ with source as (
+
+ {#-
+ Use source instead of seed:
+ #}
+ select * from {{ source('jaffle_shop', 'raw_customers') }}
+
+ ),
+
+ renamed as (
+
+ select
+ id as customer_id,
+ first_name,
+ last_name
+
+ from source
+
+ )
+
+ select * from renamed
+ ```
+
+## Step 4: Materialize the assets using the Dagster UI
+
+If the Dagster UI is still running from the previous section, click the "Reload Definitions" button in the upper right corner. If you shut it down, then you can launch it again with the same command from the previous section:
+
+```shell
+dagster dev
+```
+
+Our `raw_customers` model is now defined as a Python asset. We can also see that assets downstream of this new Python asset, such as `stg_customers` and `customers`, are now marked stale because the code definition of `raw_customers` has changed.
+
+![Asset group with dbt models and Python asset](/images/integrations/dbt/using-dbt-with-dagster/upstream-assets/asset-graph.png)
+
+Click the **Materialize all** button. This will launch a run with two steps:
+
+- Run the `raw_customers` Python function to fetch data and write the `raw_customers` table to DuckDB.
+- Run all the dbt models using `dbt build`, like in the last section.
+
+If you click to view the run, you can see a graphical representation of these steps, along with logs.
+
+![Run page for run with dbt models and Python asset](/images/integrations/dbt/using-dbt-with-dagster/upstream-assets/run-page.png)
+
+
+## What's next?
+
+At this point, you've built and materialized an upstream Dagster asset, providing source data to your dbt models. In the last section of the tutorial, we'll show you how to add a [downstream asset to the pipeline](downstream-assets).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/deltalake/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/deltalake/index.md
new file mode 100644
index 0000000000000..5f97c023440e8
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/deltalake/index.md
@@ -0,0 +1,44 @@
+---
+layout: Integration
+status: published
+name: Delta Lake
+title: Dagster & Delta Lake
+sidebar_label: Delta Lake
+excerpt: Integrate your pipelines into Delta Lake.
+date: 2022-11-07
+communityIntegration: true
+apireflink: https://delta-io.github.io/delta-rs/integrations/delta-lake-dagster/
+docslink:
+partnerlink: https://delta.io/
+categories:
+ - Storage
+enabledBy:
+enables:
+tags: [community-supported, storage]
+sidebar_custom_props:
+ logo: images/integrations/deltalake.svg
+ community: true
+---
+
+Delta Lake is a great storage format for Dagster workflows. With this integration, you can use the Delta Lake I/O Manager to read and write your Dagster assets.
+
+Here are some of the benefits that Delta Lake provides Dagster users:
+
+- Native PyArrow integration for lazy computation of large datasets
+- More efficient querying with file skipping with Z Ordering and liquid clustering
+- Built-in vacuuming to remove unnecessary files and versions
+- ACID transactions for reliable writes
+- Smooth versioning integration (versions can be use to trigger downstream updates).
+- Surfacing table stats based on the file statistics
+
+### Installation
+
+```bash
+pip install dagster-deltalake
+pip install dagster-deltalake-pandas
+pip install dagster-deltalake-polars
+```
+
+### About Delta Lake
+
+Delta Lake is an open source storage framework that enables building a Lakehouse architecture with compute engines including Spark, PrestoDB, Flink, Trino, and Hive and APIs for Scala, Java, Rust, and Python.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/deltalake/reference.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/deltalake/reference.md
new file mode 100644
index 0000000000000..5bdd4b2af1b2b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/deltalake/reference.md
@@ -0,0 +1,225 @@
+---
+title: "dagster-deltalake integration reference"
+description: Store your Dagster assets in Delta Lake
+sidebar_position: 200
+---
+
+This reference page provides information for working with [`dagster-deltalake`](/api/python-api/libraries/dagster-deltalake) features that are not covered as part of the [Using Delta Lake with Dagster tutorial](using-deltalake-with-dagster).
+
+- [Selecting specific columns in a downstream asset](#selecting-specific-columns-in-a-downstream-asset)
+- [Storing partitioned assets](#storing-partitioned-assets)
+- [Storing tables in multiple schemas](#storing-tables-in-multiple-schemas)
+- [Using the Delta Lake I/O manager with other I/O managers](#using-the-delta-lake-io-manager-with-other-io-managers)
+- [Storing and loading PyArrow Tables or Polars DataFrames in Delta Lake](#storing-and-loading-pyarrow-tables-or-polars-dataframes-in-delta-lake)
+- [Configuring storage backends](#configuring-storage-backends)
+
+## Selecting specific columns in a downstream asset
+
+Sometimes you may not want to fetch an entire table as the input to a downstream asset. With the Delta Lake I/O manager, you can select specific columns to load by supplying metadata on the downstream asset.
+
+
+
+In this example, we only use the columns containing sepal data from the `iris_dataset` table created in [Step 2](using-deltalake-with-dagster#step-2-create-delta-lake-tables) of the [Using Dagster with Delta Lake tutorial](using-deltalake-with-dagster). To select specific columns, we can add metadata to the input asset. We do this in the `metadata` parameter of the `AssetIn` that loads the `iris_dataset` asset in the `ins` parameter. We supply the key `columns` with a list of names of the columns we want to fetch.
+
+When Dagster materializes `sepal_data` and loads the `iris_dataset` asset using the Delta Lake I/O manager, it will only fetch the `sepal_length_cm` and `sepal_width_cm` columns of the `iris/iris_dataset` table and pass them to `sepal_data` as a Pandas DataFrame.
+
+## Storing partitioned assets
+
+The Delta Lake I/O manager supports storing and loading partitioned data. To correctly store and load data from the Delta table, the Delta Lake I/O manager needs to know which column contains the data defining the partition bounds. The Delta Lake I/O manager uses this information to construct the correct queries to select or replace the data.
+
+In the following sections, we describe how the I/O manager constructs these queries for different types of partitions.
+
+:::
+
+For partitioning to work, the partition dimension needs to be one of the partition columns defined on the Delta table. Tables created via the I/O manager will be configured accordingly.
+
+:::
+
+
+
+
+**Storing static partitioned assets**
+
+To store static partitioned assets in your Delta Lake, specify `partition_expr` metadata on the asset to tell the Delta Lake I/O manager which column contains the partition data:
+
+
+
+Dagster uses the `partition_expr` metadata to generate appropriate function parameters when loading the partition in the downstream asset. When loading a static partition this roughly corresponds to the following SQL statement:
+
+```sql
+SELECT *
+ WHERE [partition_expr] in ([selected partitions])
+```
+
+A partition must be selected when materializing the above assets, as described in the [Materializing partitioned assets](/guides/build/partitions-and-backfills/partitioning-assets) documentation. In this example, the query used when materializing the `Iris-setosa` partition of the above assets would be:
+
+```sql
+SELECT *
+ WHERE species = 'Iris-setosa'
+```
+
+
+
+
+**Storing time-partitioned assets**
+
+Like static partitioned assets, you can specify `partition_expr` metadata on the asset to tell the Delta Lake I/O manager which column contains the partition data:
+
+
+
+Dagster uses the `partition_expr` metadata to craft the `SELECT` statement when loading the correct partition in the downstream asset. When loading a dynamic partition, the following statement is used:
+
+```sql
+SELECT *
+ WHERE [partition_expr] = [partition_start]
+```
+
+A partition must be selected when materializing the above assets, as described in the [Materializing partitioned assets](/guides/build/partitions-and-backfills/partitioning-assets) documentation. The `[partition_start]` and `[partition_end]` bounds are of the form `YYYY-MM-DD HH:MM:SS`. In this example, the query when materializing the `2023-01-02` partition of the above assets would be:
+
+```sql
+SELECT *
+ WHERE time = '2023-01-02 00:00:00'
+```
+
+
+
+
+**Storing multi-partitioned assets**
+
+The Delta Lake I/O manager can also store data partitioned on multiple dimensions. To do this, specify the column for each partition as a dictionary of `partition_expr` metadata:
+
+
+
+Dagster uses the `partition_expr` metadata to craft the `SELECT` statement when loading the correct partition in a downstream asset. For multi-partitions, Dagster concatenates the `WHERE` statements described in the above sections to craft the correct `SELECT` statement.
+
+A partition must be selected when materializing the above assets, as described in the [Materializing partitioned assets](/todo) documentation. For example, when materializing the `2023-01-02|Iris-setosa` partition of the above assets, the following query will be used:
+
+```sql
+SELECT *
+ WHERE species = 'Iris-setosa'
+ AND time = '2023-01-02 00:00:00'
+```
+
+
+
+
+
+
+## Storing tables in multiple schemas
+
+You may want to have different assets stored in different Delta Lake schemas. The Delta Lake I/O manager allows you to specify the schema in several ways.
+
+If you want all of your assets to be stored in the same schema, you can specify the schema as configuration to the I/O manager, as we did in [Step 1](using-deltalake-with-dagster#step-1-configure-the-delta-lake-io-manager) of the [Using Dagster with Delta Lake tutorial](using-deltalake-with-dagster).
+
+If you want to store assets in different schemas, you can specify the schema as part of the asset's key:
+
+
+
+In this example, the `iris_dataset` asset will be stored in the `IRIS` schema, and the `daffodil_dataset` asset will be found in the `DAFFODIL` schema.
+
+:::
+
+The two options for specifying schema are mutually exclusive. If you provide `schema` configuration to the I/O manager, you cannot also provide it via the asset key and vice versa. If no `schema` is provided, either from configuration or asset keys, the default schema `public` will be used.
+
+:::
+
+## Using the Delta Lake I/O manager with other I/O managers
+
+You may have assets that you don't want to store in Delta Lake. You can provide an I/O manager to each asset using the `io_manager_key` parameter in the decorator:
+
+
+
+In this example:
+
+- The `iris_dataset` asset uses the I/O manager bound to the key `warehouse_io_manager` and `iris_plots` uses the I/O manager bound to the key `blob_io_manager`
+- In the object, we supply the I/O managers for those keys
+- When the assets are materialized, the `iris_dataset` will be stored in Delta Lake, and `iris_plots` will be saved in Amazon S3
+
+## Storing and loading PyArrow tables or Polars DataFrames in Delta Lake
+
+The Delta Lake I/O manager also supports storing and loading PyArrow and Polars DataFrames.
+
+
+
+
+**Storing and loading PyArrow Tables with Delta Lake**
+
+The `deltalake` package relies heavily on Apache Arrow for efficient data transfer, so PyArrow is natively supported.
+
+You can use the `DeltaLakePyArrowIOManager` in a object as in [Step 1](using-deltalake-with-dagster#step-1-configure-the-delta-lake-io-manager) of the [Using Dagster with Delta Lake tutorial](using-deltalake-with-dagster).
+
+
+
+
+
+
+## Configuring storage backends
+
+The deltalake library comes with support for many storage backends out of the box. Which exact storage is to be used, is derived from the URL of a storage location.
+
+### S3 compatible storages
+
+The S3 APIs are implemented by a number of providers and it is possible to interact with many of them. However, most S3 implementations do not offer support for atomic operations, which is a requirement for multi writer support. As such some additional setup and configuration is required.
+
+
+
+
+In case there will always be only a single writer to a table - this includes no concurrent dagster jobs writing to the same table - you can allow unsafe writes to the table.
+
+```py
+from dagster_deltalake import S3Config
+
+config = S3Config(allow_unsafe_rename=True)
+```
+
+
+
+
+
+To use DynamoDB, set the `AWS_S3_LOCKING_PROVIDER` variable to `dynamodb` and create a table named delta_rs_lock_table in Dynamo. An example DynamoDB table creation snippet using the aws CLI follows, and should be customized for your environment’s needs (e.g. read/write capacity modes):
+
+```bash
+aws dynamodb create-table --table-name delta_rs_lock_table \
+ --attribute-definitions \
+ AttributeName=key,AttributeType=S \
+ --key-schema \
+ AttributeName=key,KeyType=HASH \
+ --provisioned-throughput \
+ ReadCapacityUnits=10,WriteCapacityUnits=10
+```
+
+:::
+
+The delta-rs community is actively working on extending the available options for locking backends. This includes locking backends compatible with Databricks to allow concurrent writes from Databricks and external environments.
+
+:::
+
+
+
+
+
+Cloudflare R2 storage has built-in support for atomic copy operations. This can be leveraged by sending additional headers with the copy requests.
+
+```py
+from dagster_deltalake import S3Config
+
+config = S3Config(copy_if_not_exists="header: cf-copy-destination-if-none-match: *")
+```
+
+
+
+
+
+In cases where non-AWS S3 implementations are used, the endpoint URL or the S3 service needs to be provided.
+
+```py
+config = S3Config(endpoint="https://")
+```
+
+### Working with locally running storage (emulators)
+
+A common pattern for e.g. integration tests is to run a storage emulator like Azurite, Localstack, o.a. If not configured to use TLS, we need to configure the http client, to allow for http traffic.
+
+```py
+config = AzureConfig(use_emulator=True, client=ClientConfig(allow_http=True))
+```
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/deltalake/using-deltalake-with-dagster.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/deltalake/using-deltalake-with-dagster.md
new file mode 100644
index 0000000000000..3e83c08e46c2f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/deltalake/using-deltalake-with-dagster.md
@@ -0,0 +1,95 @@
+---
+title: "Using Delta Lake with Dagster"
+description: Store your Dagster assets in a Delta Lake
+sidebar_position: 100
+---
+
+This tutorial focuses on how to store and load Dagster [asset definitions](/guides/build/assets/defining-assets) in a Delta Lake.
+
+By the end of the tutorial, you will:
+
+- Configure a Delta Lake I/O manager
+- Create a table in Delta Lake using a Dagster asset
+- Make a Delta Lake table available in Dagster
+- Load Delta tables in downstream assets
+
+While this guide focuses on storing and loading Pandas DataFrames in Delta Lakes, Dagster also supports using PyArrow Tables and Polars DataFrames. Learn more about setting up and using the Delta Lake I/O manager with PyArrow Tables and Polars DataFrames in the [Delta Lake reference](reference).
+
+## Prerequisites
+
+To complete this tutorial, you'll need to install the `dagster-deltalake` and `dagster-deltalake-pandas` libraries:
+
+```shell
+pip install dagster-deltalake dagster-deltalake-pandas
+```
+
+## Step 1: Configure the Delta Lake I/O manager
+
+The Delta Lake I/O manager requires some configuration to set up your Delta Lake. You must provide a root path where your Delta tables will be created. Additionally, you can specify a `schema` where the Delta Lake I/O manager will create tables.
+
+
+
+With this configuration, if you materialized an asset called `iris_dataset`, the Delta Lake I/O manager would store the data within a folder `iris/iris_dataset` under the provided root directory `path/to/deltalake`.
+
+Finally, in the object, we assign the to the `io_manager` key. `io_manager` is a reserved key to set the default I/O manager for your assets.
+
+## Step 2: Create Delta Lake tables
+
+The Delta Lake I/O manager can create and update tables for your Dagster-defined assets, but you can also make existing Delta Lake tables available to Dagster.
+
+
+
+
+
+**Store a Dagster asset as a table in Delta Lake**
+
+To store data in Delta Lake using the Delta Lake I/O manager, the definitions of your assets don't need to change. You can tell Dagster to use the Delta Lake I/O manager, like in [Step 1](#step-1-configure-the-delta-lake-io-manager), and Dagster will handle storing and loading your assets in Delta Lake.
+
+
+
+In this example, we first define an [asset](/guides/build/assets/defining-assets). Here, we fetch the Iris dataset as a Pandas DataFrame and rename the columns. The type signature of the function tells the I/O manager what data type it is working with, so it's important to include the return type `pd.DataFrame`.
+
+When Dagster materializes the `iris_dataset` asset using the configuration from [Step 1](#step-1-configure-the-delta-lake-io-manager), the Delta Lake I/O manager will create the table `iris/iris_dataset` if it doesn't exist and replace the contents of the table with the value returned from the `iris_dataset` asset.
+
+
+
+
+
+### Make an existing table available in Dagster
+
+If you already have tables in your Delta Lake, you may want to make them available to other Dagster assets. You can accomplish this by defining [external assets](/guides/build/assets/external-assets) for these tables. By creating an external asset for the existing table, you tell Dagster how to find the table so it can be fetched for downstream assets.
+
+
+
+In this example, we create a for an existing table containing iris harvest data. To make the data available to other Dagster assets, we need to tell the Delta Lake I/O manager how to find the data.
+
+Because we already supplied the database and schema in the I/O manager configuration in [Step 1](#step-1-configure-the-delta-lake-io-manager), we only need to provide the table name. We do this with the `key` parameter in `AssetSpec`. When the I/O manager needs to load the `iris_harvest_data` in a downstream asset, it will select the data in the `iris/iris_harvest_data` folder as a Pandas DataFrame and provide it to the downstream asset.
+
+
+
+
+## Step 3: Load Delta Lake tables in downstream assets
+
+Once you've created an asset that represents a table in your Delta Lake, you will likely want to create additional assets that work with the data. Dagster and the Delta Lake I/O manager allow you to load the data stored in Delta tables into downstream assets.
+
+
+
+In this example, we want to provide the `iris_dataset` asset to the `iris_cleaned` asset. Refer to the Store a Dagster asset as a table in Delta Lake example in [step 2](#step-2-create-delta-lake-tables) for a look at the `iris_dataset` asset.
+
+In `iris_cleaned`, the `iris_dataset` parameter tells Dagster that the value for the `iris_dataset` asset should be provided as input to `iris_cleaned`. If this feels too magical for you, refer to the docs for explicitly specifying dependencies.
+
+When materializing these assets, Dagster will use the `DeltaLakePandasIOManager` to fetch the `iris/iris_dataset` as a Pandas DataFrame and pass the DataFrame as the `iris_dataset` parameter to `iris_cleaned`. When `iris_cleaned` returns a Pandas DataFrame, Dagster will use the `DeltaLakePandasIOManager` to store the DataFrame as the `iris/iris_cleaned` table in your Delta Lake.
+
+## Completed code example
+
+When finished, your code should look like the following:
+
+
+
+## Related
+
+For more Delta Lake features, refer to the [Delta Lake reference](reference).
+
+For more information on asset definitions, see the [Assets documentation](/guides/build/assets/defining-assets).
+
+For more information on I/O managers, refer to the [I/O manager documentation](/guides/build/io-managers/).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dingtalk.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dingtalk.md
new file mode 100644
index 0000000000000..c58bf7182e55f
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dingtalk.md
@@ -0,0 +1,23 @@
+---
+layout: Integration
+status: published
+name: DingTalk
+title: Dagster & DingTalk
+sidebar_label: DingTalk
+excerpt: The community-supported `dagster-dingtalk` package provides an integration with DingTalk.
+date:
+apireflink:
+docslink:
+partnerlink:
+categories:
+enabledBy:
+enables:
+tags: [community-supported]
+sidebar_custom_props:
+ logo: images/integrations/ding.svg
+ community: true
+---
+
+The community-supported `dagster-dingtalk` package provides an integration with DingTalk.
+
+For more information, see the [dagster-dingtalk GitHub repository](https://github.com/sqkkyzx/dagster-dingtalk).
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dlt/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dlt/index.md
new file mode 100644
index 0000000000000..b3ed2c8bd2f8a
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dlt/index.md
@@ -0,0 +1,48 @@
+---
+layout: Integration
+status: published
+name: dlt
+title: Dagster & dlt
+sidebar_label: dlt
+excerpt: Easily ingest and replicate data between systems with dlt through Dagster.
+date: 2024-08-30
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-dlt
+docslink: https://docs.dagster.io/integrations/libraries/dlt/
+partnerlink: https://dlthub.com/
+logo: /integrations/dlthub.jpeg
+categories:
+ - ETL
+enabledBy:
+enables:
+tags: [dagster-supported, etl]
+sidebar_custom_props:
+ logo: images/integrations/dlthub.jpeg
+---
+
+This integration allows you to use [dlt](https://dlthub.com/) to easily ingest and replicate data between systems through Dagster.
+
+:::note
+
+This integration is currently **experimental**.
+
+:::
+
+### Installation
+
+```bash
+pip install dagster-dlt
+```
+
+### Example
+
+
+
+:::note
+
+If you are using the [sql_database](https://dlthub.com/docs/api_reference/sources/sql_database/__init__#sql_database) source, consider setting `defer_table_reflect=True` to reduce database reads. By default, the Dagster daemon will refresh definitions roughly every minute, which will query the database for resource definitions.
+
+:::
+
+### About dlt
+
+[Data Load Tool (dlt)](https://dlthub.com/) is an open source library for creating efficient data pipelines. It offers features like secret management, data structure conversion, incremental updates, and pre-built sources and destinations, simplifying the process of loading messy data into well-structured datasets.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dlt/using-dlt-with-dagster.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dlt/using-dlt-with-dagster.md
new file mode 100644
index 0000000000000..e52e1aff43885
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/dlt/using-dlt-with-dagster.md
@@ -0,0 +1,364 @@
+---
+title: "Using dlt with Dagster"
+description: Ingest data with ease using Dagster and dlt
+---
+
+:::
+
+This feature is considered **experimental**
+
+:::
+
+The [data load tool (dlt)](https://dlthub.com/) open-source library defines a standardized approach for creating data pipelines that load often messy data sources into well-structured data sets. It offers many advanced features, such as:
+
+- Handling connection secrets
+- Converting data into the structure required for a destination
+- Incremental updates and merges
+
+dlt also provides a large collection of [pre-built, verified sources](https://dlthub.com/docs/dlt-ecosystem/verified-sources/) and [destinations](https://dlthub.com/docs/dlt-ecosystem/destinations/), allowing you to write less code (if any!) by leveraging the work of the dlt community.
+
+In this guide, we'll explain how the dlt integration works, how to set up a Dagster project for dlt, and how to use a pre-defined dlt source.
+
+## How it works
+
+The Dagster dlt integration uses [multi-assets](/guides/build/assets/defining-assets#multi-asset), a single definition that results in multiple assets. These assets are derived from the `DltSource`.
+
+The following is an example of a dlt source definition where a source is made up of two resources:
+
+```python
+@dlt.source
+def example(api_key: str = dlt.secrets.value):
+ @dlt.resource(primary_key="id", write_disposition="merge")
+ def courses():
+ response = requests.get(url=BASE_URL + "courses")
+ response.raise_for_status()
+ yield response.json().get("items")
+
+ @dlt.resource(primary_key="id", write_disposition="merge")
+ def users():
+ for page in _paginate(BASE_URL + "users"):
+ yield page
+
+ return courses, users
+```
+
+Each resource queries an API endpoint and yields the data that we wish to load into our data warehouse. The two resources defined on the source will map to Dagster assets.
+
+Next, we defined a dlt pipeline that specifies how we want the data to be loaded:
+
+```python
+pipeline = dlt.pipeline(
+ pipeline_name="example_pipeline",
+ destination="snowflake",
+ dataset_name="example_data",
+ progress="log",
+)
+```
+
+A dlt source and pipeline are the two components required to load data using dlt. These will be the parameters of our multi-asset, which will integrate dlt and Dagster.
+
+## Prerequisites
+
+To follow the steps in this guide, you'll need:
+
+- **To read the [dlt introduction](https://dlthub.com/docs/intro)**, if you've never worked with dlt before.
+- **[To install](/getting-started/installation) the following libraries**:
+
+ ```bash
+ pip install dagster dagster-dlt
+ ```
+
+ Installing `dagster-dlt` will also install the `dlt` package.
+
+## Step 1: Configure your Dagster project to support dlt
+
+The first step is to define a location for the `dlt` code used for ingesting data. We recommend creating a `dlt_sources` directory at the root of your Dagster project, but this code can reside anywhere within your Python project.
+
+Run the following to create the `dlt_sources` directory:
+
+```bash
+cd $DAGSTER_HOME && mkdir dlt_sources
+```
+
+## Step 2: Initialize dlt ingestion code
+
+In the `dlt_sources` directory, you can write ingestion code following the [dlt tutorial](https://dlthub.com/docs/tutorial/load-data-from-an-api) or you can use a verified source.
+
+In this example, we'll use the [GitHub source](https://dlthub.com/docs/dlt-ecosystem/verified-sources/github) provided by dlt.
+
+1. Run the following to create a location for the dlt source code and initialize the GitHub source:
+
+ ```bash
+ cd dlt_sources
+
+ dlt init github snowflake
+ ```
+
+ At which point you'll see the following in the command line:
+
+ ```bash
+ Looking up the init scripts in https://github.com/dlt-hub/verified-sources.git...
+ Cloning and configuring a verified source github (Source that load github issues, pull requests and reactions for a specific repository via customizable graphql query. Loads events incrementally.)
+ ```
+
+2. When prompted to proceed, enter `y`. You should see the following confirming that the GitHub source was added to the project:
+
+ ```bash
+ Verified source github was added to your project!
+ * See the usage examples and code snippets to copy from github_pipeline.py
+ * Add credentials for snowflake and other secrets in ./.dlt/secrets.toml
+ * requirements.txt was created. Install it with:
+ pip3 install -r requirements.txt
+ * Read https://dlthub.com/docs/walkthroughs/create-a-pipeline for more information
+ ```
+
+This downloaded the code required to collect data from the GitHub API. It also created a `requirements.txt` and a `.dlt/` configuration directory. These files can be removed, as we will configure our pipelines through Dagster, however, you may still find it informative to reference.
+
+```bash
+$ tree -a
+.
+├── .dlt # can be removed
+│ ├── .sources
+│ ├── config.toml
+│ └── secrets.toml
+├── .gitignore
+├── github
+│ ├── README.md
+│ ├── __init__.py
+│ ├── helpers.py
+│ ├── queries.py
+│ └── settings.py
+├── github_pipeline.py
+└── requirements.txt # can be removed
+```
+
+## Step 3: Define dlt environment variables
+
+This integration manages connections and secrets using environment variables as `dlt`. The `dlt` library can infer required environment variables used by its sources and resources. Refer to [dlt's Secrets and Configs](https://dlthub.com/docs/general-usage/credentials/configuration) documentation for more information.
+
+In the example we've been using:
+
+- The `github_reactions` source requires a GitHub access token
+- The Snowflake destination requires database connection details
+
+This results in the following required environment variables:
+
+```bash
+SOURCES__GITHUB__ACCESS_TOKEN=""
+DESTINATION__SNOWFLAKE__CREDENTIALS__DATABASE=""
+DESTINATION__SNOWFLAKE__CREDENTIALS__PASSWORD=""
+DESTINATION__SNOWFLAKE__CREDENTIALS__USERNAME=""
+DESTINATION__SNOWFLAKE__CREDENTIALS__HOST=""
+DESTINATION__SNOWFLAKE__CREDENTIALS__WAREHOUSE=""
+DESTINATION__SNOWFLAKE__CREDENTIALS__ROLE=""
+```
+
+Ensure that these variables are defined in your environment, either in your `.env` file when running locally or in the [Dagster deployment's environment variables](/guides/deploy/using-environment-variables-and-secrets).
+
+## Step 4: Define a DagsterDltResource
+
+Next, we'll define a , which provides a wrapper of a dlt pipeline runner. Use the following to define the resource, which can be shared across all dlt pipelines:
+
+```python
+from dagster_dlt import DagsterDltResource
+
+dlt_resource = DagsterDltResource()
+```
+
+We'll add the resource to our in a later step.
+
+## Step 5: Create a dlt_assets definition for GitHub
+
+The decorator takes a `dlt_source` and `dlt_pipeline` parameter. In this example, we used the `github_reactions` source and created a `dlt_pipeline` to ingest data from Github to Snowflake.
+
+In the same file containing your Dagster assets, you can create an instance of your by doing something like the following:
+
+:::
+
+If you are using the [sql_database](https://dlthub.com/docs/api_reference/sources/sql_database/__init__#sql_database) source, consider setting `defer_table_reflect=True` to reduce database reads. By default, the Dagster daemon will refresh definitions roughly every minute, which will query the database for resource definitions.
+
+:::
+
+```python
+from dagster import AssetExecutionContext, Definitions
+from dagster_dlt import DagsterDltResource, dlt_assets
+from dlt import pipeline
+from dlt_sources.github import github_reactions
+
+
+@dlt_assets(
+ dlt_source=github_reactions(
+ "dagster-io", "dagster", max_items=250
+ ),
+ dlt_pipeline=pipeline(
+ pipeline_name="github_issues",
+ dataset_name="github",
+ destination="snowflake",
+ progress="log",
+ ),
+ name="github",
+ group_name="github",
+)
+def dagster_github_assets(context: AssetExecutionContext, dlt: DagsterDltResource):
+ yield from dlt.run(context=context)
+```
+
+## Step 6: Create the Definitions object
+
+The last step is to include the assets and resource in a object. This enables Dagster tools to load everything we've defined:
+
+```python
+defs = Definitions(
+ assets=[
+ dagster_github_assets,
+ ],
+ resources={
+ "dlt": dlt_resource,
+ },
+)
+```
+
+And that's it! You should now have two assets that load data to corresponding Snowflake tables: one for issues and the other for pull requests.
+
+## Advanced usage
+
+### Overriding the translator to customize dlt assets
+
+The object can be used to customize how dlt properties map to Dagster concepts.
+
+For example, to change how the name of the asset is derived, or if you would like to change the key of the upstream source asset, you can override the method.
+
+{/* TODO convert to */}
+```python file=/integrations/dlt/dlt_dagster_translator.py
+import dlt
+from dagster_dlt import DagsterDltResource, DagsterDltTranslator, dlt_assets
+from dagster_dlt.translator import DltResourceTranslatorData
+
+from dagster import AssetExecutionContext, AssetKey, AssetSpec
+
+
+@dlt.source
+def example_dlt_source():
+ def example_resource(): ...
+
+ return example_resource
+
+
+class CustomDagsterDltTranslator(DagsterDltTranslator):
+ def get_asset_spec(self, data: DltResourceTranslatorData) -> AssetSpec:
+ """Overrides asset spec to:
+ - Override asset key to be the dlt resource name,
+ - Override upstream asset key to be a single source asset.
+ """
+ default_spec = super().get_asset_spec(data)
+ return default_spec.replace_attributes(
+ key=AssetKey(f"{data.resource.name}"),
+ deps=[AssetKey("common_upstream_dlt_dependency")],
+ )
+
+
+@dlt_assets(
+ name="example_dlt_assets",
+ dlt_source=example_dlt_source(),
+ dlt_pipeline=dlt.pipeline(
+ pipeline_name="example_pipeline_name",
+ dataset_name="example_dataset_name",
+ destination="snowflake",
+ progress="log",
+ ),
+ dagster_dlt_translator=CustomDagsterDltTranslator(),
+)
+def dlt_example_assets(context: AssetExecutionContext, dlt: DagsterDltResource):
+ yield from dlt.run(context=context)
+```
+
+In this example, we customized the translator to change how the dlt assets' names are defined. We also hard-coded the asset dependency upstream of our assets to provide a fan-out model from a single dependency to our dlt assets.
+
+### Assigning metadata to upstream external assets
+
+A common question is how to define metadata on the external assets upstream of the dlt assets.
+
+This can be accomplished by defining a with a key that matches the one defined in the method.
+
+For example, let's say we have defined a set of dlt assets named `thinkific_assets`, we can iterate over those assets and derive a with attributes like `group_name`.
+
+{/* TODO convert to */}
+```python file=/integrations/dlt/dlt_source_assets.py
+import dlt
+from dagster_dlt import DagsterDltResource, dlt_assets
+
+from dagster import AssetExecutionContext, AssetSpec
+
+
+@dlt.source
+def example_dlt_source():
+ def example_resource(): ...
+
+ return example_resource
+
+
+@dlt_assets(
+ dlt_source=example_dlt_source(),
+ dlt_pipeline=dlt.pipeline(
+ pipeline_name="example_pipeline_name",
+ dataset_name="example_dataset_name",
+ destination="snowflake",
+ progress="log",
+ ),
+)
+def example_dlt_assets(context: AssetExecutionContext, dlt: DagsterDltResource):
+ yield from dlt.run(context=context)
+
+
+thinkific_source_assets = [
+ AssetSpec(key, group_name="thinkific") for key in example_dlt_assets.dependency_keys
+]
+```
+
+### Using partitions in your dlt assets
+
+While still an experimental feature, it is possible to use partitions within your dlt assets. However, it should be noted that this may result in concurrency related issues as state is managed by dlt. For this reason, it is recommended to set concurrency limits for your partitioned dlt assets. See the [Limiting concurrency in data pipelines](/guides/operate/managing-concurrency) guide for more details.
+
+That said, here is an example of using static named partitions from a dlt source.
+
+{/* TODO convert to */}
+```python file=/integrations/dlt/dlt_partitions.py
+from typing import Optional
+
+import dlt
+from dagster_dlt import DagsterDltResource, dlt_assets
+
+from dagster import AssetExecutionContext, StaticPartitionsDefinition
+
+color_partitions = StaticPartitionsDefinition(["red", "green", "blue"])
+
+
+@dlt.source
+def example_dlt_source(color: Optional[str] = None):
+ def load_colors():
+ if color:
+ # partition-specific processing
+ ...
+ else:
+ # non-partitioned processing
+ ...
+
+
+@dlt_assets(
+ dlt_source=example_dlt_source(),
+ name="example_dlt_assets",
+ dlt_pipeline=dlt.pipeline(
+ pipeline_name="example_pipeline_name",
+ dataset_name="example_dataset_name",
+ destination="snowflake",
+ ),
+ partitions_def=color_partitions,
+)
+def compute(context: AssetExecutionContext, dlt: DagsterDltResource):
+ color = context.partition_key
+ yield from dlt.run(context=context, dlt_source=example_dlt_source(color=color))
+```
+
+## What's next?
+
+Want to see real-world examples of dlt in production? Check out how we use it internally at Dagster in the [Dagster Open Platform](https://github.com/dagster-io/dagster-open-platform) project.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/docker.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/docker.md
new file mode 100644
index 0000000000000..54a5b51b2867b
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/docker.md
@@ -0,0 +1,41 @@
+---
+layout: Integration
+status: published
+name: Docker
+title: Dagster & Docker
+sidebar_label: Docker
+excerpt: Run runs external processes in docker containers directly from Dagster.
+date: 2022-11-07
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-docker
+docslink:
+partnerlink: https://www.docker.com/
+logo: /integrations/Docker.svg
+categories:
+ - Compute
+enabledBy:
+enables:
+tags: [dagster-supported, compute]
+sidebar_custom_props:
+ logo: images/integrations/docker.svg
+---
+
+The `dagster-docker` integration library provides the `PipesDockerClient` resource, enabling you to launch Docker containers and execute external code directly from Dagster assets and ops. This integration allows you to pass parameters to Docker containers while Dagster receives real-time events, such as logs, asset checks, and asset materializations, from the initiated jobs. With minimal code changes required on the job side, this integration is both efficient and easy to implement.
+
+### Installation
+
+```bash
+pip install dagster-docker
+```
+
+### Example
+
+
+
+### Deploying to Docker?
+
+- Deploying to Dagster+: Use with a Dagster+ Hybrid deployment, the Docker agent executes Dagster jobs on a Docker cluster. Checkout the [Dagster+ Docker Agent](https://docs.dagster.io/dagster-plus/deployment/deployment-types/hybrid/docker/) guide for more information.
+- Deploying to Open Source: Visit the [Deploying Dagster to Docker](https://docs.dagster.io/guides/deploy/deployment-options/docker) guide for more information.
+
+### About Docker
+
+**Docker** is a set of platform-as-a-service products that use OS-level virtualization to deliver software in packages called containers. The service has both free and premium tiers. The software that hosts the containers is called Docker Engine.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/duckdb/index.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/duckdb/index.md
new file mode 100644
index 0000000000000..c7c199e37bb35
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/duckdb/index.md
@@ -0,0 +1,35 @@
+---
+layout: Integration
+status: published
+name: DuckDB
+title: Dagster & DuckDB
+sidebar_label: DuckDB
+excerpt: Read and write natively to DuckDB from Software Defined Assets.
+date: 2022-11-07
+apireflink: https://docs.dagster.io/api/python-api/libraries/dagster-duckdb
+docslink: https://docs.dagster.io/integrations/libraries/duckdb/
+partnerlink: https://duckdb.org/
+categories:
+ - Storage
+enabledBy:
+enables:
+tags: [dagster-supported, storage]
+sidebar_custom_props:
+ logo: images/integrations/duckdb.svg
+---
+
+This library provides an integration with the DuckDB database, and allows for an out-of-the-box [I/O Manager](/guides/build/io-managers/) so that you can make DuckDB your storage of choice.
+
+### Installation
+
+```bash
+pip install dagster-duckdb
+```
+
+### Example
+
+
+
+### About DuckDB
+
+**DuckDB** is a column-oriented in-process OLAP database. A typical OLTP relational database like SQLite is row-oriented. In row-oriented database, data is organised physically as consecutive tuples.
diff --git a/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/duckdb/reference.md b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/duckdb/reference.md
new file mode 100644
index 0000000000000..127de67758b18
--- /dev/null
+++ b/docs/docs-beta/versioned_docs/version-1.9.10/integrations/libraries/duckdb/reference.md
@@ -0,0 +1,590 @@
+---
+title: "dagster-duckdb integration reference"
+description: Store your Dagster assets in DuckDB
+sidebar_position: 200
+---
+
+This reference page provides information for working with [`dagster-duckdb`](/api/python-api/libraries/dagster-duckdb) features that are not covered as part of the [Using Dagster with DuckDB tutorial](using-duckdb-with-dagster).
+
+DuckDB resource:
+
+- [Executing custom SQL queries](#executing-custom-sql-queries)
+
+DuckDB I/O manager:
+
+- [Selecting specific columns in a downstream asset](#selecting-specific-columns-in-a-downstream-asset)
+- [Storing partitioned assets](#storing-partitioned-assets)
+- [Storing tables in multiple schemas](#storing-tables-in-multiple-schemas)
+- [Using the DuckDB I/O manager with other I/O managers](#using-the-duckdb-io-manager-with-other-io-managers)
+- [Storing and loading PySpark or Polars DataFrames in DuckDB](#storing-and-loading-pyspark-or-polars-dataframes-in-duckdb)
+- [Storing multiple DataFrame types in DuckDB](#storing-multiple-dataframe-types-in-duckdb)
+
+## DuckDB resource
+
+The DuckDB resource provides access to a [`duckdb.DuckDBPyConnection`](https://duckdb.org/docs/api/python/reference/#duckdb.DuckDBPyConnection) object. This allows you full control over how your data is stored and retrieved in your database.
+
+For further information on the DuckDB resource, see the [DuckDB resource API docs](/api/python-api/libraries/dagster-duckdb#dagster_duckdb.DuckDBResource).
+
+### Executing custom SQL queries
+
+{/* TODO convert to */}
+```python file=/integrations/duckdb/reference/resource.py startafter=start endbefore=end
+from dagster_duckdb import DuckDBResource
+
+from dagster import asset
+
+# this example executes a query against the iris_dataset table created in Step 2 of the
+# Using Dagster with DuckDB tutorial
+
+
+@asset(deps=[iris_dataset])
+def small_petals(duckdb: DuckDBResource) -> None:
+ with duckdb.get_connection() as conn: # conn is a DuckDBPyConnection
+ conn.execute(
+ "CREATE TABLE iris.small_petals AS SELECT * FROM iris.iris_dataset WHERE"
+ " 'petal_length_cm' < 1 AND 'petal_width_cm' < 1"
+ )
+```
+
+In this example, we attach the DuckDB resource to the `small_petals` asset. In the body of the asset function, we use the `get_connection` context manager on the resource to get a [`duckdb.DuckDBPyConnection`](https://duckdb.org/docs/api/python/reference/#duckdb.DuckDBPyConnection). We can use this connection to execute a custom SQL query against the `iris_dataset` table created in [Step 2: Create tables in DuckDB](using-duckdb-with-dagster#option-1-step-2) of the [Using Dagster with DuckDB tutorial](using-duckdb-with-dagster). When the `duckdb.get_connection` context is exited, the DuckDB connection will be closed.
+
+## DuckDB I/O manager
+
+The DuckDB I/O manager provides several ways to customize how your data is stored and loaded in DuckDB. However, if you find that these options do not provide enough customization for your use case, we recommend using the DuckDB resource to save and load your data. By using the resource, you will have more fine-grained control over how your data is handled, since you have full control over the SQL queries that are executed.
+
+### Selecting specific columns in a downstream asset
+
+Sometimes you may not want to fetch an entire table as the input to a downstream asset. With the DuckDB I/O manager, you can select specific columns to load by supplying metadata on the downstream asset.
+
+{/* TODO convert to */}
+```python file=/integrations/duckdb/reference/downstream_columns.py
+import pandas as pd
+
+from dagster import AssetIn, asset
+
+# this example uses the iris_dataset asset from Step 2 of the Using Dagster with DuckDB tutorial
+
+
+@asset(
+ ins={
+ "iris_sepal": AssetIn(
+ key="iris_dataset",
+ metadata={"columns": ["sepal_length_cm", "sepal_width_cm"]},
+ )
+ }
+)
+def sepal_data(iris_sepal: pd.DataFrame) -> pd.DataFrame:
+ iris_sepal["sepal_area_cm2"] = (
+ iris_sepal["sepal_length_cm"] * iris_sepal["sepal_width_cm"]
+ )
+ return iris_sepal
+```
+
+In this example, we only use the columns containing sepal data from the `IRIS_DATASET` table created in [Step 2: Create tables in DuckDB](using-duckdb-with-dagster#option-2-step-2) of the [Using Dagster with DuckDB tutorial](using-duckdb-with-dagster). To select specific columns, we can add metadata to the input asset. We do this in the `metadata` parameter of the `AssetIn` that loads the `iris_dataset` asset in the `ins` parameter. We supply the key `columns` with a list of names of the columns we want to fetch.
+
+When Dagster materializes `sepal_data` and loads the `iris_dataset` asset using the DuckDB I/O manager, it will only fetch the `sepal_length_cm` and `sepal_width_cm` columns of the `IRIS.IRIS_DATASET` table and pass them to `sepal_data` as a Pandas DataFrame.
+
+### Storing partitioned assets
+
+The DuckDB I/O manager supports storing and loading partitioned data. To correctly store and load data from the DuckDB table, the DuckDB I/O manager needs to know which column contains the data defining the partition bounds. The DuckDB I/O manager uses this information to construct the correct queries to select or replace the data.
+
+In the following sections, we describe how the I/O manager constructs these queries for different types of partitions.
+
+
+
+
+To store static partitioned assets in DuckDB, specify `partition_expr` metadata on the asset to tell the DuckDB I/O manager which column contains the partition data:
+
+{/* TODO convert to */}
+```python file=/integrations/duckdb/reference/static_partition.py startafter=start_example endbefore=end_example
+import pandas as pd
+
+from dagster import AssetExecutionContext, StaticPartitionsDefinition, asset
+
+
+@asset(
+ partitions_def=StaticPartitionsDefinition(
+ ["Iris-setosa", "Iris-virginica", "Iris-versicolor"]
+ ),
+ metadata={"partition_expr": "SPECIES"},
+)
+def iris_dataset_partitioned(context: AssetExecutionContext) -> pd.DataFrame:
+ species = context.partition_key
+
+ full_df = pd.read_csv(
+ "https://docs.dagster.io/assets/iris.csv",
+ names=[
+ "sepal_length_cm",
+ "sepal_width_cm",
+ "petal_length_cm",
+ "petal_width_cm",
+ "species",
+ ],
+ )
+
+ return full_df[full_df["Species"] == species]
+
+
+@asset
+def iris_cleaned(iris_dataset_partitioned: pd.DataFrame):
+ return iris_dataset_partitioned.dropna().drop_duplicates()
+```
+
+Dagster uses the `partition_expr` metadata to craft the `SELECT` statement when loading the partition in the downstream asset. When loading a static partition (or multiple static partitions), the following statement is used:
+
+```sql
+SELECT *
+ WHERE [partition_expr] in ([selected partitions])
+```
+
+When the `partition_expr` value is injected into this statement, the resulting SQL query must follow DuckDB's SQL syntax. Refer to the [DuckDB documentation](https://duckdb.org/docs/sql/query_syntax/select) for more information.
+
+{/* TODO fix link A partition must be selected when materializing the above assets, as described in the [Materializing partitioned assets](/concepts/partitions-schedules-sensors/partitioning-assets#materializing-partitioned-assets) documentation. */} A partition must be selected when materializing the above assets. In this example, the query used when materializing the `Iris-setosa` partition of the above assets would be:
+
+```sql
+SELECT *
+ WHERE SPECIES in ('Iris-setosa')
+```
+
+
+
+
+Like static partitioned assets, you can specify `partition_expr` metadata on the asset to tell the DuckDB I/O manager which column contains the partition data:
+
+{/* TODO convert to */}
+```python file=/integrations/duckdb/reference/time_partition.py startafter=start_example endbefore=end_example
+import pandas as pd
+
+from dagster import AssetExecutionContext, DailyPartitionsDefinition, asset
+
+
+@asset(
+ partitions_def=DailyPartitionsDefinition(start_date="2023-01-01"),
+ metadata={"partition_expr": "TO_TIMESTAMP(TIME)"},
+)
+def iris_data_per_day(context: AssetExecutionContext) -> pd.DataFrame:
+ partition = context.partition_key
+
+ # get_iris_data_for_date fetches all of the iris data for a given date,
+ # the returned dataframe contains a column named 'time' with that stores
+ # the time of the row as an integer of seconds since epoch
+ return get_iris_data_for_date(partition)
+
+
+@asset
+def iris_cleaned(iris_data_per_day: pd.DataFrame):
+ return iris_data_per_day.dropna().drop_duplicates()
+```
+
+Dagster uses the `partition_expr` metadata to craft the `SELECT` statement when loading the correct partition in the downstream asset. When loading a dynamic partition, the following statement is used:
+
+```sql
+SELECT *
+ WHERE [partition_expr] >= [partition_start]
+ AND [partition_expr] < [partition_end]
+```
+
+When the `partition_expr` value is injected into this statement, the resulting SQL query must follow DuckDB's SQL syntax. Refer to the [DuckDB documentation](https://duckdb.org/docs/sql/query_syntax/select) for more information.
+
+{/* TODO fix link: A partition must be selected when materializing the above assets, as described in the [Materializing partitioned assets](/concepts/partitions-schedules-sensors/partitioning-assets#materializing-partitioned-assets) documentation. */} A partition must be selected when materializing assets. The `[partition_start]` and `[partition_end]` bounds are of the form `YYYY-MM-DD HH:MM:SS`. In this example, the query when materializing the `2023-01-02` partition of the above assets would be:
+
+```sql
+SELECT *
+ WHERE TO_TIMESTAMP(TIME) >= '2023-01-02 00:00:00'
+ AND TO_TIMESTAMP(TIME) < '2023-01-03 00:00:00'
+```
+
+In this example, the data in the `TIME` column are integers, so the `partition_expr` metadata includes a SQL statement to convert integers to timestamps. A full list of DuckDB functions can be found [here](https://duckdb.org/docs/sql/functions/overview).
+
+
+
+
+The DuckDB I/O manager can also store data partitioned on multiple dimensions. To do this, specify the column for each partition as a dictionary of `partition_expr` metadata:
+
+{/* TODO convert to */}
+```python file=/integrations/duckdb/reference/multi_partition.py startafter=start_example endbefore=end_example
+import pandas as pd
+
+from dagster import (
+ AssetExecutionContext,
+ DailyPartitionsDefinition,
+ MultiPartitionsDefinition,
+ StaticPartitionsDefinition,
+ asset,
+)
+
+
+@asset(
+ partitions_def=MultiPartitionsDefinition(
+ {
+ "date": DailyPartitionsDefinition(start_date="2023-01-01"),
+ "species": StaticPartitionsDefinition(
+ ["Iris-setosa", "Iris-virginica", "Iris-versicolor"]
+ ),
+ }
+ ),
+ metadata={"partition_expr": {"date": "TO_TIMESTAMP(TIME)", "species": "SPECIES"}},
+)
+def iris_dataset_partitioned(context: AssetExecutionContext) -> pd.DataFrame:
+ partition = context.partition_key.keys_by_dimension
+ species = partition["species"]
+ date = partition["date"]
+
+ # get_iris_data_for_date fetches all of the iris data for a given date,
+ # the returned dataframe contains a column named 'time' with that stores
+ # the time of the row as an integer of seconds since epoch
+ full_df = get_iris_data_for_date(date)
+
+ return full_df[full_df["species"] == species]
+
+
+@asset
+def iris_cleaned(iris_dataset_partitioned: pd.DataFrame):
+ return iris_dataset_partitioned.dropna().drop_duplicates()
+```
+
+Dagster uses the `partition_expr` metadata to craft the `SELECT` statement when loading the correct partition in a downstream asset. For multi-partitions, Dagster concatenates the `WHERE` statements described in the above sections to craft the correct `SELECT` statement.
+
+{/* TODO fix link: A partition must be selected when materializing the above assets, as described in the [Materializing partitioned assets](/concepts/partitions-schedules-sensors/partitioning-assets#materializing-partitioned-assets) documentation. */} A partition must be selected when materializing assets. For example, when materializing the `2023-01-02|Iris-setosa` partition of the above assets, the following query will be used:
+
+```sql
+SELECT *
+ WHERE SPECIES in ('Iris-setosa')
+ AND TO_TIMESTAMP(TIME) >= '2023-01-02 00:00:00'
+ AND TO_TIMESTAMP(TIME) < '2023-01-03 00:00:00'
+```
+
+In this example, the data in the `TIME` column are integers, so the `partition_expr` metadata includes a SQL statement to convert integers to timestamps. A full list of DuckDB functions can be found [here](https://duckdb.org/docs/sql/functions/overview).
+
+
+
+
+### Storing tables in multiple schemas
+
+You may want to have different assets stored in different DuckDB schemas. The DuckDB I/O manager allows you to specify the schema in several ways.
+
+You can specify the default schema where data will be stored as configuration to the I/O manager, as we did in [Step 1: Configure the DuckDB I/O manager](using-duckdb-with-dagster#step-1-configure-the-duckdb-io-manager) of the [Using Dagster with DuckDB tutorial](using-duckdb-with-dagster).
+
+If you want to store assets in different schemas, you can specify the schema as metadata:
+
+{/* TODO convert to */}
+```python file=/integrations/duckdb/reference/schema.py startafter=start_metadata endbefore=end_metadata dedent=4
+daffodil_dataset = AssetSpec(
+ key=["daffodil_dataset"], metadata={"schema": "daffodil"}
+)
+
+@asset(metadata={"schema": "iris"})
+def iris_dataset() -> pd.DataFrame:
+ return pd.read_csv(
+ "https://docs.dagster.io/assets/iris.csv",
+ names=[
+ "sepal_length_cm",
+ "sepal_width_cm",
+ "petal_length_cm",
+ "petal_width_cm",
+ "species",
+ ],
+ )
+```
+
+You can also specify the schema as part of the asset's key:
+
+{/* TODO convert to */}
+```python file=/integrations/duckdb/reference/schema.py startafter=start_asset_key endbefore=end_asset_key dedent=4
+daffodil_dataset = AssetSpec(key=["daffodil", "daffodil_dataset"])
+
+@asset(key_prefix=["iris"])
+def iris_dataset() -> pd.DataFrame:
+ return pd.read_csv(
+ "https://docs.dagster.io/assets/iris.csv",
+ names=[
+ "sepal_length_cm",
+ "sepal_width_cm",
+ "petal_length_cm",
+ "petal_width_cm",
+ "species",
+ ],
+ )
+```
+
+In this example, the `iris_dataset` asset will be stored in the `IRIS` schema, and the `daffodil_dataset` asset will be found in the `DAFFODIL` schema.
+
+:::
+
+ The schema is determined in this order:
+
+
If the schema is set via metadata, that schema will be used
+
+ Otherwise, the schema set as configuration on the I/O manager will be used
+
+
+ Otherwise, if there is a key_prefix, that schema will be used
+
+
+ If none of the above are provided, the default schema will be PUBLIC
+
+
+
+:::
+
+### Using the DuckDB I/O manager with other I/O managers
+
+You may have assets that you don't want to store in DuckDB. You can provide an I/O manager to each asset using the `io_manager_key` parameter in the decorator:
+
+{/* TODO convert to */}
+```python file=/integrations/duckdb/reference/multiple_io_managers.py startafter=start_example endbefore=end_example
+import pandas as pd
+from dagster_aws.s3.io_manager import s3_pickle_io_manager
+from dagster_duckdb_pandas import DuckDBPandasIOManager
+
+from dagster import Definitions, asset
+
+
+@asset(io_manager_key="warehouse_io_manager")
+def iris_dataset() -> pd.DataFrame:
+ return pd.read_csv(
+ "https://docs.dagster.io/assets/iris.csv",
+ names=[
+ "sepal_length_cm",
+ "sepal_width_cm",
+ "petal_length_cm",
+ "petal_width_cm",
+ "species",
+ ],
+ )
+
+
+@asset(io_manager_key="blob_io_manager")
+def iris_plots(iris_dataset):
+ # plot_data is a function we've defined somewhere else
+ # that plots the data in a DataFrame
+ return plot_data(iris_dataset)
+
+
+defs = Definitions(
+ assets=[iris_dataset, iris_plots],
+ resources={
+ "warehouse_io_manager": DuckDBPandasIOManager(
+ database="path/to/my_duckdb_database.duckdb",
+ schema="IRIS",
+ ),
+ "blob_io_manager": s3_pickle_io_manager,
+ },
+)
+```
+
+In this example:
+
+- The `iris_dataset` asset uses the I/O manager bound to the key `warehouse_io_manager` and `iris_plots` uses the I/O manager bound to the key `blob_io_manager`
+- In the object, we supply the I/O managers for those keys
+- When the assets are materialized, the `iris_dataset` will be stored in DuckDB, and `iris_plots` will be saved in Amazon S3
+
+### Storing and loading PySpark or Polars DataFrames in DuckDB
+
+The DuckDB I/O manager also supports storing and loading PySpark and Polars DataFrames.
+
+
+
+
+To use the , first install the package:
+
+```shell
+pip install dagster-duckdb-pyspark
+```
+
+Then you can use the `DuckDBPySparkIOManager` in your as in [Step 1: Configure the DuckDB I/O manager](using-duckdb-with-dagster#step-1-configure-the-duckdb-io-manager) of the [Using Dagster with DuckDB tutorial](using-duckdb-with-dagster).
+
+{/* TODO convert to */}
+```python file=/integrations/duckdb/reference/pyspark_configuration.py startafter=start_configuration endbefore=end_configuration
+from dagster_duckdb_pyspark import DuckDBPySparkIOManager
+
+from dagster import Definitions
+
+defs = Definitions(
+ assets=[iris_dataset],
+ resources={
+ "io_manager": DuckDBPySparkIOManager(
+ database="path/to/my_duckdb_database.duckdb", # required
+ schema="IRIS", # optional, defaults to PUBLIC
+ )
+ },
+)
+```
+
+The `DuckDBPySparkIOManager` requires an active `SparkSession`. You can either create your own `SparkSession` or use the .
+
+
+