diff --git a/docs/content/_navigation.json b/docs/content/_navigation.json
index 72525e46e0cf5..01fd94beaa122 100644
--- a/docs/content/_navigation.json
+++ b/docs/content/_navigation.json
@@ -170,8 +170,8 @@
"path": "/concepts/partitions-schedules-sensors/sensors"
},
{
- "title": "Auto-materialization policies",
- "path": "/concepts/assets/asset-auto-execution"
+ "title": "Declarative Automation (Experimental)",
+ "path": "/concepts/automation/declarative-automation"
},
{
"title": "Asset Sensors",
diff --git a/docs/content/concepts.mdx b/docs/content/concepts.mdx
index 22879639269ca..c200c5093657e 100644
--- a/docs/content/concepts.mdx
+++ b/docs/content/concepts.mdx
@@ -67,8 +67,8 @@ Dagster offers several ways to run data pipelines without manual intervention, i
href="/concepts/partitions-schedules-sensors/sensors"
>
-
-You can set up Dagster to automatically materialize assets when criteria are met. This enables a declarative approach to asset scheduling – instead of defining imperative workflows to materialize your assets, you just describe the conditions under which they should be materialized.
-
-At a high-level, the most common way for assets to be auto-materialized is "eagerly" -- immediately after upstream changes occur, a run will be kicked off to incorporate those changes into a given asset. However, the precise rules that govern when runs are kicked off can be customized on an asset-by-asset basis.
-
-## Turning on auto-materializing
-
-To enable assets to be automatically materialized, you need to first flip a toggle in the Dagster UI.
-
-- If you're using an open source Dagster deployment, you can get to this toggle by clicking "Deployment" in the top navigation pane and then clicking on the "Daemons" tab.
-- If you're using Dagster+, you can get to this toggle by clicking "Deployment" in the top navigation pane, then clicking on the "Agents" tab, then looking under "Cloud service statuses".
-
-## Auto-materialize policies
-
-You can set up an asset to be auto-materialized by assigning it an . Each policy consists of a set of `AutoMaterializeRules`, each representing individual reasons that an asset should be materialized or not at a given point in time. If there's at least one rule determining that the asset should be materialized, and no rules determining that it should be skipped, a run will be launched to materialize that asset.
-
-It is recommended to start with the built-in and further customize from there if necessary. This policy consists of all of the supported rules, other than `materialize_on_cron` and `skip_on_not_all_parents_updated`. The supported rules are currently:
-
-| Name | Description |
-| ---------------------------------------------------------------------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------ |
-| | Materialize an asset partition if one of its parents has been updated more recently than it has. |
-| | Materialize an asset partition if it has never been materialized before. |
-| | Materialize an asset partition if it has not been materialized since the latest tick of a given cron schedule. |
-| | Materialize an asset if it is required to satisfy a of this asset or one of its downstream assets. |
-| | Skip materializing an asset partition if one of its parent asset partitions has never been materialized (for regular assets) or observed (for observable source assets). |
-| | Skip materializing an asset partition if any of its parents has not incorporated the latest data from its ancestors. |
-| | Skip materializing an asset partition if any of its parents have not been updated since the asset's last materialization. |
-
-In this example, we use to indicate that, any time that `asset1` is materialized, `asset2` should be automatically materialized right after:
-
-```python file=/concepts/assets/auto_materialize_eager.py
-from dagster import AutoMaterializePolicy, asset
-
-
-@asset
-def asset1(): ...
-
-
-@asset(auto_materialize_policy=AutoMaterializePolicy.eager(), deps=[asset1])
-def asset2(): ...
-```
-
-This example assumes that `asset1` will be materialized in some other way - e.g. manually, via a [sensor](/concepts/partitions-schedules-sensors/sensors), or via a [schedule](/concepts/automation/schedules).
-
-### Adding an auto-materialize policy to multiple assets at once
-
-If you want to apply the same to a set of assets, you can use the `auto_materialize_policy` argument when loading them with functions like and .
-
-```python file=/concepts/assets/auto_materialize_multiple.py
-from dagster import (
- AutoMaterializePolicy,
- Definitions,
- asset,
- load_assets_from_current_module,
-)
-
-
-@asset
-def asset1(): ...
-
-
-@asset(deps=[asset1])
-def asset2(): ...
-
-
-defs = Definitions(
- assets=load_assets_from_current_module(
- auto_materialize_policy=AutoMaterializePolicy.eager(),
- )
-)
-```
-
-### Run tags
-
-Runs triggered by auto-materialize policies are tagged with `dagster/auto_materialize: true`. Additional tags can be configured in [`dagster.yaml` (OSS)](/deployment/dagster-instance#auto-materialize) or [deployment settings (Dagster+)](/dagster-plus/managing-deployments/deployment-settings-reference#auto-materialize).
-
-### Customizing auto-materialize policies
-
-Auto-materialize policies can be customized by adding or removing rules. These changes will be reflected in the UI for individual assets.
-
-#### Auto-materialize only once all parents have been updated
-
-By default, the eager policy will materialize an asset whenever _any_ of its parents have been updated. In cases where an asset has many parents, this may cause more materializations than desired, as each parent update will result in an additional downstream materialization. To avoid this, the `skip_on_not_all_parents_updated` rule can be applied to a given policy to force it to wait until _all_ of an asset's parents have been updated before materializing it.
-
-```python file=/concepts/assets/auto_materialize_after_all_parents.py
-from dagster import AutoMaterializePolicy, AutoMaterializeRule, asset
-
-wait_for_all_parents_policy = AutoMaterializePolicy.eager().with_rules(
- AutoMaterializeRule.skip_on_not_all_parents_updated()
-)
-
-
-@asset(auto_materialize_policy=wait_for_all_parents_policy)
-def asset1(upstream1, upstream2): ...
-```
-
-#### Auto-materialize even if some parents are missing
-
-By default, the eager policy won't materialize an asset if any of its parents are missing. In some cases, it's desirable to allow the downstream asset to be materialized, even if some of its parent assets/partitions are missing. To enable this, the `skip_on_parent_missing` rule can be removed from a given policy to prevent this from blocking the materialization of an asset.
-
-```python file=/concepts/assets/auto_materialize_with_missing_parents.py
-from dagster import AutoMaterializePolicy, AutoMaterializeRule, asset
-
-allow_missing_parents_policy = AutoMaterializePolicy.eager().without_rules(
- AutoMaterializeRule.skip_on_parent_missing(),
-)
-
-
-@asset(auto_materialize_policy=allow_missing_parents_policy)
-def asset1(upstream1, upstream2): ...
-```
-
-#### Auto-materialize root assets on a regular cadence
-
-By default, the eager policy will only materialize an asset if it's missing or one of its parents update. This means that an unpartitioned root asset will only get auto-materialized a single time, as it has no parents which can update. In some cases, it's desirable to recompute these assets on a regular basis. To enable this, the `materialize_on_cron` rule can be added to a given policy.
-
-```python file=/concepts/assets/auto_materialize_on_cron.py
-from dagster import AutoMaterializePolicy, AutoMaterializeRule, asset
-
-materialize_on_cron_policy = AutoMaterializePolicy.eager().with_rules(
- # try to materialize this asset if it hasn't been materialized since the last cron tick
- AutoMaterializeRule.materialize_on_cron("0 9 * * *", timezone="US/Central"),
-)
-
-
-@asset(auto_materialize_policy=materialize_on_cron_policy)
-def root_asset(): ...
-```
-
-### Auto-materialization and partitions
-
-Each generally applies individually to each partition of a partitioned asset. Here's a pipeline with two daily-partitioned assets that have eager auto-materialize policies. At the end of each day, a partition for that day will be added to the set of partitions for each of the assets. Dagster will notice that the new partitions exist, but have no materializations, and then auto-materialize them.
-
-```python file=/concepts/assets/auto_materialize_time_partitions.py
-from dagster import AutoMaterializePolicy, DailyPartitionsDefinition, asset
-
-
-@asset(
- partitions_def=DailyPartitionsDefinition(start_date="2020-10-10"),
- auto_materialize_policy=AutoMaterializePolicy.eager(),
-)
-def asset1(): ...
-
-
-@asset(
- partitions_def=DailyPartitionsDefinition(start_date="2020-10-10"),
- auto_materialize_policy=AutoMaterializePolicy.eager(),
- deps=[asset1],
-)
-def asset2(): ...
-```
-
-If the last partition of `asset1` is re-materialized, e.g. manually from the UI, then the corresponding partition of `asset2` will be auto-materialized after.
-
-By default, a given will not allow more than one partition of an asset to be materialized per minute. Any partitions exceeding this threshold will be discarded. Manual intervention will be required to materialize the discarded partitions.
-
-This threshold may be increased as follows:
-
-```python file=/concepts/assets/auto_materialize_max_materializations_per_minute.py
-from dagster import AutoMaterializePolicy, DailyPartitionsDefinition, asset
-
-
-@asset(
- partitions_def=DailyPartitionsDefinition(start_date="2020-10-10"),
- auto_materialize_policy=AutoMaterializePolicy.eager(
- max_materializations_per_minute=7
- ),
-)
-def asset1(): ...
-```
-
-For time-partitioned assets, the `N` most recent partitions will be selected from the set of candidates to be materialized. For other types of partitioned assets, the selection will be random.
-
-### Auto-materialize policies and data versions
-
-[Observable source assets](/concepts/assets/asset-observations#observable-source-assets) are assets that your code doesn't materialize, but that you provide a function for that can tell when they've changed. The rule incorporates the observed data versions of these assets when determining if it should fire for a downstream asset.
-
-In this example, we check every minute to see whether `source_file` was modified. If it was, then the on `asset1` will cause it to be materialized.
-
-```python file=/concepts/assets/auto_materialize_observable_source_asset.py
-import os
-
-from dagster import AutoMaterializePolicy, DataVersion, asset, observable_source_asset
-
-
-@observable_source_asset(auto_observe_interval_minutes=1)
-def source_file():
- return DataVersion(str(os.path.getmtime("source_file.csv")))
-
-
-@asset(
- deps=[source_file],
- auto_materialize_policy=AutoMaterializePolicy.eager(),
-)
-def asset1(): ...
-```
-
-### Auto-materialize sensors
-
-You can break up the evaluation of your auto-materialization policies into multiple [sensors](/concepts/partitions-schedules-sensors/sensors), which can be turned on and off independently and run at different intervals. Additionally, an error during the evaluation of one sensor will not affect the execution of another sensor.
-
-How to enable auto-materialize sensors depends on whether you're using Dagster+ or Dagster Open Source:
-
-- **Dagster+**: Use the [Dagster+ UI or the dagster-cloud CLI](/dagster-plus/managing-deployments/managing-deployments#configuring-deployment-settings) to set `auto_materialize.use_sensors` to `true` in your deployment settings.
-- **Dagster Open Source**: Add the following to your instance's `dagster.yaml` file:
-
- ```yaml
- auto_materialize:
- use_sensors: true
- ```
-
-Once auto-materialize sensors are enabled, a sensor called `default_auto_materialize_sensor` will be created for each code location that has at least one asset with an or `auto_observe_interval_minutes` set, targeting each of those assets. Like any other sensor, auto-materialize sensors can be enabled or disabled from the **Sensors** tab in the Dagster UI.
-
-The default execution interval for auto-materialize sensors is every 30 seconds. You can further customize the mapping of assets to auto-materialize sensors, or the interval at which an auto-materialize sensor runs, by creating s in code. For example, the following auto-materialize sensor targets every eligible asset in its code location, and runs every 15 minutes:
-
-```python file=/concepts/assets/auto_materialize_sensor.py
-from dagster import AssetSelection, AutomationConditionSensorDefinition, Definitions
-
-my_custom_auto_materialize_sensor = AutomationConditionSensorDefinition(
- "my_custom_auto_materialize_sensor",
- asset_selection=AssetSelection.all(include_sources=True),
- minimum_interval_seconds=60 * 15,
-)
-
-defs = Definitions(sensors=[my_custom_auto_materialize_sensor])
-```
-
-Each asset can only be part of one - an exception will be raised if two s attempt to target the same asset. A sensor called `default_auto_materialize_sensor` will be created if there are any assets not targeted by an in code, targeting those assets.
diff --git a/docs/content/concepts/automation.mdx b/docs/content/concepts/automation.mdx
index 85a5c2e88f7d3..2264741566528 100644
--- a/docs/content/concepts/automation.mdx
+++ b/docs/content/concepts/automation.mdx
@@ -41,18 +41,16 @@ You can use sensors to run a job or materialize an asset in response to specific
You can also use sensors to act on the status of a job run. Refer to the [Sensors documentation][sensors] to learn more.
-### Auto-materialize policies
+### Declarative Automation
-If you want a declarative approach to automating your pipelines, Auto-materialize policies (AMP) may be a good fit. AMPs allow you to assign policies to assets and let Dagster determine the best approach to keeping assets up-to-date while adhering to those policies.
+Declarative Automation allows you to automatically materialize assets when specified criteria are met. Using Declarative Automation, you could update assets:
-For example, with AMPs, you can update assets based on:
+- When the asset hasn't yet been materialized
+- When an asset's upstream dependency has been updated
+- After an asset's parents have been updated since a cron tick
+- ... based on your own custom conditions
-- Whether an upstream dependency has been updated
-- Whether an upstream dependency has the latest data from its dependencies
-- Whether a materialization has occured since the last tick of a cron schedule
-- ... and more
-
-AMPs are declared on an asset-by-asset basis, but can be applied to multiple assets at once. Refer to the [Auto-materializing Assets documentation][auto-materialize-policies] to learn more.
+Materialization conditions are declared on an asset-by-asset basis. Refer to the [Declarative Automation documentation][declarative-automation] to learn more.
### Asset Sensors
@@ -83,7 +81,7 @@ The following cheatsheet contains high-level details about each of the automatio
Method
@@ -175,13 +173,13 @@ The following cheatsheet contains high-level details about each of the automatio
|
-
- Auto-materialize policies
+
+ Declarative Automation
|
- Automatically materializes an asset or selection of assets when
- specified criteria (ex: upstream changes) are met
+ Automatically materializes an asset when specified criteria (ex:
+ upstream changes) are met
|
+ This feature is currently experimental.
+
+
+Dagster can automatically materialize assets when criteria are met, enabling a declarative approach to asset materialization. Instead of defining explicit workflows to materialize assets, you describe the conditions under which they should be materialized and let the system kick off runs in response.
+
+For example, you have an asset that's scheduled to execute every day at midnight. Instead of executing whether there's new data or not, you can use Declarative Automation to materialize the asset only after its parent has been updated.
+
+Declarative Automation includes pre-built conditions to handle common use cases, such as executing on a periodic schedule or whenever an upstream dependency updates, but conditions can also be customized.
+
+---
+
+## Benefits
+
+Using Declarative Automation helps you:
+
+- Ensure you're working with the most up-to-date data
+- Optimize resource usage by only materializing assets when needed
+- Simplify how your team understands their assets by consolidating all asset logic to a single location
+- Avoid thinking about specific workflow boundaries, such as a [schedule accounting for timezones or Daylight Savings Time](/concepts/automation/schedules/customizing-executing-timezones)
+
+---
+
+## Prerequisites
+
+Before continuing, you should be familiar with:
+
+- [Asset definitions](/concepts/assets/software-defined-assets)
+- [Sensor definitions](/concepts/partitions-schedules-sensors/sensors)
+- [Code locations](/concepts/code-locations)
+
+---
+
+## How it works
+
+Declarative Automation is an automation method that kicks off runs when criteria are met. This method contains two main components:
+
+- **An automation condition (**), which represents when an individual asset should be executed.
+- **A sensor (**), which evaluates each and launches runs in response to their status.
+
+### Automation conditions
+
+Automation conditions describe the conditions under which an asset should be executed. Dagster provides two pre-built conditions:
+
+
+
+
+
+ Name
+ |
+
+ Description
+ |
+ Useful for |
+
+
+
+
+
+ AutomationCondition.eager()
+ |
+
+ This condition will materialize an asset:
+
+ - If the asset has never been materialized before, or
+ -
+ When the asset's parents update, as long as none of the parents are
+ currently missing or have an update in progress
+
+
+ |
+
+
+ -
+ Automatically propagating changes through the asset graph
+
+ - Ensuring assets remain up-to-date
+
+ |
+
+
+
+ AutomationCondition.cron(cron_schedule)
+ |
+
+ This condition will materialize an asset once per cron schedule tick,
+ after all of its parents have been updated since the tick
+ |
+
+ Regularly updating an asset without worrying about the specifics of how
+ its parents update
+ |
+
+
+
+
+Automation conditions can be set on the decorator or on an :
+
+```python
+from dagster import AssetSpec, AutomationCondition, asset
+
+@asset(automation_condition=AutomationCondition.eager())
+def my_eager_asset(): ...
+
+AssetSpec("my_cron_asset", automation_condition=AutomationCondition.cron("@daily"))
+```
+
+The core framework is extremely flexible, allowing you to build custom conditions from the ground up. Refer to the Customizing automation conditions guide for more information.
+
+### Sensors
+
+When automation conditions for an asset are met, a sensor will kick off a run to materialize the asset. This sensor, named `default_automation_condition_sensor`, will be available for each code location and monitor all assets within that location. To use multiple sensors or change the properties of the default sensor, refer to the documentation.
+
+For an automation condition sensor to run, it must be turned on and an active [`dagster-daemon` process](/deployment/dagster-daemon) must be running. If you used [`dagster dev` to start the Dagster UI/webserver](/guides/running-dagster-locally), the daemon process will be automatically launched alongside the webserver.
+
+After these criteria are met, the sensor's evaluation history will be visible in the UI:
+
+
+
+
+
+You'll also be able to view a detailed history of each asset's evaluations on the asset's [**Asset Details** page](/concepts/webserver/ui#asset-details). This allows you to see why an asset was or wasn't materialized at different points in time:
+
+
+
+
+
+---
+
+## Getting started
+
+To use Declarative Automation, you'll need to enable the automation condition sensor in the Dagster UI:
+
+1. Navigate to **Overview > Sensors**
+2. Locate the desired code location.
+3. In the code location, toggle the `default_automation_condition_sensor` sensor to on.
+
+From here, you can:
+
+- Define custom automation conditions
+- View a history of each evaluation for the sensor
+- Navigate to individual assets to see a history of their evaluations
+
+---
+
+## Related
+
+
+
+
+
+
diff --git a/docs/content/concepts/webserver/ui.mdx b/docs/content/concepts/webserver/ui.mdx
index d205bfe1d278b..09da6f3b1670d 100644
--- a/docs/content/concepts/webserver/ui.mdx
+++ b/docs/content/concepts/webserver/ui.mdx
@@ -163,7 +163,7 @@ height={1514}
- **Events** - The asset's materialization history
- **Checks** - The [Asset checks](/concepts/assets/asset-checks) defined for the asset
- **Lineage** - The asset's lineage in the [**Global asset lineage** page](#global-asset-lineage)
- - **Automation** - The [Auto-materialize policies](/concepts/assets/asset-auto-execution) associated with the asset
+ - **Automation** - The [Declarative Automation conditions](/concepts/automation/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/insights) documentation for more information.
- **Accessed by**: Clicking an asset in the [**Asset catalog**](#asset-catalog)
diff --git a/docs/content/dagster-plus/managing-deployments/deployment-settings-reference.mdx b/docs/content/dagster-plus/managing-deployments/deployment-settings-reference.mdx
index 8e407cd38a3d3..d5a1785d454a8 100644
--- a/docs/content/dagster-plus/managing-deployments/deployment-settings-reference.mdx
+++ b/docs/content/dagster-plus/managing-deployments/deployment-settings-reference.mdx
@@ -33,9 +33,6 @@ run_retries:
max_retries: 0
sso_default_role: EDITOR
-
-auto_materialize:
- run_tags: {}
```
---
@@ -49,7 +46,6 @@ For each deployment, you can configure settings for:
- [Run retries](#run-retries-run_retries)
- [SSO default role](#sso-default-role)
- [Non-isolated runs](#non-isolated-runs)
-- [Auto-materialize](#auto-materialize)
### Run queue (run_queue)
@@ -228,25 +224,3 @@ non_isolated_runs:
-
-### Auto-materialize
-
-Configure [auto-materializing assets](/concepts/assets/asset-auto-execution) for your deployment.
-
-```yaml
-auto_materialize:
- run_tags:
- foo: bar
- use_sensors: False
-```
-
-
-
- Tags to apply to all runs created for an auto-materialize policy.
-
-
-
diff --git a/docs/content/deployment/dagster-instance.mdx b/docs/content/deployment/dagster-instance.mdx
index cfabb136dc6c4..ee7667ae3e2b3 100644
--- a/docs/content/deployment/dagster-instance.mdx
+++ b/docs/content/deployment/dagster-instance.mdx
@@ -791,13 +791,3 @@ schedules:
```
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.
-
-### Auto-materialize
-
-The `auto_materialize` key allows you to adjust configuration related to [auto-materializing assets](/concepts/assets/asset-auto-execution).
-
-```yaml file=/deploying/dagster_instance/dagster.yaml startafter=start_marker_auto_materialize endbefore=end_marker_auto_materialize
-auto_materialize:
- run_tags:
- key: value
-```
diff --git a/docs/content/guides/dagster/managing-ml.mdx b/docs/content/guides/dagster/managing-ml.mdx
index 13736fe71cc1e..4df5b51bca934 100644
--- a/docs/content/guides/dagster/managing-ml.mdx
+++ b/docs/content/guides/dagster/managing-ml.mdx
@@ -28,7 +28,7 @@ You might have thought about your data sources, feature sets, and the best model
Whether you have a large or small model, Dagster can help automate data refreshes and model training based on your business needs.
-[Auto-materializing assets](/concepts/assets/asset-auto-execution) can be used to update a machine learning model when the upstream data is updated. This can be done by setting the `AutoMaterializePolicy` to `eager`, which means that our machine learning model asset will be refreshed anytime our data asset is updated.
+Auto-materializing assets can be used to update a machine learning model when the upstream data is updated. This can be done by setting the `AutoMaterializePolicy` to `eager`, which means that our machine learning model asset will be refreshed anytime our data asset is updated.
```python file=/guides/dagster/managing_ml/managing_ml_code.py startafter=eager_materilization_start endbefore=eager_materilization_end
from dagster import AutoMaterializePolicy, asset
diff --git a/docs/content/integrations/airflow.mdx b/docs/content/integrations/airflow.mdx
index 0f95c9478c73c..41d0d0c53ab73 100644
--- a/docs/content/integrations/airflow.mdx
+++ b/docs/content/integrations/airflow.mdx
@@ -128,8 +128,8 @@ While Airflow and Dagster have some significant differences, there are many conc
-
- Auto-materialization
+
+ Declarative Automation
@@ -144,8 +144,8 @@ While Airflow and Dagster have some significant differences, there are many conc
. When this is the case,
backfills
- and
- auto-materialize
+ and
+ Declarative Automation
will only materialize later partitions after earlier partitions have
completed.
|
diff --git a/docs/content/integrations/airflow/from-airflow-to-dagster.mdx b/docs/content/integrations/airflow/from-airflow-to-dagster.mdx
index b2a5ef9e3fb24..9f00d12238e2d 100644
--- a/docs/content/integrations/airflow/from-airflow-to-dagster.mdx
+++ b/docs/content/integrations/airflow/from-airflow-to-dagster.mdx
@@ -328,8 +328,8 @@ While Airflow and Dagster have some significant differences, there are many conc
-
- Auto-materialization
+
+ Declarative Automation
@@ -344,8 +344,8 @@ While Airflow and Dagster have some significant differences, there are many conc
. When this is the case,
backfills
- and
- auto-materialize
+ and
+ Declarative Automation
will only materialize later partitions after earlier partitions have
completed.
diff --git a/docs/next/public/images/concepts/automation/declarative-automation/default-automation-sensor.png b/docs/next/public/images/concepts/automation/declarative-automation/default-automation-sensor.png
new file mode 100644
index 0000000000000..8dec4cea0b7e5
Binary files /dev/null and b/docs/next/public/images/concepts/automation/declarative-automation/default-automation-sensor.png differ
diff --git a/docs/next/public/images/concepts/automation/declarative-automation/evaluations-asset-details.png b/docs/next/public/images/concepts/automation/declarative-automation/evaluations-asset-details.png
new file mode 100644
index 0000000000000..450520d80bd0b
Binary files /dev/null and b/docs/next/public/images/concepts/automation/declarative-automation/evaluations-asset-details.png differ
diff --git a/docs/next/util/redirectUrls.json b/docs/next/util/redirectUrls.json
index 5ef4a00b65553..1a7d215453a7c 100644
--- a/docs/next/util/redirectUrls.json
+++ b/docs/next/util/redirectUrls.json
@@ -4,6 +4,11 @@
"destination": "/concepts/metadata-tags/asset-metadata",
"statusCode": 302
},
+ {
+ "source": "/concepts/assets/asset-auto-execution",
+ "destination": "/concepts/automation/declarative-automation",
+ "statusCode": 302
+ },
{
"source": "/concepts/assets/software-defined-assets#recording-materialization-metadata",
"destination": "/concepts/metadata-tags/asset-metadata",
diff --git a/examples/docs_snippets/docs_snippets/deploying/dagster_instance/dagster.yaml b/examples/docs_snippets/docs_snippets/deploying/dagster_instance/dagster.yaml
index f281bef13ff14..31bf1a05040f7 100644
--- a/examples/docs_snippets/docs_snippets/deploying/dagster_instance/dagster.yaml
+++ b/examples/docs_snippets/docs_snippets/deploying/dagster_instance/dagster.yaml
@@ -401,10 +401,6 @@ schedules:
# end_marker_schedules
-# start_marker_auto_materialize
-
auto_materialize:
run_tags:
- key: value
-
-# end_marker_auto_materialize
\ No newline at end of file
+ key: value
\ No newline at end of file